You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by ko...@apache.org on 2021/05/23 20:34:20 UTC

[avro] branch master updated: AVRO-3146: Fix Decimal Scale Exception Handling (#1228)

This is an automated email from the ASF dual-hosted git repository.

kojiromike pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/avro.git


The following commit(s) were added to refs/heads/master by this push:
     new c8f953b  AVRO-3146: Fix Decimal Scale Exception Handling (#1228)
c8f953b is described below

commit c8f953bc5e6599b90283992b643bdc91adbef84b
Author: Michael A. Smith <mi...@smith-li.com>
AuthorDate: Sun May 23 16:34:08 2021 -0400

    AVRO-3146: Fix Decimal Scale Exception Handling (#1228)
    
    Avro is supposed to check if the scale for a decimal logical schema is large enough for the datum to be written. But Python provides the exponent as negative, so the exponent is always less than the scale. This change corrects the comparison by ensuring scale and exponent have the same sign.
    
    In testing the above, avro attempts to raise AvroTypeException with improper arguments, leading to a secondary exception within the first. This change also fixes AvroTypeException so that it is flexible and won't crash when invoked with different arguments.
    
    It also adds a special AvroTypeException subtype to provide useful debugging details when a decimal schema tries to write a datum with an exponent that is too large.
    
    Finally, many test cases in Avro consist of running the same code over variations on pairs of schema and datum. But unittest exits at the first test failure! To get a comprehensive view of tests, even when some fail, I have refactored test_io so that individual test methods are generated over the set of test cases.
---
 lang/py/avro/errors.py           |  55 ++-
 lang/py/avro/io.py               |  11 +-
 lang/py/avro/test/test_io.py     | 796 ++++++++++++++++++++++++---------------
 lang/py/avro/test/test_schema.py |   4 +-
 4 files changed, 548 insertions(+), 318 deletions(-)

diff --git a/lang/py/avro/errors.py b/lang/py/avro/errors.py
index 0490f90..0a5a7ed 100644
--- a/lang/py/avro/errors.py
+++ b/lang/py/avro/errors.py
@@ -1,5 +1,4 @@
 #!/usr/bin/env python3
-# -*- mode: python -*-
 # -*- coding: utf-8 -*-
 
 ##
@@ -22,6 +21,14 @@
 import json
 
 
+def _safe_pretty(schema):
+    """Try to pretty-print a schema, but never raise an exception within another exception."""
+    try:
+        return json.dumps(json.loads(str(schema)), indent=2)
+    except Exception:  # Never raise an exception within another exception.
+        return schema
+
+
 class AvroException(Exception):
     """The base class for exceptions in avro."""
 
@@ -44,21 +51,43 @@ class IgnoredLogicalType(AvroWarning):
 
 class AvroTypeException(AvroException):
     """Raised when datum is not an example of schema."""
-    def __init__(self, expected_schema, datum):
-        pretty_expected = json.dumps(json.loads(str(expected_schema)), indent=2)
-        fail_msg = "The datum {} is not an example of the schema {}".format(datum, pretty_expected)
-        super(AvroTypeException, self).__init__(fail_msg)
+
+    def __init__(self, *args):
+        try:
+            expected_schema, datum = args[:2]
+        except (IndexError, ValueError):
+            return super().__init__(*args)
+        return super().__init__(
+            f"The datum {datum} is not an example of the schema {_safe_pretty(expected_schema)}"
+        )
+
+
+class AvroOutOfScaleException(AvroTypeException):
+    """Raised when attempting to write a decimal datum with an exponent too large for the decimal schema."""
+
+    def __init__(self, *args):
+        try:
+            scale, datum, exponent = args[:3]
+        except (IndexError, ValueError):
+            return super().__init__(*args)
+        return super().__init__(
+            f"The exponent of {datum}, {exponent}, is too large for the schema scale of {scale}"
+        )
 
 
 class SchemaResolutionException(AvroException):
-    def __init__(self, fail_msg, writers_schema=None, readers_schema=None):
-        pretty_writers = json.dumps(json.loads(str(writers_schema)), indent=2)
-        pretty_readers = json.dumps(json.loads(str(readers_schema)), indent=2)
-        if writers_schema:
-            fail_msg += "\nWriter's Schema: {}".format(pretty_writers)
-        if readers_schema:
-            fail_msg += "\nReader's Schema: {}".format(pretty_readers)
-        super(AvroException, self).__init__(fail_msg)
+    def __init__(self, fail_msg, writers_schema=None, readers_schema=None, *args):
+        writers_message = (
+            f"\nWriter's Schema: {_safe_pretty(writers_schema)}"
+            if writers_schema
+            else ""
+        )
+        readers_message = (
+            f"\nReader's Schema: {_safe_pretty(readers_schema)}"
+            if readers_schema
+            else ""
+        )
+        super().__init__((fail_msg or "") + writers_message + readers_message, *args)
 
 
 class DataFileException(AvroException):
diff --git a/lang/py/avro/io.py b/lang/py/avro/io.py
index 96459d4..05570bb 100644
--- a/lang/py/avro/io.py
+++ b/lang/py/avro/io.py
@@ -1,5 +1,4 @@
 #!/usr/bin/env python3
-# -*- mode: python -*-
 # -*- coding: utf-8 -*-
 
 ##
@@ -172,6 +171,7 @@ def _iterate_node(node):
 # Iteration #
 #############
 
+
 def _default_iterator(_):
     """Immediately raise StopIteration.
 
@@ -217,6 +217,7 @@ _ITERATORS['error'] = _ITERATORS['request'] = _ITERATORS['record']
 # Decoder/Encoder
 #
 
+
 class BinaryDecoder:
     """Read leaf values."""
 
@@ -490,8 +491,8 @@ class BinaryEncoder:
         signed long is 8, 8 bytes are written.
         """
         sign, digits, exp = datum.as_tuple()
-        if exp > scale:
-            raise avro.errors.AvroTypeException('Scale provided in schema does not match the decimal')
+        if (-1 * exp) > scale:
+            raise avro.errors.AvroOutOfScaleException(scale, datum, exp)
 
         unscaled_datum = 0
         for digit in digits:
@@ -516,8 +517,8 @@ class BinaryEncoder:
         Decimal in fixed are encoded as size of fixed bytes.
         """
         sign, digits, exp = datum.as_tuple()
-        if exp > scale:
-            raise avro.errors.AvroTypeException('Scale provided in schema does not match the decimal')
+        if (-1 * exp) > scale:
+            raise avro.errors.AvroOutOfScaleException(scale, datum, exp)
 
         unscaled_datum = 0
         for digit in digits:
diff --git a/lang/py/avro/test/test_io.py b/lang/py/avro/test/test_io.py
index 4ca8800..9b39382 100644
--- a/lang/py/avro/test/test_io.py
+++ b/lang/py/avro/test/test_io.py
@@ -1,5 +1,4 @@
 #!/usr/bin/env python3
-# -*- mode: python -*-
 # -*- coding: utf-8 -*-
 
 ##
@@ -23,127 +22,199 @@ import binascii
 import datetime
 import decimal
 import io
+import itertools
+import json
 import unittest
+import warnings
 
 import avro.io
 import avro.schema
 import avro.timezones
 
-SCHEMAS_TO_VALIDATE = (
-    ('"null"', None),
-    ('"boolean"', True),
-    ('"string"', 'adsfasdf09809dsf-=adsf'),
-    ('"bytes"', b'12345abcd'),
-    ('"int"', 1234),
-    ('"long"', 1234),
-    ('"float"', 1234.0),
-    ('"double"', 1234.0),
-    ('{"type": "fixed", "name": "Test", "size": 1}', b'B'),
-    ('{"type": "fixed", "logicalType": "decimal", "name": "Test", "size": 8, "precision": 5, "scale": 4}',
-     decimal.Decimal('3.1415')),
-    ('{"type": "fixed", "logicalType": "decimal", "name": "Test", "size": 8, "precision": 5, "scale": 4}',
-     decimal.Decimal('-3.1415')),
-    ('{"type": "bytes", "logicalType": "decimal", "precision": 5, "scale": 4}', decimal.Decimal('3.1415')),
-    ('{"type": "bytes", "logicalType": "decimal", "precision": 5, "scale": 4}', decimal.Decimal('-3.1415')),
-    ('{"type": "enum", "name": "Test", "symbols": ["A", "B"]}', 'B'),
-    ('{"type": "array", "items": "long"}', [1, 3, 2]),
-    ('{"type": "map", "values": "long"}', {'a': 1,
-                                           'b': 3,
-                                           'c': 2}),
-    ('["string", "null", "long"]', None),
-    ('{"type": "int", "logicalType": "date"}', datetime.date(2000, 1, 1)),
-    ('{"type": "int", "logicalType": "time-millis"}', datetime.time(23, 59, 59, 999000)),
-    ('{"type": "int", "logicalType": "time-millis"}', datetime.time(0, 0, 0, 000000)),
-    ('{"type": "long", "logicalType": "time-micros"}', datetime.time(23, 59, 59, 999999)),
-    ('{"type": "long", "logicalType": "time-micros"}', datetime.time(0, 0, 0, 000000)),
-    (
-        '{"type": "long", "logicalType": "timestamp-millis"}',
-        datetime.datetime(1000, 1, 1, 0, 0, 0, 000000, tzinfo=avro.timezones.utc)
-    ),
-    (
-        '{"type": "long", "logicalType": "timestamp-millis"}',
-        datetime.datetime(9999, 12, 31, 23, 59, 59, 999000, tzinfo=avro.timezones.utc)
-    ),
-    (
-        '{"type": "long", "logicalType": "timestamp-millis"}',
-        datetime.datetime(2000, 1, 18, 2, 2, 1, 100000, tzinfo=avro.timezones.tst)
-    ),
-    (
-        '{"type": "long", "logicalType": "timestamp-micros"}',
-        datetime.datetime(1000, 1, 1, 0, 0, 0, 000000, tzinfo=avro.timezones.utc)
-    ),
-    (
-        '{"type": "long", "logicalType": "timestamp-micros"}',
-        datetime.datetime(9999, 12, 31, 23, 59, 59, 999999, tzinfo=avro.timezones.utc)
-    ),
-    (
-        '{"type": "long", "logicalType": "timestamp-micros"}',
-        datetime.datetime(2000, 1, 18, 2, 2, 1, 123499, tzinfo=avro.timezones.tst)
-    ),
-    ('{"type": "string", "logicalType": "uuid"}', u'a4818e1c-8e59-11eb-8dcd-0242ac130003'),  # UUID1
-    ('{"type": "string", "logicalType": "uuid"}', u'570feebe-2bbc-4937-98df-285944e1dbbd'),  # UUID4
-    ('{"type": "string", "logicalType": "unknown-logical-type"}', u'12345abcd'),
-    ('{"type": "string", "logicalType": "timestamp-millis"}', u'12345abcd'),
-    ("""\
-   {"type": "record",
-    "name": "Test",
-    "fields": [{"name": "f", "type": "long"}]}
-   """, {'f': 5}),
-    ("""\
-   {"type": "record",
-    "name": "Lisp",
-    "fields": [{"name": "value",
-                "type": ["null", "string",
-                         {"type": "record",
-                          "name": "Cons",
-                          "fields": [{"name": "car", "type": "Lisp"},
-                                     {"name": "cdr", "type": "Lisp"}]}]}]}
-   """, {'value': {'car': {'value': 'head'}, 'cdr': {'value': None}}}),
+SCHEMAS_TO_VALIDATE = tuple(
+    (json.dumps(schema), datum)
+    for schema, datum in (
+        ("null", None),
+        ("boolean", True),
+        ("string", "adsfasdf09809dsf-=adsf"),
+        ("bytes", b"12345abcd"),
+        ("int", 1234),
+        ("long", 1234),
+        ("float", 1234.0),
+        ("double", 1234.0),
+        ({"type": "fixed", "name": "Test", "size": 1}, b"B"),
+        (
+            {
+                "type": "fixed",
+                "logicalType": "decimal",
+                "name": "Test",
+                "size": 8,
+                "precision": 5,
+                "scale": 4,
+            },
+            decimal.Decimal("3.1415"),
+        ),
+        (
+            {
+                "type": "fixed",
+                "logicalType": "decimal",
+                "name": "Test",
+                "size": 8,
+                "precision": 5,
+                "scale": 4,
+            },
+            decimal.Decimal("-3.1415"),
+        ),
+        (
+            {"type": "bytes", "logicalType": "decimal", "precision": 5, "scale": 4},
+            decimal.Decimal("3.1415"),
+        ),
+        (
+            {"type": "bytes", "logicalType": "decimal", "precision": 5, "scale": 4},
+            decimal.Decimal("-3.1415"),
+        ),
+        ({"type": "enum", "name": "Test", "symbols": ["A", "B"]}, "B"),
+        ({"type": "array", "items": "long"}, [1, 3, 2]),
+        ({"type": "map", "values": "long"}, {"a": 1, "b": 3, "c": 2}),
+        (["string", "null", "long"], None),
+        ({"type": "int", "logicalType": "date"}, datetime.date(2000, 1, 1)),
+        (
+            {"type": "int", "logicalType": "time-millis"},
+            datetime.time(23, 59, 59, 999000),
+        ),
+        ({"type": "int", "logicalType": "time-millis"}, datetime.time(0, 0, 0, 000000)),
+        (
+            {"type": "long", "logicalType": "time-micros"},
+            datetime.time(23, 59, 59, 999999),
+        ),
+        (
+            {"type": "long", "logicalType": "time-micros"},
+            datetime.time(0, 0, 0, 000000),
+        ),
+        (
+            {"type": "long", "logicalType": "timestamp-millis"},
+            datetime.datetime(1000, 1, 1, 0, 0, 0, 000000, tzinfo=avro.timezones.utc),
+        ),
+        (
+            {"type": "long", "logicalType": "timestamp-millis"},
+            datetime.datetime(
+                9999, 12, 31, 23, 59, 59, 999000, tzinfo=avro.timezones.utc
+            ),
+        ),
+        (
+            {"type": "long", "logicalType": "timestamp-millis"},
+            datetime.datetime(2000, 1, 18, 2, 2, 1, 100000, tzinfo=avro.timezones.tst),
+        ),
+        (
+            {"type": "long", "logicalType": "timestamp-micros"},
+            datetime.datetime(1000, 1, 1, 0, 0, 0, 000000, tzinfo=avro.timezones.utc),
+        ),
+        (
+            {"type": "long", "logicalType": "timestamp-micros"},
+            datetime.datetime(
+                9999, 12, 31, 23, 59, 59, 999999, tzinfo=avro.timezones.utc
+            ),
+        ),
+        (
+            {"type": "long", "logicalType": "timestamp-micros"},
+            datetime.datetime(2000, 1, 18, 2, 2, 1, 123499, tzinfo=avro.timezones.tst),
+        ),
+        (
+            {"type": "string", "logicalType": "uuid"},
+            "a4818e1c-8e59-11eb-8dcd-0242ac130003",
+        ),  # UUID1
+        (
+            {"type": "string", "logicalType": "uuid"},
+            "570feebe-2bbc-4937-98df-285944e1dbbd",
+        ),  # UUID4
+        ({"type": "string", "logicalType": "unknown-logical-type"}, "12345abcd"),
+        ({"type": "string", "logicalType": "timestamp-millis"}, "12345abcd"),
+        (
+            {
+                "type": "record",
+                "name": "Test",
+                "fields": [{"name": "f", "type": "long"}],
+            },
+            {"f": 5},
+        ),
+        (
+            {
+                "type": "record",
+                "name": "Lisp",
+                "fields": [
+                    {
+                        "name": "value",
+                        "type": [
+                            "null",
+                            "string",
+                            {
+                                "type": "record",
+                                "name": "Cons",
+                                "fields": [
+                                    {"name": "car", "type": "Lisp"},
+                                    {"name": "cdr", "type": "Lisp"},
+                                ],
+                            },
+                        ],
+                    }
+                ],
+            },
+            {"value": {"car": {"value": "head"}, "cdr": {"value": None}}},
+        ),
+    )
 )
 
 BINARY_ENCODINGS = (
-    (0, b'00'),
-    (-1, b'01'),
-    (1, b'02'),
-    (-2, b'03'),
-    (2, b'04'),
-    (-64, b'7f'),
-    (64, b'80 01'),
-    (8192, b'80 80 01'),
-    (-8193, b'81 80 01'),
+    (0, b"00"),
+    (-1, b"01"),
+    (1, b"02"),
+    (-2, b"03"),
+    (2, b"04"),
+    (-64, b"7f"),
+    (64, b"80 01"),
+    (8192, b"80 80 01"),
+    (-8193, b"81 80 01"),
 )
 
 DEFAULT_VALUE_EXAMPLES = (
-    ('"null"', 'null', None),
-    ('"boolean"', 'true', True),
-    ('"string"', '"foo"', u'foo'),
-    ('"bytes"', '"\u00FF\u00FF"', u'\xff\xff'),
-    ('"int"', '5', 5),
-    ('"long"', '5', 5),
-    ('"float"', '1.1', 1.1),
-    ('"double"', '1.1', 1.1),
-    ('{"type": "fixed", "name": "F", "size": 2}', '"\u00FF\u00FF"', u'\xff\xff'),
-    ('{"type": "enum", "name": "F", "symbols": ["FOO", "BAR"]}', '"FOO"', 'FOO'),
-    ('{"type": "array", "items": "int"}', '[1, 2, 3]', [1, 2, 3]),
-    ('{"type": "map", "values": "int"}', '{"a": 1, "b": 2}', {'a': 1,
-                                                              'b': 2}),
-    ('["int", "null"]', '5', 5),
-    ('{"type": "record", "name": "F", "fields": [{"name": "A", "type": "int"}]}',
-     '{"A": 5}', {'A': 5}),
+    ("null", None),
+    ("boolean", True),
+    ("string", "foo"),
+    ("bytes", "\xff\xff"),
+    ("int", 5),
+    ("long", 5),
+    ("float", 1.1),
+    ("double", 1.1),
+    ({"type": "fixed", "name": "F", "size": 2}, "\xff\xff"),
+    ({"type": "enum", "name": "F", "symbols": ["FOO", "BAR"]}, "FOO"),
+    ({"type": "array", "items": "int"}, [1, 2, 3]),
+    ({"type": "map", "values": "int"}, {"a": 1, "b": 2}),
+    (["int", "null"], 5),
+    (
+        {"type": "record", "name": "F", "fields": [{"name": "A", "type": "int"}]},
+        {"A": 5},
+    ),
 )
 
-LONG_RECORD_SCHEMA = avro.schema.parse("""\
-  {"type": "record",
-   "name": "Test",
-   "fields": [{"name": "A", "type": "int"},
-              {"name": "B", "type": "int"},
-              {"name": "C", "type": "int"},
-              {"name": "D", "type": "int"},
-              {"name": "E", "type": "int"},
-              {"name": "F", "type": "int"},
-              {"name": "G", "type": "int"}]}""")
+LONG_RECORD_SCHEMA = avro.schema.parse(
+    json.dumps(
+        {
+            "type": "record",
+            "name": "Test",
+            "fields": [
+                {"name": "A", "type": "int"},
+                {"name": "B", "type": "int"},
+                {"name": "C", "type": "int"},
+                {"name": "D", "type": "int"},
+                {"name": "E", "type": "int"},
+                {"name": "F", "type": "int"},
+                {"name": "G", "type": "int"},
+            ],
+        }
+    )
+)
 
-LONG_RECORD_DATUM = {'A': 1, 'B': 2, 'C': 3, 'D': 4, 'E': 5, 'F': 6, 'G': 7}
+LONG_RECORD_DATUM = {"A": 1, "B": 2, "C": 3, "D": 4, "E": 5, "F": 6, "G": 7}
 
 
 def avro_hexlify(reader):
@@ -154,14 +225,7 @@ def avro_hexlify(reader):
     while (ord(current_byte) & 0x80) != 0:
         current_byte = reader.read(1)
         b.append(binascii.hexlify(current_byte))
-    return b' '.join(b)
-
-
-def print_test_name(test_name):
-    print('')
-    print(test_name)
-    print('=' * len(test_name))
-    print('')
+    return b" ".join(b)
 
 
 def write_datum(datum, writers_schema):
@@ -179,227 +243,363 @@ def read_datum(buffer, writers_schema, readers_schema=None):
     return datum_reader.read(decoder)
 
 
-def check_binary_encoding(number_type):
-    print_test_name('TEST BINARY %s ENCODING' % number_type.upper())
-    correct = 0
-    for datum, hex_encoding in BINARY_ENCODINGS:
-        print('Datum: %d' % datum)
-        print('Correct Encoding: %s' % hex_encoding)
-
-        writers_schema = avro.schema.parse('"%s"' % number_type.lower())
-        writer, encoder, datum_writer = write_datum(datum, writers_schema)
-        writer.seek(0)
-        hex_val = avro_hexlify(writer)
-
-        print('Read Encoding: %s' % hex_val)
-        if hex_encoding == hex_val:
-            correct += 1
-        print('')
-    return correct
-
-
-def check_skip_number(number_type):
-    print_test_name('TEST SKIP %s' % number_type.upper())
-    correct = 0
-    for value_to_skip, hex_encoding in BINARY_ENCODINGS:
+class IoValidateTestCase(unittest.TestCase):
+    def __init__(self, test_schema, test_datum):
+        """Ignore the normal signature for unittest.TestCase because we are generating
+        many test cases from this one class. This is safe as long as the autoloader
+        ignores this class. The autoloader will ignore this class as long as it has
+        no methods starting with `test_`.
+        """
+        super().__init__("io_valid")
+        self.test_schema = avro.schema.parse(test_schema)
+        self.test_datum = test_datum
+        # Never hide repeated warnings when running this test case.
+        warnings.simplefilter("always")
+
+    def io_valid(self):
+        """
+        In these cases, the provided data should be valid with the given schema.
+        """
+        with warnings.catch_warnings(record=True) as actual_warnings:
+            self.assertTrue(
+                avro.io.validate(self.test_schema, self.test_datum),
+                f"{self.test_datum} did not validate in the schema {self.test_schema}",
+            )
+
+
+class RoundTripTestCase(unittest.TestCase):
+    def __init__(self, test_schema, test_datum):
+        """Ignore the normal signature for unittest.TestCase because we are generating
+        many test cases from this one class. This is safe as long as the autoloader
+        ignores this class. The autoloader will ignore this class as long as it has
+        no methods starting with `test_`.
+        """
+        super().__init__("io_round_trip")
+        self.test_schema = avro.schema.parse(test_schema)
+        self.test_datum = test_datum
+        # Never hide repeated warnings when running this test case.
+        warnings.simplefilter("always")
+
+    def io_round_trip(self):
+        """
+        A datum should be the same after being encoded and then decoded.
+        """
+        with warnings.catch_warnings(record=True) as actual_warnings:
+            writer, encoder, datum_writer = write_datum(
+                self.test_datum, self.test_schema
+            )
+            round_trip_datum = read_datum(writer, self.test_schema)
+            expected, round_trip, message = (
+                (
+                    str(self.test_datum),
+                    round_trip_datum.to_eng_string(),
+                    "Decimal datum changed value after encode and decode",
+                )
+                if isinstance(round_trip_datum, decimal.Decimal)
+                else (
+                    self.test_datum.astimezone(tz=avro.timezones.utc),
+                    round_trip_datum,
+                    "DateTime datum changed value after encode and decode",
+                )
+                if isinstance(round_trip_datum, datetime.datetime)
+                else (
+                    self.test_datum,
+                    round_trip_datum,
+                    "Datum changed value after encode and decode",
+                )
+            )
+            self.assertEqual(expected, round_trip, message)
+
+
+class BinaryEncodingTestCase(unittest.TestCase):
+    def __init__(self, skip, test_type, test_datum, test_hex):
+        """Ignore the normal signature for unittest.TestCase because we are generating
+        many test cases from this one class. This is safe as long as the autoloader
+        ignores this class. The autoloader will ignore this class as long as it has
+        no methods starting with `test_`.
+        """
+        super().__init__(f"check_{'skip' if skip else 'binary'}_encoding")
+        self.writers_schema = avro.schema.parse(f'"{test_type}"')
+        self.test_datum = test_datum
+        self.test_hex = test_hex
+        # Never hide repeated warnings when running this test case.
+        warnings.simplefilter("always")
+
+    def check_binary_encoding(self):
+        with warnings.catch_warnings(record=True) as actual_warnings:
+            writer, encoder, datum_writer = write_datum(
+                self.test_datum, self.writers_schema
+            )
+            writer.seek(0)
+            hex_val = avro_hexlify(writer)
+            self.assertEqual(
+                self.test_hex,
+                hex_val,
+                "Binary encoding did not match expected hex representation.",
+            )
+
+    def check_skip_encoding(self):
         VALUE_TO_READ = 6253
-        print('Value to Skip: %d' % value_to_skip)
-
-        # write the value to skip and a known value
-        writers_schema = avro.schema.parse('"%s"' % number_type.lower())
-        writer, encoder, datum_writer = write_datum(value_to_skip, writers_schema)
-        datum_writer.write(VALUE_TO_READ, encoder)
-
-        # skip the value
-        reader = io.BytesIO(writer.getvalue())
-        decoder = avro.io.BinaryDecoder(reader)
-        decoder.skip_long()
-
-        # read data from string buffer
-        datum_reader = avro.io.DatumReader(writers_schema)
-        read_value = datum_reader.read(decoder)
-
-        print('Read Value: %d' % read_value)
-        if read_value == VALUE_TO_READ:
-            correct += 1
-        print('')
-    return correct
-
-
-class TestIO(unittest.TestCase):
-    #
-    # BASIC FUNCTIONALITY
-    #
-
-    def test_validate(self):
-        print_test_name('TEST VALIDATE')
-        passed = 0
-        for example_schema, datum in SCHEMAS_TO_VALIDATE:
-            print('Schema: %s' % example_schema)
-            print('Datum: %s' % datum)
-            validated = avro.io.validate(avro.schema.parse(example_schema), datum)
-            print('Valid: %s' % validated)
-            if validated:
-                passed += 1
-        self.assertEqual(passed, len(SCHEMAS_TO_VALIDATE))
-
-    def test_round_trip(self):
-        print_test_name('TEST ROUND TRIP')
-        correct = 0
-        for example_schema, datum in SCHEMAS_TO_VALIDATE:
-            print('Schema: %s' % example_schema)
-            print('Datum: %s' % datum)
-
-            writers_schema = avro.schema.parse(example_schema)
-            writer, encoder, datum_writer = write_datum(datum, writers_schema)
-            round_trip_datum = read_datum(writer, writers_schema)
-
-            print('Round Trip Datum: %s' % round_trip_datum)
-            if isinstance(round_trip_datum, decimal.Decimal):
-                round_trip_datum = round_trip_datum.to_eng_string()
-                datum = str(datum)
-            elif isinstance(round_trip_datum, datetime.datetime):
-                datum = datum.astimezone(tz=avro.timezones.utc)
-            if datum == round_trip_datum:
-                correct += 1
-        self.assertEqual(correct, len(SCHEMAS_TO_VALIDATE))
-
-    #
-    # BINARY ENCODING OF INT AND LONG
-    #
-
-    def test_binary_int_encoding(self):
-        correct = check_binary_encoding('int')
-        self.assertEqual(correct, len(BINARY_ENCODINGS))
-
-    def test_binary_long_encoding(self):
-        correct = check_binary_encoding('long')
-        self.assertEqual(correct, len(BINARY_ENCODINGS))
-
-    def test_skip_int(self):
-        correct = check_skip_number('int')
-        self.assertEqual(correct, len(BINARY_ENCODINGS))
-
-    def test_skip_long(self):
-        correct = check_skip_number('long')
-        self.assertEqual(correct, len(BINARY_ENCODINGS))
-
-    #
-    # SCHEMA RESOLUTION
-    #
-
-    def test_schema_promotion(self):
-        print_test_name('TEST SCHEMA PROMOTION')
+        with warnings.catch_warnings(record=True) as actual_warnings:
+            # write the value to skip and a known value
+            writer, encoder, datum_writer = write_datum(
+                self.test_datum, self.writers_schema
+            )
+            datum_writer.write(VALUE_TO_READ, encoder)
+
+            # skip the value
+            reader = io.BytesIO(writer.getvalue())
+            decoder = avro.io.BinaryDecoder(reader)
+            decoder.skip_long()
+
+            # read data from string buffer
+            datum_reader = avro.io.DatumReader(self.writers_schema)
+            read_value = datum_reader.read(decoder)
+
+            self.assertEqual(
+                read_value,
+                VALUE_TO_READ,
+                "Unexpected value after skipping a binary encoded value.",
+            )
+
+
+class SchemaPromotionTestCase(unittest.TestCase):
+    def __init__(self, write_type, read_type):
+        """Ignore the normal signature for unittest.TestCase because we are generating
+        many test cases from this one class. This is safe as long as the autoloader
+        ignores this class. The autoloader will ignore this class as long as it has
+        no methods starting with `test_`.
+        """
+        super().__init__("check_schema_promotion")
+        self.writers_schema = avro.schema.parse(f'"{write_type}"')
+        self.readers_schema = avro.schema.parse(f'"{read_type}"')
+        # Never hide repeated warnings when running this test case.
+        warnings.simplefilter("always")
+
+    def check_schema_promotion(self):
+        """Test schema promotion"""
         # note that checking writers_schema.type in read_data
         # allows us to handle promotion correctly
-        promotable_schemas = ['"int"', '"long"', '"float"', '"double"']
-        incorrect = 0
-        for i, ws in enumerate(promotable_schemas):
-            writers_schema = avro.schema.parse(ws)
-            datum_to_write = 219
-            for rs in promotable_schemas[i + 1:]:
-                readers_schema = avro.schema.parse(rs)
-                writer, enc, dw = write_datum(datum_to_write, writers_schema)
-                datum_read = read_datum(writer, writers_schema, readers_schema)
-                print('Writer: %s Reader: %s' % (writers_schema, readers_schema))
-                print('Datum Read: %s' % datum_read)
-                if datum_read != datum_to_write:
-                    incorrect += 1
-        self.assertEqual(incorrect, 0)
+        DATUM_TO_WRITE = 219
+        with warnings.catch_warnings(record=True) as actual_warnings:
+            writer, enc, dw = write_datum(DATUM_TO_WRITE, self.writers_schema)
+            datum_read = read_datum(writer, self.writers_schema, self.readers_schema)
+            self.assertEqual(
+                datum_read,
+                DATUM_TO_WRITE,
+                f"Datum changed between schema that were supposed to promote: writer: {self.writers_schema} reader: {self.readers_schema}.",
+            )
+
+
+class DefaultValueTestCase(unittest.TestCase):
+    def __init__(self, field_type, default):
+        """Ignore the normal signature for unittest.TestCase because we are generating
+        many test cases from this one class. This is safe as long as the autoloader
+        ignores this class. The autoloader will ignore this class as long as it has
+        no methods starting with `test_`.
+        """
+        super().__init__("check_default_value")
+        self.field_type = field_type
+        self.default = default
+        # Never hide repeated warnings when running this test case.
+        warnings.simplefilter("always")
+
+    def check_default_value(self):
+        with warnings.catch_warnings(record=True) as actual_warnings:
+            datum_to_read = {"H": self.default}
+            readers_schema = avro.schema.parse(
+                json.dumps(
+                    {
+                        "type": "record",
+                        "name": "Test",
+                        "fields": [
+                            {
+                                "name": "H",
+                                "type": self.field_type,
+                                "default": self.default,
+                            }
+                        ],
+                    }
+                )
+            )
+            writer, _, _ = write_datum(LONG_RECORD_DATUM, LONG_RECORD_SCHEMA)
+            datum_read = read_datum(writer, LONG_RECORD_SCHEMA, readers_schema)
+            self.assertEqual(datum_to_read, datum_read)
+
+
+class TestMisc(unittest.TestCase):
+    def test_decimal_bytes_small_scale(self):
+        """Avro should raise an AvroTypeException when attempting to write a decimal with a larger exponent than the schema's scale."""
+        datum = decimal.Decimal("3.1415")
+        _, _, exp = datum.as_tuple()
+        scale = -1 * exp - 1
+        schema = avro.schema.parse(
+            json.dumps(
+                {
+                    "type": "bytes",
+                    "logicalType": "decimal",
+                    "precision": 5,
+                    "scale": scale,
+                }
+            )
+        )
+        self.assertRaises(
+            avro.errors.AvroOutOfScaleException, write_datum, datum, schema
+        )
+
+    def test_decimal_fixed_small_scale(self):
+        """Avro should raise an AvroTypeException when attempting to write a decimal with a larger exponent than the schema's scale."""
+        datum = decimal.Decimal("3.1415")
+        _, _, exp = datum.as_tuple()
+        scale = -1 * exp - 1
+        schema = avro.schema.parse(
+            json.dumps(
+                {
+                    "type": "fixed",
+                    "logicalType": "decimal",
+                    "name": "Test",
+                    "size": 8,
+                    "precision": 5,
+                    "scale": scale,
+                }
+            )
+        )
+        self.assertRaises(
+            avro.errors.AvroOutOfScaleException, write_datum, datum, schema
+        )
 
     def test_unknown_symbol(self):
-        print_test_name('TEST UNKNOWN SYMBOL')
-        writers_schema = avro.schema.parse("""\
-      {"type": "enum", "name": "Test",
-       "symbols": ["FOO", "BAR"]}""")
-        datum_to_write = 'FOO'
-
-        readers_schema = avro.schema.parse("""\
-      {"type": "enum", "name": "Test",
-       "symbols": ["BAR", "BAZ"]}""")
+        datum_to_write = "FOO"
+        writers_schema = avro.schema.parse(
+            json.dumps({"type": "enum", "name": "Test", "symbols": ["FOO", "BAR"]})
+        )
+        readers_schema = avro.schema.parse(
+            json.dumps({"type": "enum", "name": "Test", "symbols": ["BAR", "BAZ"]})
+        )
 
         writer, encoder, datum_writer = write_datum(datum_to_write, writers_schema)
         reader = io.BytesIO(writer.getvalue())
         decoder = avro.io.BinaryDecoder(reader)
         datum_reader = avro.io.DatumReader(writers_schema, readers_schema)
-        self.assertRaises(avro.errors.SchemaResolutionException, datum_reader.read, decoder)
-
-    def test_default_value(self):
-        print_test_name('TEST DEFAULT VALUE')
-        writers_schema = LONG_RECORD_SCHEMA
-        datum_to_write = LONG_RECORD_DATUM
-
-        correct = 0
-        for field_type, default_json, default_datum in DEFAULT_VALUE_EXAMPLES:
-            readers_schema = avro.schema.parse("""\
-        {"type": "record", "name": "Test",
-         "fields": [{"name": "H", "type": %s, "default": %s}]}
-        """ % (field_type, default_json))
-            datum_to_read = {'H': default_datum}
-
-            writer, encoder, datum_writer = write_datum(datum_to_write, writers_schema)
-            datum_read = read_datum(writer, writers_schema, readers_schema)
-            print('Datum Read: %s' % datum_read)
-            if datum_to_read == datum_read:
-                correct += 1
-        self.assertEqual(correct, len(DEFAULT_VALUE_EXAMPLES))
+        self.assertRaises(
+            avro.errors.SchemaResolutionException, datum_reader.read, decoder
+        )
 
     def test_no_default_value(self):
-        print_test_name('TEST NO DEFAULT VALUE')
         writers_schema = LONG_RECORD_SCHEMA
         datum_to_write = LONG_RECORD_DATUM
 
-        readers_schema = avro.schema.parse("""\
-      {"type": "record", "name": "Test",
-       "fields": [{"name": "H", "type": "int"}]}""")
+        readers_schema = avro.schema.parse(
+            json.dumps(
+                {
+                    "type": "record",
+                    "name": "Test",
+                    "fields": [{"name": "H", "type": "int"}],
+                }
+            )
+        )
 
         writer, encoder, datum_writer = write_datum(datum_to_write, writers_schema)
         reader = io.BytesIO(writer.getvalue())
         decoder = avro.io.BinaryDecoder(reader)
         datum_reader = avro.io.DatumReader(writers_schema, readers_schema)
-        self.assertRaises(avro.errors.SchemaResolutionException, datum_reader.read, decoder)
+        self.assertRaises(
+            avro.errors.SchemaResolutionException, datum_reader.read, decoder
+        )
 
     def test_projection(self):
-        print_test_name('TEST PROJECTION')
         writers_schema = LONG_RECORD_SCHEMA
         datum_to_write = LONG_RECORD_DATUM
 
-        readers_schema = avro.schema.parse("""\
-      {"type": "record", "name": "Test",
-       "fields": [{"name": "E", "type": "int"},
-                  {"name": "F", "type": "int"}]}""")
-        datum_to_read = {'E': 5, 'F': 6}
+        readers_schema = avro.schema.parse(
+            json.dumps(
+                {
+                    "type": "record",
+                    "name": "Test",
+                    "fields": [
+                        {"name": "E", "type": "int"},
+                        {"name": "F", "type": "int"},
+                    ],
+                }
+            )
+        )
+        datum_to_read = {"E": 5, "F": 6}
 
         writer, encoder, datum_writer = write_datum(datum_to_write, writers_schema)
         datum_read = read_datum(writer, writers_schema, readers_schema)
-        print('Datum Read: %s' % datum_read)
         self.assertEqual(datum_to_read, datum_read)
 
     def test_field_order(self):
-        print_test_name('TEST FIELD ORDER')
         writers_schema = LONG_RECORD_SCHEMA
         datum_to_write = LONG_RECORD_DATUM
 
-        readers_schema = avro.schema.parse("""\
-      {"type": "record", "name": "Test",
-       "fields": [{"name": "F", "type": "int"},
-                  {"name": "E", "type": "int"}]}""")
-        datum_to_read = {'E': 5, 'F': 6}
+        readers_schema = avro.schema.parse(
+            json.dumps(
+                {
+                    "type": "record",
+                    "name": "Test",
+                    "fields": [
+                        {"name": "F", "type": "int"},
+                        {"name": "E", "type": "int"},
+                    ],
+                }
+            )
+        )
+        datum_to_read = {"E": 5, "F": 6}
 
         writer, encoder, datum_writer = write_datum(datum_to_write, writers_schema)
         datum_read = read_datum(writer, writers_schema, readers_schema)
-        print('Datum Read: %s' % datum_read)
         self.assertEqual(datum_to_read, datum_read)
 
     def test_type_exception(self):
-        print_test_name('TEST TYPE EXCEPTION')
-        writers_schema = avro.schema.parse("""\
-      {"type": "record", "name": "Test",
-       "fields": [{"name": "F", "type": "int"},
-                  {"name": "E", "type": "int"}]}""")
-        datum_to_write = {'E': 5, 'F': 'Bad'}
-        self.assertRaises(avro.errors.AvroTypeException, write_datum, datum_to_write, writers_schema)
-
-
-if __name__ == '__main__':
+        writers_schema = avro.schema.parse(
+            json.dumps(
+                {
+                    "type": "record",
+                    "name": "Test",
+                    "fields": [
+                        {"name": "F", "type": "int"},
+                        {"name": "E", "type": "int"},
+                    ],
+                }
+            )
+        )
+        datum_to_write = {"E": 5, "F": "Bad"}
+        self.assertRaises(
+            avro.errors.AvroTypeException, write_datum, datum_to_write, writers_schema
+        )
+
+
+def load_tests(loader, default_tests, pattern):
+    """Generate test cases across many test schema."""
+    suite = unittest.TestSuite()
+    suite.addTests(loader.loadTestsFromTestCase(TestMisc))
+    suite.addTests(
+        IoValidateTestCase(schema_str, datum)
+        for schema_str, datum in SCHEMAS_TO_VALIDATE
+    )
+    suite.addTests(
+        RoundTripTestCase(schema_str, datum)
+        for schema_str, datum in SCHEMAS_TO_VALIDATE
+    )
+    for skip in False, True:
+        for type_ in "int", "long":
+            suite.addTests(
+                BinaryEncodingTestCase(skip, type_, datum, hex_)
+                for datum, hex_ in BINARY_ENCODINGS
+            )
+    suite.addTests(
+        SchemaPromotionTestCase(write_type, read_type)
+        for write_type, read_type in itertools.combinations(
+            ("int", "long", "float", "double"), 2
+        )
+    )
+    suite.addTests(
+        DefaultValueTestCase(field_type, default)
+        for field_type, default in DEFAULT_VALUE_EXAMPLES
+    )
+    return suite
+
+
+if __name__ == "__main__":
     unittest.main()
diff --git a/lang/py/avro/test/test_schema.py b/lang/py/avro/test/test_schema.py
index b9f2887..6fcfb0a 100644
--- a/lang/py/avro/test/test_schema.py
+++ b/lang/py/avro/test/test_schema.py
@@ -499,7 +499,7 @@ class SchemaParseTestCase(unittest.TestCase):
         ignores this class. The autoloader will ignore this class as long as it has
         no methods starting with `test_`.
         """
-        super(SchemaParseTestCase, self).__init__(
+        super().__init__(
             'parse_valid' if test_schema.valid else 'parse_invalid')
         self.test_schema = test_schema
         # Never hide repeated warnings when running this test case.
@@ -538,7 +538,7 @@ class RoundTripParseTestCase(unittest.TestCase):
         ignores this class. The autoloader will ignore this class as long as it has
         no methods starting with `test_`.
         """
-        super(RoundTripParseTestCase, self).__init__('parse_round_trip')
+        super().__init__('parse_round_trip')
         self.test_schema = test_schema
 
     def parse_round_trip(self):