You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by GitBox <gi...@apache.org> on 2020/11/09 19:28:55 UTC

[GitHub] [avro] gabriel-tincu opened a new pull request #979: AVRO-1751: Add python3 compatibility

gabriel-tincu opened a new pull request #979:
URL: https://github.com/apache/avro/pull/979


   Keeping in line with the same logic present in the avro java library
   that handles type promotions, this aims to be a faithful adaptation of
   that code in python.
   
   Most tests were also ported, with the exception of the ones dealing with
   enum default, since the python api does not support that yet
   
   See :
   - https://github.com/apache/avro/blob/master/lang/java/avro/src/main/java/org/apache/avro/SchemaCompatibility.java
   - https://issues.apache.org/jira/browse/AVRO-1751
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-724290720


   > Please be aware the py3 library is deprecated. The py lib has full support for modern python 3, and tests in multiple versions of python. Would it be possible to do this change in that library instead?
   
   done


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r526336487



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        that = cast(SchemaCompatibilityResult, that)
+        merged = copy(self.incompatibilities)
+        merged.extend(copy(that.incompatibilities))
+        if self.compatibility is SchemaCompatibilityType.compatible:
+            compat = that.compatibility
+            messages = that.messages
+            locations = that.locations
+        else:
+            compat = self.compatibility
+            messages = self.messages.union(that.messages)
+            locations = self.locations.union(that.locations)
+        return SchemaCompatibilityResult(
+            compatibility=compat, incompatibilities=merged, messages=messages, locations=locations
+        )
+
+    @staticmethod
+    def compatible():
+        return SchemaCompatibilityResult(SchemaCompatibilityType.compatible)
+
+    @staticmethod
+    def incompatible(incompat_type: SchemaIncompatibilityType, message: str, location: List[str]):
+        locations = "/".join(location)
+        if len(location) > 1:
+            locations = locations[1:]
+        ret = SchemaCompatibilityResult(
+            compatibility=SchemaCompatibilityType.incompatible,
+            incompatibilities=[incompat_type],
+            locations={locations},
+            messages={message},
+        )
+        return ret
+
+    def __str__(self):
+        return f"{self.compatibility}: {self.messages}"
+
+
+class ReaderWriter:
+    def __init__(self, reader: Schema, writer: Schema) -> None:
+        self.reader, self.writer = reader, writer
+
+    def __hash__(self) -> SchemaType.INT:
+        return id(self.reader) ^ id(self.writer)
+
+    def __eq__(self, other) -> bool:
+        if not isinstance(other, ReaderWriter):
+            return False
+        return self.reader is other.reader and self.writer is other.writer
+
+
+class ReaderWriterCompatibilityChecker:
+    ROOT_REFERENCE_TOKEN = "/"
+
+    def __init__(self):
+        self.memoize_map: Dict[ReaderWriter, SchemaCompatibilityResult] = {}
+
+    def get_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        reference_token: str = ROOT_REFERENCE_TOKEN,
+        location: Optional[List[str]] = None
+    ) -> SchemaCompatibilityResult:
+        if location is None:
+            location = []
+        location.append(reference_token)
+        pair = ReaderWriter(reader, writer)
+        if pair in self.memoize_map:
+            result = self.memoize_map[pair]
+            if result.compatibility is SchemaCompatibilityType.recursion_in_progress:
+                result = SchemaCompatibilityResult.compatible()
+        else:
+            self.memoize_map[pair] = SchemaCompatibilityResult()
+            result = self.calculate_compatibility(reader, writer, location)
+            self.memoize_map[pair] = result
+        location.pop()

Review comment:
       Other than using a context manager (which seems to be overkill) i really can't think of any way other than the present (using a collection)




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r526336487



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        that = cast(SchemaCompatibilityResult, that)
+        merged = copy(self.incompatibilities)
+        merged.extend(copy(that.incompatibilities))
+        if self.compatibility is SchemaCompatibilityType.compatible:
+            compat = that.compatibility
+            messages = that.messages
+            locations = that.locations
+        else:
+            compat = self.compatibility
+            messages = self.messages.union(that.messages)
+            locations = self.locations.union(that.locations)
+        return SchemaCompatibilityResult(
+            compatibility=compat, incompatibilities=merged, messages=messages, locations=locations
+        )
+
+    @staticmethod
+    def compatible():
+        return SchemaCompatibilityResult(SchemaCompatibilityType.compatible)
+
+    @staticmethod
+    def incompatible(incompat_type: SchemaIncompatibilityType, message: str, location: List[str]):
+        locations = "/".join(location)
+        if len(location) > 1:
+            locations = locations[1:]
+        ret = SchemaCompatibilityResult(
+            compatibility=SchemaCompatibilityType.incompatible,
+            incompatibilities=[incompat_type],
+            locations={locations},
+            messages={message},
+        )
+        return ret
+
+    def __str__(self):
+        return f"{self.compatibility}: {self.messages}"
+
+
+class ReaderWriter:
+    def __init__(self, reader: Schema, writer: Schema) -> None:
+        self.reader, self.writer = reader, writer
+
+    def __hash__(self) -> SchemaType.INT:
+        return id(self.reader) ^ id(self.writer)
+
+    def __eq__(self, other) -> bool:
+        if not isinstance(other, ReaderWriter):
+            return False
+        return self.reader is other.reader and self.writer is other.writer
+
+
+class ReaderWriterCompatibilityChecker:
+    ROOT_REFERENCE_TOKEN = "/"
+
+    def __init__(self):
+        self.memoize_map: Dict[ReaderWriter, SchemaCompatibilityResult] = {}
+
+    def get_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        reference_token: str = ROOT_REFERENCE_TOKEN,
+        location: Optional[List[str]] = None
+    ) -> SchemaCompatibilityResult:
+        if location is None:
+            location = []
+        location.append(reference_token)
+        pair = ReaderWriter(reader, writer)
+        if pair in self.memoize_map:
+            result = self.memoize_map[pair]
+            if result.compatibility is SchemaCompatibilityType.recursion_in_progress:
+                result = SchemaCompatibilityResult.compatible()
+        else:
+            self.memoize_map[pair] = SchemaCompatibilityResult()
+            result = self.calculate_compatibility(reader, writer, location)
+            self.memoize_map[pair] = result
+        location.pop()

Review comment:
       @kojiromike Other than using a context manager (which seems to be overkill), the easiest solution would be to just pass in to the other parameters `location + ["foo"]`. This would drop the mutability issue but would add a bit to the codebase. Also, why do you believe this is hidden?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r522226083



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        that = cast(SchemaCompatibilityResult, that)
+        merged = copy(self.incompatibilities)
+        merged.extend(copy(that.incompatibilities))
+        if self.compatibility is SchemaCompatibilityType.compatible:
+            compat = that.compatibility
+            messages = that.messages
+            locations = that.locations
+        else:
+            compat = self.compatibility
+            messages = self.messages.union(that.messages)
+            locations = self.locations.union(that.locations)
+        return SchemaCompatibilityResult(
+            compatibility=compat, incompatibilities=merged, messages=messages, locations=locations
+        )
+
+    @staticmethod
+    def compatible():

Review comment:
       I think we should avoid static methods. Instead, consider defining a module constant for COMPATIBLE that any other module can import.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r526946672



##########
File path: lang/py/avro/test/test_compatibility.py
##########
@@ -0,0 +1,665 @@
+import json
+
+from avro.compatibility import ReaderWriterCompatibilityChecker, SchemaCompatibilityType, SchemaType
+from avro.schema import ArraySchema, MapSchema, Names, PrimitiveSchema, Schema, UnionSchema, parse
+
+#  ================================================================================================
+#            These tests are more or less directly lifted from the java avro codebase
+#            There's one test per Java file, so expect the first one to be a mammoth
+#  ================================================================================================
+BOOLEAN_SCHEMA = PrimitiveSchema(SchemaType.BOOLEAN)
+NULL_SCHEMA = PrimitiveSchema(SchemaType.NULL)
+INT_SCHEMA = PrimitiveSchema(SchemaType.INT)
+LONG_SCHEMA = PrimitiveSchema(SchemaType.LONG)
+STRING_SCHEMA = PrimitiveSchema(SchemaType.STRING)
+BYTES_SCHEMA = PrimitiveSchema(SchemaType.BYTES)
+FLOAT_SCHEMA = PrimitiveSchema(SchemaType.FLOAT)
+DOUBLE_SCHEMA = PrimitiveSchema(SchemaType.DOUBLE)
+INT_ARRAY_SCHEMA = ArraySchema(SchemaType.INT, names=Names())
+LONG_ARRAY_SCHEMA = ArraySchema(SchemaType.LONG, names=Names())
+STRING_ARRAY_SCHEMA = ArraySchema(SchemaType.STRING, names=Names())
+INT_MAP_SCHEMA = MapSchema(SchemaType.INT, names=Names())
+LONG_MAP_SCHEMA = MapSchema(SchemaType.LONG, names=Names())
+STRING_MAP_SCHEMA = MapSchema(SchemaType.STRING, names=Names())
+ENUM1_AB_SCHEMA = parse(json.dumps({"type": SchemaType.ENUM, "name": "Enum1", "symbols": ["A", "B"]}))
+ENUM1_ABC_SCHEMA = parse(json.dumps({"type": SchemaType.ENUM, "name": "Enum1", "symbols": ["A", "B", "C"]}))
+ENUM1_BC_SCHEMA = parse(json.dumps({"type": SchemaType.ENUM, "name": "Enum1", "symbols": ["B", "C"]}))
+ENUM2_AB_SCHEMA = parse(json.dumps({"type": SchemaType.ENUM, "name": "Enum2", "symbols": ["A", "B"]}))
+ENUM_ABC_ENUM_DEFAULT_A_SCHEMA = parse(
+    json.dumps({
+        "type": "enum",
+        "name": "Enum",
+        "symbols": ["A", "B", "C"],
+        "default": "A"
+    })
+)
+ENUM_AB_ENUM_DEFAULT_A_SCHEMA = parse(json.dumps({"type": SchemaType.ENUM, "name": "Enum", "symbols": ["A", "B"], "default": "A"}))
+ENUM_ABC_ENUM_DEFAULT_A_RECORD = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record",
+        "fields": [{
+            "name": "Field",
+            "type": {
+                "type": SchemaType.ENUM,
+                "name": "Enum",
+                "symbols": ["A", "B", "C"],
+                "default": "A"
+            }
+        }]
+    })
+)
+ENUM_AB_ENUM_DEFAULT_A_RECORD = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record",
+        "fields": [{
+            "name": "Field",
+            "type": {
+                "type": SchemaType.ENUM,
+                "name": "Enum",
+                "symbols": ["A", "B"],
+                "default": "A"
+            }
+        }]
+    })
+)
+ENUM_ABC_FIELD_DEFAULT_B_ENUM_DEFAULT_A_RECORD = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record",
+        "fields": [{
+            "name": "Field",
+            "type": {
+                "type": SchemaType.ENUM,
+                "name": "Enum",
+                "symbols": ["A", "B", "C"],
+                "default": "A"
+            },
+            "default": "B"
+        }]
+    })
+)
+ENUM_AB_FIELD_DEFAULT_A_ENUM_DEFAULT_B_RECORD = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record",
+        "fields": [{
+            "name": "Field",
+            "type": {
+                "type": SchemaType.ENUM,
+                "name": "Enum",
+                "symbols": ["A", "B"],
+                "default": "B"
+            },
+            "default": "A"
+        }]
+    })
+)
+EMPTY_UNION_SCHEMA = UnionSchema([], names=Names())
+NULL_UNION_SCHEMA = UnionSchema([SchemaType.NULL], names=Names())
+INT_UNION_SCHEMA = UnionSchema([SchemaType.INT], names=Names())
+LONG_UNION_SCHEMA = UnionSchema([SchemaType.LONG], names=Names())
+FLOAT_UNION_SCHEMA = UnionSchema([SchemaType.FLOAT], names=Names())
+DOUBLE_UNION_SCHEMA = UnionSchema([SchemaType.DOUBLE], names=Names())
+STRING_UNION_SCHEMA = UnionSchema([SchemaType.STRING], names=Names())
+BYTES_UNION_SCHEMA = UnionSchema([SchemaType.BYTES], names=Names())
+INT_STRING_UNION_SCHEMA = UnionSchema([SchemaType.INT, SchemaType.STRING], names=Names())
+STRING_INT_UNION_SCHEMA = UnionSchema([SchemaType.STRING, SchemaType.INT], names=Names())
+INT_FLOAT_UNION_SCHEMA = UnionSchema([SchemaType.INT, SchemaType.FLOAT], names=Names())
+INT_LONG_UNION_SCHEMA = UnionSchema([SchemaType.INT, SchemaType.LONG], names=Names())
+INT_LONG_FLOAT_DOUBLE_UNION_SCHEMA = UnionSchema([SchemaType.INT, SchemaType.LONG, SchemaType.FLOAT, SchemaType.DOUBLE], names=Names())
+NULL_INT_ARRAY_UNION_SCHEMA = UnionSchema([{"type": SchemaType.NULL}, {"type": SchemaType.ARRAY, "items": SchemaType.INT}], names=Names())
+NULL_INT_MAP_UNION_SCHEMA = UnionSchema([{"type": SchemaType.NULL}, {"type": SchemaType.MAP, "values": SchemaType.INT}], names=Names())
+EMPTY_RECORD1 = parse(json.dumps({"type": SchemaType.RECORD, "name": "Record1", "fields": []}))
+EMPTY_RECORD2 = parse(json.dumps({"type": SchemaType.RECORD, "name": "Record2", "fields": []}))
+A_INT_RECORD1 = parse(json.dumps({"type": SchemaType.RECORD, "name": "Record1", "fields": [{"name": "a", "type": SchemaType.INT}]}))
+A_LONG_RECORD1 = parse(json.dumps({"type": SchemaType.RECORD, "name": "Record1", "fields": [{"name": "a", "type": SchemaType.LONG}]}))
+A_INT_B_INT_RECORD1 = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record1",
+        "fields": [{
+            "name": "a",
+            "type": SchemaType.INT
+        }, {
+            "name": "b",
+            "type": SchemaType.INT
+        }]
+    })
+)
+A_DINT_RECORD1 = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record1",
+        "fields": [{
+            "name": "a",
+            "type": SchemaType.INT,
+            "default": 0
+        }]
+    })
+)
+A_INT_B_DINT_RECORD1 = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record1",
+        "fields": [{
+            "name": "a",
+            "type": SchemaType.INT
+        }, {
+            "name": "b",
+            "type": SchemaType.INT,
+            "default": 0
+        }]
+    })
+)
+A_DINT_B_DINT_RECORD1 = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record1",
+        "fields": [{
+            "name": "a",
+            "type": SchemaType.INT,
+            "default": 0
+        }, {
+            "name": "b",
+            "type": SchemaType.INT,
+            "default": 0
+        }]
+    })
+)
+A_DINT_B_DFIXED_4_BYTES_RECORD1 = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record1",
+        "fields": [{
+            "name": "a",
+            "type": SchemaType.INT,
+            "default": 0
+        }, {
+            "name": "b",
+            "type": {
+                "type": SchemaType.FIXED,
+                "name": "Fixed",
+                "size": 4
+            }
+        }]
+    })
+)
+A_DINT_B_DFIXED_8_BYTES_RECORD1 = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record1",
+        "fields": [{
+            "name": "a",
+            "type": SchemaType.INT,
+            "default": 0
+        }, {
+            "name": "b",
+            "type": {
+                "type": SchemaType.FIXED,
+                "name": "Fixed",
+                "size": 8
+            }
+        }]
+    })
+)
+A_DINT_B_DINT_STRING_UNION_RECORD1 = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record1",
+        "fields": [{
+            "name": "a",
+            "type": SchemaType.INT,
+            "default": 0
+        }, {
+            "name": "b",
+            "type": [SchemaType.INT, SchemaType.STRING],
+            "default": 0
+        }]
+    })
+)
+A_DINT_B_DINT_UNION_RECORD1 = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record1",
+        "fields": [{
+            "name": "a",
+            "type": SchemaType.INT,
+            "default": 0
+        }, {
+            "name": "b",
+            "type": [SchemaType.INT],
+            "default": 0
+        }]
+    })
+)
+A_DINT_B_DENUM_1_RECORD1 = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record1",
+        "fields": [{
+            "name": "a",
+            "type": SchemaType.INT,
+            "default": 0
+        }, {
+            "name": "b",
+            "type": {
+                "type": SchemaType.ENUM,
+                "name": "Enum1",
+                "symbols": ["A", "B"]
+            }
+        }]
+    })
+)
+A_DINT_B_DENUM_2_RECORD1 = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record1",
+        "fields": [{
+            "name": "a",
+            "type": SchemaType.INT,
+            "default": 0
+        }, {
+            "name": "b",
+            "type": {
+                "type": SchemaType.ENUM,
+                "name": "Enum2",
+                "symbols": ["A", "B"]
+            }
+        }]
+    })
+)
+FIXED_4_BYTES = parse(json.dumps({"type": SchemaType.FIXED, "name": "Fixed", "size": 4}))
+FIXED_8_BYTES = parse(json.dumps({"type": SchemaType.FIXED, "name": "Fixed", "size": 8}))
+NS_RECORD1 = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record1",
+        "fields": [{
+            "name": "f1",
+            "type": [
+                SchemaType.NULL, {
+                    "type": SchemaType.ARRAY,
+                    "items": {
+                        "type": SchemaType.RECORD,
+                        "name": "InnerRecord1",
+                        "namespace": "ns1",
+                        "fields": [{
+                            "name": "a",
+                            "type": SchemaType.INT
+                        }]
+                    }
+                }
+            ]
+        }]
+    })
+)
+NS_RECORD2 = parse(
+    json.dumps({
+        "type": SchemaType.RECORD,
+        "name": "Record1",
+        "fields": [{
+            "name": "f1",
+            "type": [
+                SchemaType.NULL, {
+                    "type": SchemaType.ARRAY,
+                    "items": {
+                        "type": SchemaType.RECORD,
+                        "name": "InnerRecord1",
+                        "namespace": "ns2",
+                        "fields": [{
+                            "name": "a",
+                            "type": SchemaType.INT
+                        }]
+                    }
+                }
+            ]
+        }]
+    })
+)
+
+UNION_INT_RECORD1 = UnionSchema([{"type": SchemaType.INT}, {"type": SchemaType.RECORD, "name": "Record1", "fields": [{"name": "field1", "type": SchemaType.INT}]}])
+UNION_INT_RECORD2 = UnionSchema([{"type": SchemaType.INT}, {"type": "record", "name": "Record2", "fields": [{"name": "field1", "type": SchemaType.INT}]}])
+UNION_INT_ENUM1_AB = UnionSchema([{"type": SchemaType.INT}, ENUM1_AB_SCHEMA.to_json()])
+UNION_INT_FIXED_4_BYTES = UnionSchema([{"type": SchemaType.INT}, FIXED_4_BYTES.to_json()])
+UNION_INT_BOOLEAN = UnionSchema([{"type": SchemaType.INT}, {"type": SchemaType.BOOLEAN}])
+UNION_INT_ARRAY_INT = UnionSchema([{"type": SchemaType.INT}, INT_ARRAY_SCHEMA.to_json()])
+UNION_INT_MAP_INT = UnionSchema([{"type": SchemaType.INT}, INT_MAP_SCHEMA.to_json()])
+UNION_INT_NULL = UnionSchema([{"type": SchemaType.INT}, {"type": SchemaType.NULL}])
+FIXED_4_ANOTHER_NAME = parse(json.dumps({"type": SchemaType.FIXED, "name": "AnotherName", "size": 4}))

Review comment:
       These are beautifully easy-to-read tests!




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r526335951



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        that = cast(SchemaCompatibilityResult, that)
+        merged = copy(self.incompatibilities)
+        merged.extend(copy(that.incompatibilities))
+        if self.compatibility is SchemaCompatibilityType.compatible:
+            compat = that.compatibility
+            messages = that.messages
+            locations = that.locations
+        else:
+            compat = self.compatibility
+            messages = self.messages.union(that.messages)
+            locations = self.locations.union(that.locations)
+        return SchemaCompatibilityResult(
+            compatibility=compat, incompatibilities=merged, messages=messages, locations=locations
+        )
+
+    @staticmethod
+    def compatible():
+        return SchemaCompatibilityResult(SchemaCompatibilityType.compatible)
+
+    @staticmethod
+    def incompatible(incompat_type: SchemaIncompatibilityType, message: str, location: List[str]):
+        locations = "/".join(location)
+        if len(location) > 1:
+            locations = locations[1:]
+        ret = SchemaCompatibilityResult(
+            compatibility=SchemaCompatibilityType.incompatible,
+            incompatibilities=[incompat_type],
+            locations={locations},
+            messages={message},
+        )
+        return ret
+
+    def __str__(self):
+        return f"{self.compatibility}: {self.messages}"
+
+
+class ReaderWriter:
+    def __init__(self, reader: Schema, writer: Schema) -> None:
+        self.reader, self.writer = reader, writer
+
+    def __hash__(self) -> SchemaType.INT:
+        return id(self.reader) ^ id(self.writer)
+
+    def __eq__(self, other) -> bool:
+        if not isinstance(other, ReaderWriter):
+            return False
+        return self.reader is other.reader and self.writer is other.writer
+
+
+class ReaderWriterCompatibilityChecker:
+    ROOT_REFERENCE_TOKEN = "/"
+
+    def __init__(self):
+        self.memoize_map: Dict[ReaderWriter, SchemaCompatibilityResult] = {}
+
+    def get_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        reference_token: str = ROOT_REFERENCE_TOKEN,
+        location: Optional[List[str]] = None
+    ) -> SchemaCompatibilityResult:
+        if location is None:
+            location = []
+        location.append(reference_token)
+        pair = ReaderWriter(reader, writer)
+        if pair in self.memoize_map:
+            result = self.memoize_map[pair]
+            if result.compatibility is SchemaCompatibilityType.recursion_in_progress:
+                result = SchemaCompatibilityResult.compatible()
+        else:
+            self.memoize_map[pair] = SchemaCompatibilityResult()
+            result = self.calculate_compatibility(reader, writer, location)
+            self.memoize_map[pair] = result
+        location.pop()
+        return result
+
+    # pylSchemaType.INT: disable=too-many-return-statements
+    def calculate_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        location: List[str],
+    ) -> SchemaCompatibilityResult:
+        assert reader is not None
+        assert writer is not None
+        result = SchemaCompatibilityResult.compatible()
+        if reader.type == writer.type:
+            if reader.type in {
+                SchemaType.NULL, SchemaType.BOOLEAN, SchemaType.INT,
+                SchemaType.LONG, SchemaType.FLOAT, SchemaType.DOUBLE,
+                SchemaType.BYTES, SchemaType.STRING
+            }:

Review comment:
       Added as module constant




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] RyanSkraba commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
RyanSkraba commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-736597144


   That's an excellent point!  Does that mean that pip will just stop working with python 3.5 installations in January?  I know that the Databricks LTS expects python 3.5 to work until mid-2021!  (That being said, it also comes with a version of Avro pre-installed, so it's probably not our problem).
   
   My opinion is that when pip drops support for an end-of-life release, we should also drop support!  Maybe we should just create a JIRA every time, and consider the PR that updates the build environment to be the "community vote"?


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r522224299



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        that = cast(SchemaCompatibilityResult, that)
+        merged = copy(self.incompatibilities)

Review comment:
       Consider `merged = [ *self.incompatibilities, *that.incompatibilities ]`




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-739444376


   @RyanSkraba Made it work for 3.5 ... give it another look please 


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu edited a comment on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu edited a comment on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-730169687


   > This is great, thanks for taking it on! I have a few thoughts I think can improve it:
   > 
   >     1. Even though this is a port of a Java implementation, I don't think we need the static methods -- those aren't necessary in Python. Instead, consider making them "just functions" in the module.
   > 
   >     2. The major parts of the implementation would be much better if they had some docstrings explaining them.
   > 
   >     3. Mutating 'location' in a few places is a little obscure. Ideally I'd prefer to avoid mutation altogether, but it has its uses. If it's unavoidable, then if there's a way to make it visible and easy to trace it will really help debugging later on.
   
   Dropped the static methods, addressed most of the PR comments. Added some docstrings lifted from the java codebase (on the more meatier methods). Also added some comments and a new separate commit regarding the location issue


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r522221515



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()

Review comment:
       Curious why you use the ternary for the first two here, but `or X` for the last one.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r526945950



##########
File path: lang/py/avro/errors.py
##########
@@ -83,3 +83,7 @@ class UnsupportedCodec(NotImplementedError, AvroException):
 
 class UsageError(RuntimeError, AvroException):
     """An exception raised when incorrect arguments were passed."""
+
+
+class AvroRuntimeException(AvroException):

Review comment:
       Can this be a subclass of both `RuntimeError` and `AvroException` like `UsageError` above?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r527675827



##########
File path: lang/py/avro/errors.py
##########
@@ -83,3 +83,7 @@ class UnsupportedCodec(NotImplementedError, AvroException):
 
 class UsageError(RuntimeError, AvroException):
     """An exception raised when incorrect arguments were passed."""
+
+
+class AvroRuntimeException(RuntimeError, AvroException):
+    """Raised then compatibility parsing encounters an unknown type"""

Review comment:
       Fixed




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu edited a comment on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu edited a comment on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-736323751


   > Hello! I've definitely confirmed that the build on master is failing _in addition to_ the maven central connectivity issues! I'm looking at this and raised one more issue: https://issues.apache.org/jira/browse/AVRO-2994
   > 
   > You will not get a green build on Travis until that's fixed :/
   > 
   > In the meantime, I manually patched the docker image locally and tried a local build x . It looks like it's failing in python 3.5 because of the f-string syntax !
   > 
   > ```
   > ======================================================================
   > ERROR: avro.test.test_compatibility (unittest.loader._FailedTest)
   > ----------------------------------------------------------------------
   > ImportError: Failed to import test module: avro.test.test_compatibility
   > Traceback (most recent call last):
   >   File "/usr/lib/python3.5/unittest/loader.py", line 428, in _find_test_path
   >     module = self._get_module_from_name(name)
   >   File "/usr/lib/python3.5/unittest/loader.py", line 369, in _get_module_from_name
   >     __import__(name)
   >   File "/home/rskraba/avro/lang/py/avro/test/test_compatibility.py", line 565
   >     assert location in result.locations, f"expected {location}, found {result.location}"
   >                                                                                        ^
   > SyntaxError: invalid syntax
   > ```
   > 
   > Python 3.5 reached end-of-life fairly recently if I remember correctly, but we're still supporting it.
   
   @RyanSkraba Sorry about that, I missed that particular reference to a stale class member. For some reason (probably because the test passes) that did not blow up locally. I fixed the issue and repushed


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r526335765



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        that = cast(SchemaCompatibilityResult, that)
+        merged = copy(self.incompatibilities)
+        merged.extend(copy(that.incompatibilities))
+        if self.compatibility is SchemaCompatibilityType.compatible:
+            compat = that.compatibility
+            messages = that.messages
+            locations = that.locations
+        else:
+            compat = self.compatibility
+            messages = self.messages.union(that.messages)
+            locations = self.locations.union(that.locations)
+        return SchemaCompatibilityResult(
+            compatibility=compat, incompatibilities=merged, messages=messages, locations=locations
+        )
+
+    @staticmethod
+    def compatible():

Review comment:
       Removed usage of both static methods




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] RyanSkraba commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
RyanSkraba commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-735765939


   Hello!  I've definitely confirmed that the build on master is failing _in addition to_ the maven central connectivity issues!  I'm looking at this and raised one more issue: https://issues.apache.org/jira/browse/AVRO-2994
   
   You will not get a green build on Travis until that's fixed :/
   
   In the meantime, I manually patched the docker image locally and tried a local build :x: .  It looks like it's failing in python 3.5 because of the f-string syntax !
   
   ```
   ======================================================================
   ERROR: avro.test.test_compatibility (unittest.loader._FailedTest)
   ----------------------------------------------------------------------
   ImportError: Failed to import test module: avro.test.test_compatibility
   Traceback (most recent call last):
     File "/usr/lib/python3.5/unittest/loader.py", line 428, in _find_test_path
       module = self._get_module_from_name(name)
     File "/usr/lib/python3.5/unittest/loader.py", line 369, in _get_module_from_name
       __import__(name)
     File "/home/rskraba/avro/lang/py/avro/test/test_compatibility.py", line 565
       assert location in result.locations, f"expected {location}, found {result.location}"
                                                                                          ^
   SyntaxError: invalid syntax
   ```
   
   Python 3.5 reached end-of-life fairly recently if I remember correctly, but we're still supporting it.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r527020826



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,314 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.errors import AvroRuntimeException
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+PRIMITIVE_TYPES = {
+    SchemaType.NULL, SchemaType.BOOLEAN, SchemaType.INT,
+    SchemaType.LONG, SchemaType.FLOAT, SchemaType.DOUBLE,
+    SchemaType.BYTES, SchemaType.STRING
+}
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations or {"/"}
+        self.messages = messages or set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        """
+        Merges the current {@code SchemaCompatibilityResult} with the supplied result
+        into a new instance, combining the list of Incompatibilities and regressing to the
+        SchemaCompatibilityType.incompatible state if any incompatibilities are encountered.
+        :param that: SchemaCompatibilityResult
+        :return: SchemaCompatibilityResult
+        """
+        that = cast(SchemaCompatibilityResult, that)
+        merged = [*copy(self.incompatibilities), *copy(that.incompatibilities)]
+        if self.compatibility is SchemaCompatibilityType.compatible:
+            compat = that.compatibility
+            messages = that.messages
+            locations = that.locations
+        else:
+            compat = self.compatibility
+            messages = self.messages.union(that.messages)
+            locations = self.locations.union(that.locations)
+        return SchemaCompatibilityResult(
+            compatibility=compat, incompatibilities=merged, messages=messages, locations=locations
+        )
+
+
+CompatibleResult = SchemaCompatibilityResult(SchemaCompatibilityType.compatible)
+
+
+class ReaderWriter:
+    def __init__(self, reader: Schema, writer: Schema) -> None:
+        self.reader, self.writer = reader, writer
+
+    def __hash__(self) -> SchemaType.INT:
+        return id(self.reader) ^ id(self.writer)
+
+    def __eq__(self, other) -> bool:
+        if not isinstance(other, ReaderWriter):
+            return False
+        return self.reader is other.reader and self.writer is other.writer
+
+
+class ReaderWriterCompatibilityChecker:
+    ROOT_REFERENCE_TOKEN = "/"
+
+    def __init__(self):
+        self.memoize_map: Dict[ReaderWriter, SchemaCompatibilityResult] = {}
+
+    def get_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        reference_token: str = ROOT_REFERENCE_TOKEN,
+        location: Optional[List[str]] = None
+    ) -> SchemaCompatibilityResult:
+        if location is None:
+            location = []
+        pair = ReaderWriter(reader, writer)
+        if pair in self.memoize_map:
+            result = self.memoize_map[pair]
+            if result.compatibility is SchemaCompatibilityType.recursion_in_progress:
+                result = CompatibleResult
+        else:
+            self.memoize_map[pair] = SchemaCompatibilityResult()
+            result = self.calculate_compatibility(reader, writer, location + [reference_token])
+            self.memoize_map[pair] = result
+        return result
+
+    # pylSchemaType.INT: disable=too-many-return-statements
+    def calculate_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        location: List[str],
+    ) -> SchemaCompatibilityResult:
+        """
+        Calculates the compatibility of a reader/writer schema pair. Will be positive if the reader is capable of reading
+        whatever the writer may write
+        :param reader: avro.schema.Schema
+        :param writer: avro.schema.Schema
+        :param location: List[str]
+        :return: SchemaCompatibilityResult
+        """
+        assert reader is not None
+        assert writer is not None
+        result = CompatibleResult
+        if reader.type == writer.type:
+            if reader.type in PRIMITIVE_TYPES:
+                return result
+            if reader.type == SchemaType.ARRAY:
+                reader, writer = cast(ArraySchema, reader), cast(ArraySchema, writer)
+                return result.merged_with(self.get_compatibility(reader.items, writer.items, "items", location))
+            if reader.type == SchemaType.MAP:
+                reader, writer = cast(MapSchema, reader), cast(MapSchema, writer)
+                return result.merged_with(self.get_compatibility(reader.values, writer.values, "values", location))
+            if reader.type == SchemaType.FIXED:
+                reader, writer = cast(FixedSchema, reader), cast(FixedSchema, writer)
+                result = result.merged_with(self.check_schema_names(reader, writer, location))
+                return result.merged_with(self.check_fixed_size(reader, writer, location))
+            if reader.type == SchemaType.ENUM:
+                reader, writer = cast(EnumSchema, reader), cast(EnumSchema, writer)
+                result = result.merged_with(self.check_schema_names(reader, writer, location))
+                return result.merged_with(self.check_reader_enum_contains_writer_enum(reader, writer, location))
+            if reader.type == SchemaType.RECORD:
+                reader, writer = cast(RecordSchema, reader), cast(RecordSchema, writer)
+                result = result.merged_with(self.check_schema_names(reader, writer, location))
+                return result.merged_with(self.check_reader_writer_record_fields(reader, writer, location))
+            if reader.type == SchemaType.UNION:
+                reader, writer = cast(UnionSchema, reader), cast(UnionSchema, writer)
+                for i, writer_branch in enumerate(writer.schemas):
+                    compat = self.get_compatibility(reader, writer_branch)
+                    if compat.compatibility is SchemaCompatibilityType.incompatible:
+                        result = result.merged_with(
+                            incompatible(
+                                SchemaIncompatibilityType.missing_union_branch,
+                                f"reader union lacking writer type: {writer_branch.type.upper()}", location + [f"{i}"]
+                            )
+                        )
+                return result
+            raise AvroRuntimeException(f"Unknown schema type: {reader.type}")
+        if writer.type == SchemaType.UNION:
+            writer = cast(UnionSchema, writer)
+            for s in writer.schemas:
+                result = result.merged_with(self.get_compatibility(reader, s))
+            return result
+        if reader.type in {SchemaType.NULL, SchemaType.BOOLEAN, SchemaType.INT}:
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type == SchemaType.LONG:
+            if writer.type == SchemaType.INT:
+                return result
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type == SchemaType.FLOAT:
+            if writer.type in {SchemaType.INT, SchemaType.LONG}:
+                return result
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type == SchemaType.DOUBLE:
+            if writer.type in {SchemaType.INT, SchemaType.LONG, SchemaType.FLOAT}:
+                return result
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type == SchemaType.BYTES:
+            if writer.type == SchemaType.STRING:
+                return result
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type == SchemaType.STRING:
+            if writer.type == SchemaType.BYTES:
+                return result
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type in {SchemaType.ARRAY, SchemaType.MAP, SchemaType.FIXED, SchemaType.ENUM, SchemaType.RECORD}:
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type == SchemaType.UNION:
+            reader = cast(UnionSchema, reader)
+            for reader_branch in reader.schemas:
+                compat = self.get_compatibility(reader_branch, writer)
+                if compat.compatibility is SchemaCompatibilityType.compatible:
+                    return result
+            # No branch in reader compatible with writer
+            message = f"reader union lacking writer type {writer.type}"
+            return result.merged_with(
+                incompatible(
+                    SchemaIncompatibilityType.missing_union_branch, message, location
+                )
+            )
+        raise AvroRuntimeException(f"Unknown schema type: {reader.type}")
+
+    # pylSchemaType.INT: enable=too-many-return-statements
+
+    @staticmethod

Review comment:
       sorry... missed those ones...  fixed now

##########
File path: lang/py/avro/errors.py
##########
@@ -83,3 +83,7 @@ class UnsupportedCodec(NotImplementedError, AvroException):
 
 class UsageError(RuntimeError, AvroException):
     """An exception raised when incorrect arguments were passed."""
+
+
+class AvroRuntimeException(AvroException):

Review comment:
       fixed




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-730797739


   I'm good with the code. Rat, however, is failing the tests because the code doesn't have the approved license headers. Please include the header from the top of errors.py at the top of the other files.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu edited a comment on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu edited a comment on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-739444376


   @kojiromike  @RyanSkraba Made it work for 3.5 ... give it another look please . The edit is in the last commit only, which can conveniently be reverted, should 3.5 be deprecated before this gets merged


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r522233036



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        that = cast(SchemaCompatibilityResult, that)
+        merged = copy(self.incompatibilities)
+        merged.extend(copy(that.incompatibilities))
+        if self.compatibility is SchemaCompatibilityType.compatible:
+            compat = that.compatibility
+            messages = that.messages
+            locations = that.locations
+        else:
+            compat = self.compatibility
+            messages = self.messages.union(that.messages)
+            locations = self.locations.union(that.locations)
+        return SchemaCompatibilityResult(
+            compatibility=compat, incompatibilities=merged, messages=messages, locations=locations
+        )
+
+    @staticmethod
+    def compatible():
+        return SchemaCompatibilityResult(SchemaCompatibilityType.compatible)
+
+    @staticmethod
+    def incompatible(incompat_type: SchemaIncompatibilityType, message: str, location: List[str]):
+        locations = "/".join(location)
+        if len(location) > 1:
+            locations = locations[1:]
+        ret = SchemaCompatibilityResult(
+            compatibility=SchemaCompatibilityType.incompatible,
+            incompatibilities=[incompat_type],
+            locations={locations},
+            messages={message},
+        )
+        return ret
+
+    def __str__(self):
+        return f"{self.compatibility}: {self.messages}"
+
+
+class ReaderWriter:
+    def __init__(self, reader: Schema, writer: Schema) -> None:
+        self.reader, self.writer = reader, writer
+
+    def __hash__(self) -> SchemaType.INT:
+        return id(self.reader) ^ id(self.writer)
+
+    def __eq__(self, other) -> bool:
+        if not isinstance(other, ReaderWriter):
+            return False
+        return self.reader is other.reader and self.writer is other.writer
+
+
+class ReaderWriterCompatibilityChecker:
+    ROOT_REFERENCE_TOKEN = "/"
+
+    def __init__(self):
+        self.memoize_map: Dict[ReaderWriter, SchemaCompatibilityResult] = {}
+
+    def get_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        reference_token: str = ROOT_REFERENCE_TOKEN,
+        location: Optional[List[str]] = None
+    ) -> SchemaCompatibilityResult:
+        if location is None:
+            location = []
+        location.append(reference_token)
+        pair = ReaderWriter(reader, writer)
+        if pair in self.memoize_map:
+            result = self.memoize_map[pair]
+            if result.compatibility is SchemaCompatibilityType.recursion_in_progress:
+                result = SchemaCompatibilityResult.compatible()
+        else:
+            self.memoize_map[pair] = SchemaCompatibilityResult()
+            result = self.calculate_compatibility(reader, writer, location)
+            self.memoize_map[pair] = result
+        location.pop()
+        return result
+
+    # pylSchemaType.INT: disable=too-many-return-statements
+    def calculate_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        location: List[str],
+    ) -> SchemaCompatibilityResult:
+        assert reader is not None
+        assert writer is not None
+        result = SchemaCompatibilityResult.compatible()
+        if reader.type == writer.type:
+            if reader.type in {
+                SchemaType.NULL, SchemaType.BOOLEAN, SchemaType.INT,
+                SchemaType.LONG, SchemaType.FLOAT, SchemaType.DOUBLE,
+                SchemaType.BYTES, SchemaType.STRING
+            }:

Review comment:
       Should probably just have a module constant for the set of primitive types.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike merged pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike merged pull request #979:
URL: https://github.com/apache/avro/pull/979


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r522228111



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        that = cast(SchemaCompatibilityResult, that)
+        merged = copy(self.incompatibilities)
+        merged.extend(copy(that.incompatibilities))
+        if self.compatibility is SchemaCompatibilityType.compatible:
+            compat = that.compatibility
+            messages = that.messages
+            locations = that.locations
+        else:
+            compat = self.compatibility
+            messages = self.messages.union(that.messages)
+            locations = self.locations.union(that.locations)
+        return SchemaCompatibilityResult(
+            compatibility=compat, incompatibilities=merged, messages=messages, locations=locations
+        )
+
+    @staticmethod
+    def compatible():
+        return SchemaCompatibilityResult(SchemaCompatibilityType.compatible)
+
+    @staticmethod
+    def incompatible(incompat_type: SchemaIncompatibilityType, message: str, location: List[str]):

Review comment:
       I'd prefer to avoid static methods. Could this be a module-level function instead?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r527204395



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        that = cast(SchemaCompatibilityResult, that)
+        merged = copy(self.incompatibilities)
+        merged.extend(copy(that.incompatibilities))
+        if self.compatibility is SchemaCompatibilityType.compatible:
+            compat = that.compatibility
+            messages = that.messages
+            locations = that.locations
+        else:
+            compat = self.compatibility
+            messages = self.messages.union(that.messages)
+            locations = self.locations.union(that.locations)
+        return SchemaCompatibilityResult(
+            compatibility=compat, incompatibilities=merged, messages=messages, locations=locations
+        )
+
+    @staticmethod
+    def compatible():
+        return SchemaCompatibilityResult(SchemaCompatibilityType.compatible)
+
+    @staticmethod
+    def incompatible(incompat_type: SchemaIncompatibilityType, message: str, location: List[str]):
+        locations = "/".join(location)
+        if len(location) > 1:
+            locations = locations[1:]
+        ret = SchemaCompatibilityResult(
+            compatibility=SchemaCompatibilityType.incompatible,
+            incompatibilities=[incompat_type],
+            locations={locations},
+            messages={message},
+        )
+        return ret
+
+    def __str__(self):
+        return f"{self.compatibility}: {self.messages}"
+
+
+class ReaderWriter:
+    def __init__(self, reader: Schema, writer: Schema) -> None:
+        self.reader, self.writer = reader, writer
+
+    def __hash__(self) -> SchemaType.INT:
+        return id(self.reader) ^ id(self.writer)
+
+    def __eq__(self, other) -> bool:
+        if not isinstance(other, ReaderWriter):
+            return False
+        return self.reader is other.reader and self.writer is other.writer
+
+
+class ReaderWriterCompatibilityChecker:
+    ROOT_REFERENCE_TOKEN = "/"
+
+    def __init__(self):
+        self.memoize_map: Dict[ReaderWriter, SchemaCompatibilityResult] = {}
+
+    def get_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        reference_token: str = ROOT_REFERENCE_TOKEN,
+        location: Optional[List[str]] = None
+    ) -> SchemaCompatibilityResult:
+        if location is None:
+            location = []
+        location.append(reference_token)
+        pair = ReaderWriter(reader, writer)
+        if pair in self.memoize_map:
+            result = self.memoize_map[pair]
+            if result.compatibility is SchemaCompatibilityType.recursion_in_progress:
+                result = SchemaCompatibilityResult.compatible()
+        else:
+            self.memoize_map[pair] = SchemaCompatibilityResult()
+            result = self.calculate_compatibility(reader, writer, location)
+            self.memoize_map[pair] = result
+        location.pop()

Review comment:
       I opted to pass a modified list downstream. That way the code size kind of stays the same and it's still readable IMO




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-732937987


   > Hello! I retriggered travis and I'll watch this -- it looks like travis occasionally experiences broken connections to maven central since the switch to travis-ci.com! We used to see this frequently, but not ALL the time. I'm not sure what we can do in the short term...
   
   I guess i can also make some mock pushes along the way. Have you guys considered using github workflows? Also, are broken connections due to rate limiting on the maven side or something else? Thanks for looking into this !


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r527037802



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        that = cast(SchemaCompatibilityResult, that)
+        merged = copy(self.incompatibilities)
+        merged.extend(copy(that.incompatibilities))
+        if self.compatibility is SchemaCompatibilityType.compatible:
+            compat = that.compatibility
+            messages = that.messages
+            locations = that.locations
+        else:
+            compat = self.compatibility
+            messages = self.messages.union(that.messages)
+            locations = self.locations.union(that.locations)
+        return SchemaCompatibilityResult(
+            compatibility=compat, incompatibilities=merged, messages=messages, locations=locations
+        )
+
+    @staticmethod
+    def compatible():
+        return SchemaCompatibilityResult(SchemaCompatibilityType.compatible)
+
+    @staticmethod
+    def incompatible(incompat_type: SchemaIncompatibilityType, message: str, location: List[str]):
+        locations = "/".join(location)
+        if len(location) > 1:
+            locations = locations[1:]
+        ret = SchemaCompatibilityResult(
+            compatibility=SchemaCompatibilityType.incompatible,
+            incompatibilities=[incompat_type],
+            locations={locations},
+            messages={message},
+        )
+        return ret
+
+    def __str__(self):
+        return f"{self.compatibility}: {self.messages}"
+
+
+class ReaderWriter:
+    def __init__(self, reader: Schema, writer: Schema) -> None:
+        self.reader, self.writer = reader, writer
+
+    def __hash__(self) -> SchemaType.INT:
+        return id(self.reader) ^ id(self.writer)
+
+    def __eq__(self, other) -> bool:
+        if not isinstance(other, ReaderWriter):
+            return False
+        return self.reader is other.reader and self.writer is other.writer
+
+
+class ReaderWriterCompatibilityChecker:
+    ROOT_REFERENCE_TOKEN = "/"
+
+    def __init__(self):
+        self.memoize_map: Dict[ReaderWriter, SchemaCompatibilityResult] = {}
+
+    def get_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        reference_token: str = ROOT_REFERENCE_TOKEN,
+        location: Optional[List[str]] = None
+    ) -> SchemaCompatibilityResult:
+        if location is None:
+            location = []
+        location.append(reference_token)
+        pair = ReaderWriter(reader, writer)
+        if pair in self.memoize_map:
+            result = self.memoize_map[pair]
+            if result.compatibility is SchemaCompatibilityType.recursion_in_progress:
+                result = SchemaCompatibilityResult.compatible()
+        else:
+            self.memoize_map[pair] = SchemaCompatibilityResult()
+            result = self.calculate_compatibility(reader, writer, location)
+            self.memoize_map[pair] = result
+        location.pop()

Review comment:
       It's just my opinion, but I think when objects are mutated it's easier to miss things and make mistakes, like
   
   ```
   somelist = ['a', 'b', 'c']
   procedure(somelist)
   value = somelist[0]  # May raise an IndexError!
   ```
   
   Of course you can look in procedure to see if it's changing `list`, but if it's written like
   
   ```
   somelist = ['a', 'b', 'c']
   newlist = somefunc(somelist)
   value = newlist[0]
   ```
   
   It's a more explicit signal about what's happening to the data.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] RyanSkraba commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
RyanSkraba commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-732755308


   Hello!  I retriggered travis and I'll watch this -- it looks like travis occasionally experiences broken connections to maven central since the switch to travis-ci.com!  We used to see this frequently, but not ALL the time.  I'm not sure what we can do in the short term...


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r526335468



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):

Review comment:
       moved 

##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        that = cast(SchemaCompatibilityResult, that)
+        merged = copy(self.incompatibilities)

Review comment:
       updated




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r527342767



##########
File path: lang/py/avro/errors.py
##########
@@ -83,3 +83,7 @@ class UnsupportedCodec(NotImplementedError, AvroException):
 
 class UsageError(RuntimeError, AvroException):
     """An exception raised when incorrect arguments were passed."""
+
+
+class AvroRuntimeException(RuntimeError, AvroException):
+    """Raised then compatibility parsing encounters an unknown type"""

Review comment:
       Should this say "when"?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-736566396


   @RyanSkraba by the way, pip compatibility with python 3.5 ends in January. So… do we need to hold another vote, or do we have a simple process for ending support for dead snakes?


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r526336487



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        that = cast(SchemaCompatibilityResult, that)
+        merged = copy(self.incompatibilities)
+        merged.extend(copy(that.incompatibilities))
+        if self.compatibility is SchemaCompatibilityType.compatible:
+            compat = that.compatibility
+            messages = that.messages
+            locations = that.locations
+        else:
+            compat = self.compatibility
+            messages = self.messages.union(that.messages)
+            locations = self.locations.union(that.locations)
+        return SchemaCompatibilityResult(
+            compatibility=compat, incompatibilities=merged, messages=messages, locations=locations
+        )
+
+    @staticmethod
+    def compatible():
+        return SchemaCompatibilityResult(SchemaCompatibilityType.compatible)
+
+    @staticmethod
+    def incompatible(incompat_type: SchemaIncompatibilityType, message: str, location: List[str]):
+        locations = "/".join(location)
+        if len(location) > 1:
+            locations = locations[1:]
+        ret = SchemaCompatibilityResult(
+            compatibility=SchemaCompatibilityType.incompatible,
+            incompatibilities=[incompat_type],
+            locations={locations},
+            messages={message},
+        )
+        return ret
+
+    def __str__(self):
+        return f"{self.compatibility}: {self.messages}"
+
+
+class ReaderWriter:
+    def __init__(self, reader: Schema, writer: Schema) -> None:
+        self.reader, self.writer = reader, writer
+
+    def __hash__(self) -> SchemaType.INT:
+        return id(self.reader) ^ id(self.writer)
+
+    def __eq__(self, other) -> bool:
+        if not isinstance(other, ReaderWriter):
+            return False
+        return self.reader is other.reader and self.writer is other.writer
+
+
+class ReaderWriterCompatibilityChecker:
+    ROOT_REFERENCE_TOKEN = "/"
+
+    def __init__(self):
+        self.memoize_map: Dict[ReaderWriter, SchemaCompatibilityResult] = {}
+
+    def get_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        reference_token: str = ROOT_REFERENCE_TOKEN,
+        location: Optional[List[str]] = None
+    ) -> SchemaCompatibilityResult:
+        if location is None:
+            location = []
+        location.append(reference_token)
+        pair = ReaderWriter(reader, writer)
+        if pair in self.memoize_map:
+            result = self.memoize_map[pair]
+            if result.compatibility is SchemaCompatibilityType.recursion_in_progress:
+                result = SchemaCompatibilityResult.compatible()
+        else:
+            self.memoize_map[pair] = SchemaCompatibilityResult()
+            result = self.calculate_compatibility(reader, writer, location)
+            self.memoize_map[pair] = result
+        location.pop()

Review comment:
       Other than using a context manager (which seems to be overkill), the easiest solution would be to just pass in to the other parameters `location + [foo"]`. This would drop the mutability issue but would add a bit to the codebase. Also, why do you believe this is hidden?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r522220865



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):

Review comment:
       Exceptions should be declared in lang/py/avro/errors.py so that they can be identified and handled without import loops. Exceptions should also ideally inherit from more base Avro exception types.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] RyanSkraba commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
RyanSkraba commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-734358861


   Hello!  For the Github Actions issue -- I raised https://issues.apache.org/jira/browse/AVRO-2991 to discuss the impact, advantages and disadvantages!   For the unreliable Travis <-> maven repo connections, I raised https://issues.apache.org/jira/browse/AVRO-2992.  We might be able to fix this quickly by setting up a maven cache.
   
   In the meantime, I'll hit the rebuild button until I see some green (or at least some non-Java/Maven errors on your python PR!)  If you're interested, it's really quite easy to set up travis-ci.org on your own fork (so you have your own Rebuild button).
   
   This is really unfortunate!  Thanks for your patience :/


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] ivanyu commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
ivanyu commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-797462105


   Hi @kojiromike 
   Would it be possible to merge this?


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-731306782


   > I'm good with the code. [Rat](https://creadur.apache.org/rat/), however, is failing the tests because the code doesn't have the approved license headers. Please include the header from the top of errors.py at the top of the other files.
   
   @kojiromike I updated the 2 new files with the same header as `errors.py` but the pipeline is still red. After some scrolling around i bumped into this
   ```
   [ERROR] Failed to execute goal on project trevni-avro: Could not resolve dependencies for project org.apache.avro:trevni-avro:jar:1.11.0-SNAPSHOT: Could not transfer artifact org.apache.commons:commons-lang3:jar:3.7 from/to central (https://repo.maven.apache.org/maven2): Connection reset -> [Help 1]
   ```
   
   Is this a transient error? Would rerunning / repushing have a chance of getting past ?


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-736323751


   > Hello! I've definitely confirmed that the build on master is failing _in addition to_ the maven central connectivity issues! I'm looking at this and raised one more issue: https://issues.apache.org/jira/browse/AVRO-2994
   > 
   > You will not get a green build on Travis until that's fixed :/
   > 
   > In the meantime, I manually patched the docker image locally and tried a local build x . It looks like it's failing in python 3.5 because of the f-string syntax !
   > 
   > ```
   > ======================================================================
   > ERROR: avro.test.test_compatibility (unittest.loader._FailedTest)
   > ----------------------------------------------------------------------
   > ImportError: Failed to import test module: avro.test.test_compatibility
   > Traceback (most recent call last):
   >   File "/usr/lib/python3.5/unittest/loader.py", line 428, in _find_test_path
   >     module = self._get_module_from_name(name)
   >   File "/usr/lib/python3.5/unittest/loader.py", line 369, in _get_module_from_name
   >     __import__(name)
   >   File "/home/rskraba/avro/lang/py/avro/test/test_compatibility.py", line 565
   >     assert location in result.locations, f"expected {location}, found {result.location}"
   >                                                                                        ^
   > SyntaxError: invalid syntax
   > ```
   > 
   > Python 3.5 reached end-of-life fairly recently if I remember correctly, but we're still supporting it.
   
   @RyanSkraba Sorry about that, I missed that particular reference to a stale class member. For some reason (probably because the test passes) that did not blow up locally 


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-724260207


   Please be aware the py3 library is deprecated. The py lib has full support for modern python 3, and tests in multiple versions of python. Would it be possible to do this change in that library instead?


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike edited a comment on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike edited a comment on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-730797739


   I'm good with the code. [Rat](https://creadur.apache.org/rat/), however, is failing the tests because the code doesn't have the approved license headers. Please include the header from the top of errors.py at the top of the other files.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-736602189


   > Maybe we should just create a JIRA every time, and consider the PR that updates the build environment to be the "community vote"?
   
   That makes sense to me.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r522234938



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        that = cast(SchemaCompatibilityResult, that)
+        merged = copy(self.incompatibilities)
+        merged.extend(copy(that.incompatibilities))
+        if self.compatibility is SchemaCompatibilityType.compatible:
+            compat = that.compatibility
+            messages = that.messages
+            locations = that.locations
+        else:
+            compat = self.compatibility
+            messages = self.messages.union(that.messages)
+            locations = self.locations.union(that.locations)
+        return SchemaCompatibilityResult(
+            compatibility=compat, incompatibilities=merged, messages=messages, locations=locations
+        )
+
+    @staticmethod
+    def compatible():
+        return SchemaCompatibilityResult(SchemaCompatibilityType.compatible)
+
+    @staticmethod
+    def incompatible(incompat_type: SchemaIncompatibilityType, message: str, location: List[str]):
+        locations = "/".join(location)
+        if len(location) > 1:
+            locations = locations[1:]
+        ret = SchemaCompatibilityResult(
+            compatibility=SchemaCompatibilityType.incompatible,
+            incompatibilities=[incompat_type],
+            locations={locations},
+            messages={message},
+        )
+        return ret
+
+    def __str__(self):
+        return f"{self.compatibility}: {self.messages}"
+
+
+class ReaderWriter:
+    def __init__(self, reader: Schema, writer: Schema) -> None:
+        self.reader, self.writer = reader, writer
+
+    def __hash__(self) -> SchemaType.INT:
+        return id(self.reader) ^ id(self.writer)
+
+    def __eq__(self, other) -> bool:
+        if not isinstance(other, ReaderWriter):
+            return False
+        return self.reader is other.reader and self.writer is other.writer
+
+
+class ReaderWriterCompatibilityChecker:
+    ROOT_REFERENCE_TOKEN = "/"
+
+    def __init__(self):
+        self.memoize_map: Dict[ReaderWriter, SchemaCompatibilityResult] = {}
+
+    def get_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        reference_token: str = ROOT_REFERENCE_TOKEN,
+        location: Optional[List[str]] = None
+    ) -> SchemaCompatibilityResult:
+        if location is None:
+            location = []
+        location.append(reference_token)
+        pair = ReaderWriter(reader, writer)
+        if pair in self.memoize_map:
+            result = self.memoize_map[pair]
+            if result.compatibility is SchemaCompatibilityType.recursion_in_progress:
+                result = SchemaCompatibilityResult.compatible()
+        else:
+            self.memoize_map[pair] = SchemaCompatibilityResult()
+            result = self.calculate_compatibility(reader, writer, location)
+            self.memoize_map[pair] = result
+        location.pop()

Review comment:
       Is there any way to make mutating the location list more visible? I have the sense that this hidden mutation can lead to debuggability problems.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r526945205



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,314 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.errors import AvroRuntimeException
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+PRIMITIVE_TYPES = {
+    SchemaType.NULL, SchemaType.BOOLEAN, SchemaType.INT,
+    SchemaType.LONG, SchemaType.FLOAT, SchemaType.DOUBLE,
+    SchemaType.BYTES, SchemaType.STRING
+}
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations or {"/"}
+        self.messages = messages or set()
+        self.compatibility = compatibility
+        self.incompatibilities = incompatibilities or []
+
+    def merged_with(self, that):
+        """
+        Merges the current {@code SchemaCompatibilityResult} with the supplied result
+        into a new instance, combining the list of Incompatibilities and regressing to the
+        SchemaCompatibilityType.incompatible state if any incompatibilities are encountered.
+        :param that: SchemaCompatibilityResult
+        :return: SchemaCompatibilityResult
+        """
+        that = cast(SchemaCompatibilityResult, that)
+        merged = [*copy(self.incompatibilities), *copy(that.incompatibilities)]
+        if self.compatibility is SchemaCompatibilityType.compatible:
+            compat = that.compatibility
+            messages = that.messages
+            locations = that.locations
+        else:
+            compat = self.compatibility
+            messages = self.messages.union(that.messages)
+            locations = self.locations.union(that.locations)
+        return SchemaCompatibilityResult(
+            compatibility=compat, incompatibilities=merged, messages=messages, locations=locations
+        )
+
+
+CompatibleResult = SchemaCompatibilityResult(SchemaCompatibilityType.compatible)
+
+
+class ReaderWriter:
+    def __init__(self, reader: Schema, writer: Schema) -> None:
+        self.reader, self.writer = reader, writer
+
+    def __hash__(self) -> SchemaType.INT:
+        return id(self.reader) ^ id(self.writer)
+
+    def __eq__(self, other) -> bool:
+        if not isinstance(other, ReaderWriter):
+            return False
+        return self.reader is other.reader and self.writer is other.writer
+
+
+class ReaderWriterCompatibilityChecker:
+    ROOT_REFERENCE_TOKEN = "/"
+
+    def __init__(self):
+        self.memoize_map: Dict[ReaderWriter, SchemaCompatibilityResult] = {}
+
+    def get_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        reference_token: str = ROOT_REFERENCE_TOKEN,
+        location: Optional[List[str]] = None
+    ) -> SchemaCompatibilityResult:
+        if location is None:
+            location = []
+        pair = ReaderWriter(reader, writer)
+        if pair in self.memoize_map:
+            result = self.memoize_map[pair]
+            if result.compatibility is SchemaCompatibilityType.recursion_in_progress:
+                result = CompatibleResult
+        else:
+            self.memoize_map[pair] = SchemaCompatibilityResult()
+            result = self.calculate_compatibility(reader, writer, location + [reference_token])
+            self.memoize_map[pair] = result
+        return result
+
+    # pylSchemaType.INT: disable=too-many-return-statements
+    def calculate_compatibility(
+        self,
+        reader: Schema,
+        writer: Schema,
+        location: List[str],
+    ) -> SchemaCompatibilityResult:
+        """
+        Calculates the compatibility of a reader/writer schema pair. Will be positive if the reader is capable of reading
+        whatever the writer may write
+        :param reader: avro.schema.Schema
+        :param writer: avro.schema.Schema
+        :param location: List[str]
+        :return: SchemaCompatibilityResult
+        """
+        assert reader is not None
+        assert writer is not None
+        result = CompatibleResult
+        if reader.type == writer.type:
+            if reader.type in PRIMITIVE_TYPES:
+                return result
+            if reader.type == SchemaType.ARRAY:
+                reader, writer = cast(ArraySchema, reader), cast(ArraySchema, writer)
+                return result.merged_with(self.get_compatibility(reader.items, writer.items, "items", location))
+            if reader.type == SchemaType.MAP:
+                reader, writer = cast(MapSchema, reader), cast(MapSchema, writer)
+                return result.merged_with(self.get_compatibility(reader.values, writer.values, "values", location))
+            if reader.type == SchemaType.FIXED:
+                reader, writer = cast(FixedSchema, reader), cast(FixedSchema, writer)
+                result = result.merged_with(self.check_schema_names(reader, writer, location))
+                return result.merged_with(self.check_fixed_size(reader, writer, location))
+            if reader.type == SchemaType.ENUM:
+                reader, writer = cast(EnumSchema, reader), cast(EnumSchema, writer)
+                result = result.merged_with(self.check_schema_names(reader, writer, location))
+                return result.merged_with(self.check_reader_enum_contains_writer_enum(reader, writer, location))
+            if reader.type == SchemaType.RECORD:
+                reader, writer = cast(RecordSchema, reader), cast(RecordSchema, writer)
+                result = result.merged_with(self.check_schema_names(reader, writer, location))
+                return result.merged_with(self.check_reader_writer_record_fields(reader, writer, location))
+            if reader.type == SchemaType.UNION:
+                reader, writer = cast(UnionSchema, reader), cast(UnionSchema, writer)
+                for i, writer_branch in enumerate(writer.schemas):
+                    compat = self.get_compatibility(reader, writer_branch)
+                    if compat.compatibility is SchemaCompatibilityType.incompatible:
+                        result = result.merged_with(
+                            incompatible(
+                                SchemaIncompatibilityType.missing_union_branch,
+                                f"reader union lacking writer type: {writer_branch.type.upper()}", location + [f"{i}"]
+                            )
+                        )
+                return result
+            raise AvroRuntimeException(f"Unknown schema type: {reader.type}")
+        if writer.type == SchemaType.UNION:
+            writer = cast(UnionSchema, writer)
+            for s in writer.schemas:
+                result = result.merged_with(self.get_compatibility(reader, s))
+            return result
+        if reader.type in {SchemaType.NULL, SchemaType.BOOLEAN, SchemaType.INT}:
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type == SchemaType.LONG:
+            if writer.type == SchemaType.INT:
+                return result
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type == SchemaType.FLOAT:
+            if writer.type in {SchemaType.INT, SchemaType.LONG}:
+                return result
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type == SchemaType.DOUBLE:
+            if writer.type in {SchemaType.INT, SchemaType.LONG, SchemaType.FLOAT}:
+                return result
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type == SchemaType.BYTES:
+            if writer.type == SchemaType.STRING:
+                return result
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type == SchemaType.STRING:
+            if writer.type == SchemaType.BYTES:
+                return result
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type in {SchemaType.ARRAY, SchemaType.MAP, SchemaType.FIXED, SchemaType.ENUM, SchemaType.RECORD}:
+            return result.merged_with(self.type_mismatch(reader, writer, location))
+        if reader.type == SchemaType.UNION:
+            reader = cast(UnionSchema, reader)
+            for reader_branch in reader.schemas:
+                compat = self.get_compatibility(reader_branch, writer)
+                if compat.compatibility is SchemaCompatibilityType.compatible:
+                    return result
+            # No branch in reader compatible with writer
+            message = f"reader union lacking writer type {writer.type}"
+            return result.merged_with(
+                incompatible(
+                    SchemaIncompatibilityType.missing_union_branch, message, location
+                )
+            )
+        raise AvroRuntimeException(f"Unknown schema type: {reader.type}")
+
+    # pylSchemaType.INT: enable=too-many-return-statements
+
+    @staticmethod

Review comment:
       There still seem to be some staticmethods in here :/




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] kojiromike commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
kojiromike commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-731453985


   @gabriel-tincu repushing might help, but I wouldn't keep trying if it doesn't work. Maybe @Fokko or @RyanSkraba have some idea? It doesn't seem to have anything to do with this PR...


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu edited a comment on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu edited a comment on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-724290720


   > Please be aware the py3 library is deprecated. The py lib has full support for modern python 3, and tests in multiple versions of python. Would it be possible to do this change in that library instead?
   
   This will take a while... i'm converting this to draft until it's fully there


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on pull request #979:
URL: https://github.com/apache/avro/pull/979#issuecomment-730169687


   > This is great, thanks for taking it on! I have a few thoughts I think can improve it:
   > 
   >     1. Even though this is a port of a Java implementation, I don't think we need the static methods -- those aren't necessary in Python. Instead, consider making them "just functions" in the module.
   > 
   >     2. The major parts of the implementation would be much better if they had some docstrings explaining them.
   > 
   >     3. Mutating 'location' in a few places is a little obscure. Ideally I'd prefer to avoid mutation altogether, but it has its uses. If it's unavoidable, then if there's a way to make it visible and easy to trace it will really help debugging later on.
   
   Dropped the static methods, addressed most of the PR comments. Added some docstrings lifted from the java codebase (on the more meatier methods). Also added some comments wrt the location issue


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [avro] gabriel-tincu commented on a change in pull request #979: AVRO-1751: Add python3 compatibility

Posted by GitBox <gi...@apache.org>.
gabriel-tincu commented on a change in pull request #979:
URL: https://github.com/apache/avro/pull/979#discussion_r526340215



##########
File path: lang/py/avro/compatibility.py
##########
@@ -0,0 +1,318 @@
+from copy import copy
+from enum import Enum
+from typing import Dict, List, Optional, Set, cast
+
+from avro.schema import ArraySchema, EnumSchema, Field, FixedSchema, MapSchema, NamedSchema, RecordSchema, Schema, UnionSchema
+
+
+class SchemaType(str, Enum):
+    ARRAY = "array"
+    BOOLEAN = "boolean"
+    BYTES = "bytes"
+    DOUBLE = "double"
+    ENUM = "enum"
+    FIXED = "fixed"
+    FLOAT = "float"
+    INT = "int"
+    LONG = "long"
+    MAP = "map"
+    NULL = "null"
+    RECORD = "record"
+    STRING = "string"
+    UNION = "union"
+
+
+class SchemaCompatibilityType(Enum):
+    compatible = "compatible"
+    incompatible = "incompatible"
+    recursion_in_progress = "recursion_in_progress"
+
+
+class SchemaIncompatibilityType(Enum):
+    name_mismatch = "name_mismatch"
+    fixed_size_mismatch = "fixed_size_mismatch"
+    missing_enum_symbols = "missing_enum_symbols"
+    reader_field_missing_default_value = "reader_field_missing_default_value"
+    type_mismatch = "type_mismatch"
+    missing_union_branch = "missing_union_branch"
+
+
+class AvroRuntimeException(Exception):
+    pass
+
+
+class SchemaCompatibilityResult:
+    def __init__(
+        self,
+        compatibility: SchemaCompatibilityType = SchemaCompatibilityType.recursion_in_progress,
+        incompatibilities: List[SchemaIncompatibilityType] = None,
+        messages: Optional[Set[str]] = None,
+        locations: Optional[Set[str]] = None,
+    ):
+        self.locations = locations if locations else {"/"}
+        self.messages = messages if messages else set()

Review comment:
       fixed inconsistent use 




----------------------------------------------------------------
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:
users@infra.apache.org