You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "opwvhk (via GitHub)" <gi...@apache.org> on 2023/09/21 14:57:46 UTC

[GitHub] [avro] opwvhk opened a new pull request, #2513: [WIP] AVRO-3666: [JAVA] Separate parsing from Schema class

opwvhk opened a new pull request, #2513:
URL: https://github.com/apache/avro/pull/2513

   ## What is the purpose of the change
   
   Allow using pluggable parser implementations, allowing multiple formats to be parsed with the same code.
   
   
   ## Verifying this change
   
   This change adds test for all added code. The change in `IDLReader` is for readability and is covered by existing tests.
   
   Question: how can I better handle the collection parameter to the `FormattedSchemaParser` interface?
   
   
   
   ## Documentation
   
   This code introduces a new feature: the documentation (Getting Started with Java) now refers to the new code, and the Javadoc documents the rest.
   
   <!--
   
   *Thank you very much for contributing to Apache Avro - we are happy that you want to help us improve Avro. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Avro a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/AVRO/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "AVRO-XXXX: [component] Title of the pull request", where *AVRO-XXXX* should be replaced by the actual issue number. 
       The *component* is optional, but can help identify the correct reviewers faster: either the language ("java", "python") or subsystem such as "build" or "doc" are good candidates.  
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests. You can [build the entire project](https://github.com/apache/avro/blob/master/BUILD.md) or just the [language-specific SDK](https://avro.apache.org/project/how-to-contribute/#unit-tests).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Every commit message references Jira issues in their subject lines. In addition, commits follow the guidelines from [How to write a good git commit message](https://chris.beams.io/posts/git-commit/)
       1. Subject is separated from body by a blank line
       1. Subject is limited to 50 characters (not including Jira issue reference)
       1. Subject does not end with a period
       1. Subject uses the imperative mood ("add", not "adding")
       1. Body wraps at 72 characters
       1. Body explains "what" and "why", not "how"
   
   -->
   


-- 
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.

To unsubscribe, e-mail: dev-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] AVRO-3666: [JAVA] Separate parsing from Schema class [avro]

Posted by "opwvhk (via GitHub)" <gi...@apache.org>.
opwvhk commented on code in PR #2513:
URL: https://github.com/apache/avro/pull/2513#discussion_r1400248983


##########
lang/java/avro/src/main/java/org/apache/avro/ParseContext.java:
##########
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.apache.avro.util.SchemaResolver;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Class to define a name context, useful to reference schemata with. This
+ * allows for the following:
+ *
+ * <ul>
+ * <li>Provide a default namespace for nested contexts, as found for example in
+ * JSON based schema definitions.</li>
+ * <li>Find schemata by name, including primitives.</li>
+ * <li>Collect new named schemata.</li>
+ * </ul>
+ *
+ * <p>
+ * Note: this class has no use for most Avro users, but is a key component when
+ * implementing a schema parser.
+ * </p>
+ *
+ * @see <a href="https://avro.apache.org/docs/current/specification/">JSON based
+ *      schema definition</a>
+ **/
+public class ParseContext {
+  private static final Map<String, Schema.Type> PRIMITIVES = new HashMap<>();
+
+  static {
+    PRIMITIVES.put("string", Schema.Type.STRING);
+    PRIMITIVES.put("bytes", Schema.Type.BYTES);
+    PRIMITIVES.put("int", Schema.Type.INT);
+    PRIMITIVES.put("long", Schema.Type.LONG);
+    PRIMITIVES.put("float", Schema.Type.FLOAT);
+    PRIMITIVES.put("double", Schema.Type.DOUBLE);
+    PRIMITIVES.put("boolean", Schema.Type.BOOLEAN);
+    PRIMITIVES.put("null", Schema.Type.NULL);
+  }
+
+  private static final Set<Schema.Type> NAMED_SCHEMA_TYPES = EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM,
+      Schema.Type.FIXED);
+  private final Map<String, Schema> oldSchemas;
+  private final Map<String, Schema> newSchemas;
+  // Visible for use in JsonSchemaParser
+  final NameValidator nameValidator;
+  private final String namespace;

Review Comment:
   The idea behind it was indeed to handle nested namespaces. I'm not yet certain what the best option is though: should I use this as a helping hand for parsers? Or force every parser to handle its own nesting?
   
   Given your difficulties, I think I should opt for the latter (and remove it). What do you think?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] AVRO-3666: [JAVA] Separate parsing from Schema class [avro]

Posted by "opwvhk (via GitHub)" <gi...@apache.org>.
opwvhk commented on code in PR #2513:
URL: https://github.com/apache/avro/pull/2513#discussion_r1400246116


##########
lang/java/avro/src/main/java/org/apache/avro/ParseContext.java:
##########
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.apache.avro.util.SchemaResolver;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Class to define a name context, useful to reference schemata with. This
+ * allows for the following:
+ *
+ * <ul>
+ * <li>Provide a default namespace for nested contexts, as found for example in
+ * JSON based schema definitions.</li>
+ * <li>Find schemata by name, including primitives.</li>
+ * <li>Collect new named schemata.</li>
+ * </ul>
+ *
+ * <p>
+ * Note: this class has no use for most Avro users, but is a key component when
+ * implementing a schema parser.
+ * </p>
+ *
+ * @see <a href="https://avro.apache.org/docs/current/specification/">JSON based
+ *      schema definition</a>
+ **/
+public class ParseContext {
+  private static final Map<String, Schema.Type> PRIMITIVES = new HashMap<>();
+
+  static {
+    PRIMITIVES.put("string", Schema.Type.STRING);
+    PRIMITIVES.put("bytes", Schema.Type.BYTES);
+    PRIMITIVES.put("int", Schema.Type.INT);
+    PRIMITIVES.put("long", Schema.Type.LONG);
+    PRIMITIVES.put("float", Schema.Type.FLOAT);
+    PRIMITIVES.put("double", Schema.Type.DOUBLE);
+    PRIMITIVES.put("boolean", Schema.Type.BOOLEAN);
+    PRIMITIVES.put("null", Schema.Type.NULL);
+  }
+
+  private static final Set<Schema.Type> NAMED_SCHEMA_TYPES = EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM,
+      Schema.Type.FIXED);
+  private final Map<String, Schema> oldSchemas;
+  private final Map<String, Schema> newSchemas;
+  // Visible for use in JsonSchemaParser
+  final NameValidator nameValidator;
+  private final String namespace;
+
+  /**
+   * Create a {@code ParseContext} for the default/{@code null} namespace, using
+   * default name validation for new schemata.
+   */
+  public ParseContext() {
+    this(NameValidator.UTF_VALIDATOR, null);
+  }
+
+  /**
+   * Create a {@code ParseContext} for the specified namespace, using default name
+   * validation for new schemata.
+   */
+  public ParseContext(String namespace) {
+    this(NameValidator.UTF_VALIDATOR, namespace);
+  }
+
+  /**
+   * Create a {@code ParseContext} for the default/{@code null} namespace, using
+   * the specified name validation for new schemata.
+   */
+  public ParseContext(NameValidator nameValidator) {
+    this(nameValidator, null);
+  }
+
+  /**
+   * Create a {@code ParseContext} for the specified namespace, using the
+   * specified name validation for new schemata.
+   */
+  public ParseContext(NameValidator nameValidator, String namespace) {
+    this(nameValidator, new LinkedHashMap<>(), new LinkedHashMap<>(), namespace);
+  }
+
+  private ParseContext(NameValidator nameValidator, Map<String, Schema> oldSchemas, Map<String, Schema> newSchemas,
+      String namespace) {
+    this.nameValidator = nameValidator;
+    this.oldSchemas = oldSchemas;
+    this.newSchemas = newSchemas;
+    this.namespace = notEmpty(namespace) ? namespace : null;
+  }
+
+  /**
+   * Create a derived context using a different fallback namespace.
+   *
+   * @param namespace the fallback namespace to resolve names with
+   * @return a new context
+   */
+  public ParseContext namespace(String namespace) {
+    return new ParseContext(nameValidator, oldSchemas, newSchemas, namespace);
+  }
+
+  /**
+   * Return the fallback namespace.
+   *
+   * @return the namespace
+   */
+  public String namespace() {
+    return namespace;
+  }
+
+  /**
+   * Tell whether this context contains a schema with the given name.
+   *
+   * @param name a schema name
+   * @return {@code true} if the context contains a schema with this name,
+   *         {@code false} otherwise
+   */
+  public boolean contains(String name) {
+    return PRIMITIVES.containsKey(name) || oldSchemas.containsKey(name) || newSchemas.containsKey(name);
+  }
+
+  /**
+   * Resolve a schema by name. That is:
+   *
+   * <ul>
+   * <li>If {@code name} is a primitive name, return a (new) schema for it</li>
+   * <li>If {@code name} contains a dot, resolve the schema by full name only</li>
+   * <li>Otherwise: resolve the schema in the current and in the null namespace
+   * (the former takes precedence)</li>
+   * </ul>
+   *
+   * Resolving means that the schema is returned if known, and otherwise an
+   * unresolved schema (a reference) is returned.
+   *
+   * @param name the schema name to resolve
+   * @return the schema
+   * @throws SchemaParseException when the schema does not exist
+   */
+  public Schema resolve(String name) {
+    Schema.Type type = PRIMITIVES.get(name);
+    if (type != null) {
+      return Schema.create(type);
+    }
+
+    String fullName = fullName(name, namespace);
+    Schema schema = getSchema(fullName);
+    if (schema == null) {
+      schema = getSchema(name);
+    }
+
+    return schema != null ? schema : SchemaResolver.unresolvedSchema(fullName);
+  }
+
+  private Schema getSchema(String fullName) {
+    Schema schema = oldSchemas.get(fullName);
+    if (schema == null) {
+      schema = newSchemas.get(fullName);
+    }
+    return schema;
+  }
+
+  // Visible for testing
+  String fullName(String name, String space) {
+    int lastDot = name.lastIndexOf('.');
+    if (lastDot < 0) { // short name
+      if (!notEmpty(space)) {
+        space = namespace;

Review Comment:
   This is indeed one of the points where this implementation can be improved. It's already on the nomination to be removed if not needed.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [WIP] AVRO-3666: [JAVA] Separate parsing from Schema class [avro]

Posted by "github-advanced-security[bot] (via GitHub)" <gi...@apache.org>.
github-advanced-security[bot] commented on code in PR #2513:
URL: https://github.com/apache/avro/pull/2513#discussion_r1373095935


##########
lang/java/avro/src/test/java/org/apache/avro/ParseContextTest.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.EnumSet;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ParseContextTest {
+  Schema fooRecord, fooRecordCopy, barEnum, bazFixed, mehRecord;
+  ParseContext fooBarBaz;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+    fooRecord = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    fooRecordCopy = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    barEnum = SchemaBuilder.enumeration("ns.Bar").symbols();
+    bazFixed = SchemaBuilder.fixed("ns.Baz").size(8);
+    mehRecord = SchemaBuilder.record("ns.Meh").fields().endRecord();
+
+    fooBarBaz = new ParseContext();
+    fooBarBaz.put(fooRecord);
+    fooBarBaz.put(barEnum);
+    fooBarBaz.put(bazFixed);
+  }
+
+  @Test
+  public void checkNewNameContextContainsPrimitives() {
+    EnumSet<Schema.Type> complexTypes = EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM, Schema.Type.FIXED,
+        Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP);
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(complexTypes);
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : complexTypes) {
+      assertFalse(context.contains(type.getName()));
+    }
+    for (Schema.Type type : primitives) {
+      assertTrue(context.contains(type.getName()));
+    }
+  }
+
+  @Test
+  public void primitivesAreNotCached() {
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM,
+        Schema.Type.FIXED, Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP));
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : primitives) {
+      Schema first = context.resolve(type.getName());
+      Schema second = context.resolve(type.getName());
+      assertEquals(first, second);
+      assertNotSame(first, second);
+
+      first.addProp("logicalType", "brick");
+      assertNotEquals(first, second);
+    }
+  }
+
+  @Test
+  public void validateSchemaTests() {
+    assertTrue(fooBarBaz.contains(fooRecord));

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [ParseContext.contains](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/3138)



##########
lang/java/avro/src/test/java/org/apache/avro/ParseContextTest.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.EnumSet;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ParseContextTest {
+  Schema fooRecord, fooRecordCopy, barEnum, bazFixed, mehRecord;
+  ParseContext fooBarBaz;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+    fooRecord = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    fooRecordCopy = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    barEnum = SchemaBuilder.enumeration("ns.Bar").symbols();
+    bazFixed = SchemaBuilder.fixed("ns.Baz").size(8);
+    mehRecord = SchemaBuilder.record("ns.Meh").fields().endRecord();
+
+    fooBarBaz = new ParseContext();
+    fooBarBaz.put(fooRecord);
+    fooBarBaz.put(barEnum);
+    fooBarBaz.put(bazFixed);
+  }
+
+  @Test
+  public void checkNewNameContextContainsPrimitives() {
+    EnumSet<Schema.Type> complexTypes = EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM, Schema.Type.FIXED,
+        Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP);
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(complexTypes);
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : complexTypes) {
+      assertFalse(context.contains(type.getName()));
+    }
+    for (Schema.Type type : primitives) {
+      assertTrue(context.contains(type.getName()));
+    }
+  }
+
+  @Test
+  public void primitivesAreNotCached() {
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM,
+        Schema.Type.FIXED, Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP));
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : primitives) {
+      Schema first = context.resolve(type.getName());
+      Schema second = context.resolve(type.getName());
+      assertEquals(first, second);
+      assertNotSame(first, second);
+
+      first.addProp("logicalType", "brick");
+      assertNotEquals(first, second);
+    }
+  }
+
+  @Test
+  public void validateSchemaTests() {
+    assertTrue(fooBarBaz.contains(fooRecord));
+    assertTrue(fooBarBaz.contains(barEnum));

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [ParseContext.contains](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/3139)



##########
lang/java/avro/src/test/java/org/apache/avro/ParseContextTest.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.EnumSet;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ParseContextTest {
+  Schema fooRecord, fooRecordCopy, barEnum, bazFixed, mehRecord;
+  ParseContext fooBarBaz;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+    fooRecord = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    fooRecordCopy = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    barEnum = SchemaBuilder.enumeration("ns.Bar").symbols();
+    bazFixed = SchemaBuilder.fixed("ns.Baz").size(8);
+    mehRecord = SchemaBuilder.record("ns.Meh").fields().endRecord();
+
+    fooBarBaz = new ParseContext();
+    fooBarBaz.put(fooRecord);
+    fooBarBaz.put(barEnum);
+    fooBarBaz.put(bazFixed);
+  }
+
+  @Test
+  public void checkNewNameContextContainsPrimitives() {
+    EnumSet<Schema.Type> complexTypes = EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM, Schema.Type.FIXED,
+        Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP);
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(complexTypes);
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : complexTypes) {
+      assertFalse(context.contains(type.getName()));
+    }
+    for (Schema.Type type : primitives) {
+      assertTrue(context.contains(type.getName()));
+    }
+  }
+
+  @Test
+  public void primitivesAreNotCached() {
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM,
+        Schema.Type.FIXED, Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP));
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : primitives) {
+      Schema first = context.resolve(type.getName());
+      Schema second = context.resolve(type.getName());
+      assertEquals(first, second);
+      assertNotSame(first, second);
+
+      first.addProp("logicalType", "brick");
+      assertNotEquals(first, second);
+    }
+  }
+
+  @Test
+  public void validateSchemaTests() {
+    assertTrue(fooBarBaz.contains(fooRecord));
+    assertTrue(fooBarBaz.contains(barEnum));
+    assertTrue(fooBarBaz.contains(bazFixed));
+    assertFalse(fooBarBaz.contains(mehRecord));

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [ParseContext.contains](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/3141)



##########
lang/java/avro/src/test/java/org/apache/avro/ParseContextTest.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.EnumSet;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ParseContextTest {
+  Schema fooRecord, fooRecordCopy, barEnum, bazFixed, mehRecord;
+  ParseContext fooBarBaz;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+    fooRecord = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    fooRecordCopy = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    barEnum = SchemaBuilder.enumeration("ns.Bar").symbols();
+    bazFixed = SchemaBuilder.fixed("ns.Baz").size(8);
+    mehRecord = SchemaBuilder.record("ns.Meh").fields().endRecord();
+
+    fooBarBaz = new ParseContext();
+    fooBarBaz.put(fooRecord);
+    fooBarBaz.put(barEnum);
+    fooBarBaz.put(bazFixed);
+  }
+
+  @Test
+  public void checkNewNameContextContainsPrimitives() {
+    EnumSet<Schema.Type> complexTypes = EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM, Schema.Type.FIXED,
+        Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP);
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(complexTypes);
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : complexTypes) {
+      assertFalse(context.contains(type.getName()));
+    }
+    for (Schema.Type type : primitives) {
+      assertTrue(context.contains(type.getName()));
+    }
+  }
+
+  @Test
+  public void primitivesAreNotCached() {
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM,
+        Schema.Type.FIXED, Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP));
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : primitives) {
+      Schema first = context.resolve(type.getName());
+      Schema second = context.resolve(type.getName());
+      assertEquals(first, second);
+      assertNotSame(first, second);
+
+      first.addProp("logicalType", "brick");
+      assertNotEquals(first, second);
+    }
+  }
+
+  @Test
+  public void validateSchemaTests() {
+    assertTrue(fooBarBaz.contains(fooRecord));
+    assertTrue(fooBarBaz.contains(barEnum));
+    assertTrue(fooBarBaz.contains(bazFixed));
+    assertFalse(fooBarBaz.contains(mehRecord));
+
+    assertTrue(fooBarBaz.contains(fooRecord.getFullName()));
+    assertTrue(fooBarBaz.contains(barEnum.getFullName()));
+    assertTrue(fooBarBaz.contains(bazFixed.getFullName()));
+    assertFalse(fooBarBaz.contains(mehRecord.getFullName()));
+  }
+
+  @Test
+  public void validateNameResolvingAgainstDefaultNamespace() {
+    ParseContext context = new ParseContext("");
+    assertEquals("Bar", context.resolveName("Bar", ""));
+    assertEquals("Bar", context.resolveName("Bar", null));
+    assertEquals("foo.Bar", context.resolveName("Bar", "foo"));
+  }
+
+  @Test
+  public void validateNameResolvingAgainstSetNamespace() {
+    ParseContext context = new ParseContext("ns");
+    assertEquals("ns.Bar", context.resolveName("Bar", ""));
+    assertEquals("ns.Bar", context.resolveName("Bar", null));
+    assertEquals("foo.Bar", context.resolveName("Bar", "foo"));
+  }
+
+  @Test
+  public void validateSchemaRetrievalFailure() {
+    Schema unknown = Schema.createFixed("unknown", null, null, 0);
+
+    assertThrows(AvroRuntimeException.class, () -> fooBarBaz.resolve("unknown"));
+    assertSame(unknown, fooBarBaz.resolve("unknown"));
+  }
+
+  @Test
+  public void validateSchemaRetrievalByFullName() {
+    assertSame(fooRecord, fooBarBaz.resolve(fooRecord.getFullName()));
+  }
+
+  @Test
+  public void validateSchemaRetrievalByNameAndInheritedNamespace() {
+    assertSame(fooRecord, fooBarBaz.namespace(fooRecord.getNamespace()).resolve(fooRecord.getName()));
+  }
+
+  @Test
+  public void verifyPutIsIdempotent() {
+    ParseContext context = new ParseContext();
+    assertFalse(context.contains(fooRecord));
+
+    context.put(fooRecord);
+    assertTrue(context.contains(fooRecord));

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [ParseContext.contains](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/3143)



##########
lang/java/avro/src/test/java/org/apache/avro/ParseContextTest.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.EnumSet;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ParseContextTest {
+  Schema fooRecord, fooRecordCopy, barEnum, bazFixed, mehRecord;
+  ParseContext fooBarBaz;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+    fooRecord = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    fooRecordCopy = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    barEnum = SchemaBuilder.enumeration("ns.Bar").symbols();
+    bazFixed = SchemaBuilder.fixed("ns.Baz").size(8);
+    mehRecord = SchemaBuilder.record("ns.Meh").fields().endRecord();
+
+    fooBarBaz = new ParseContext();
+    fooBarBaz.put(fooRecord);
+    fooBarBaz.put(barEnum);
+    fooBarBaz.put(bazFixed);
+  }
+
+  @Test
+  public void checkNewNameContextContainsPrimitives() {
+    EnumSet<Schema.Type> complexTypes = EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM, Schema.Type.FIXED,
+        Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP);
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(complexTypes);
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : complexTypes) {
+      assertFalse(context.contains(type.getName()));
+    }
+    for (Schema.Type type : primitives) {
+      assertTrue(context.contains(type.getName()));
+    }
+  }
+
+  @Test
+  public void primitivesAreNotCached() {
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM,
+        Schema.Type.FIXED, Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP));
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : primitives) {
+      Schema first = context.resolve(type.getName());
+      Schema second = context.resolve(type.getName());
+      assertEquals(first, second);
+      assertNotSame(first, second);
+
+      first.addProp("logicalType", "brick");
+      assertNotEquals(first, second);
+    }
+  }
+
+  @Test
+  public void validateSchemaTests() {
+    assertTrue(fooBarBaz.contains(fooRecord));
+    assertTrue(fooBarBaz.contains(barEnum));
+    assertTrue(fooBarBaz.contains(bazFixed));
+    assertFalse(fooBarBaz.contains(mehRecord));
+
+    assertTrue(fooBarBaz.contains(fooRecord.getFullName()));
+    assertTrue(fooBarBaz.contains(barEnum.getFullName()));
+    assertTrue(fooBarBaz.contains(bazFixed.getFullName()));
+    assertFalse(fooBarBaz.contains(mehRecord.getFullName()));
+  }
+
+  @Test
+  public void validateNameResolvingAgainstDefaultNamespace() {
+    ParseContext context = new ParseContext("");
+    assertEquals("Bar", context.resolveName("Bar", ""));
+    assertEquals("Bar", context.resolveName("Bar", null));
+    assertEquals("foo.Bar", context.resolveName("Bar", "foo"));
+  }
+
+  @Test
+  public void validateNameResolvingAgainstSetNamespace() {
+    ParseContext context = new ParseContext("ns");
+    assertEquals("ns.Bar", context.resolveName("Bar", ""));
+    assertEquals("ns.Bar", context.resolveName("Bar", null));
+    assertEquals("foo.Bar", context.resolveName("Bar", "foo"));
+  }
+
+  @Test
+  public void validateSchemaRetrievalFailure() {
+    Schema unknown = Schema.createFixed("unknown", null, null, 0);
+
+    assertThrows(AvroRuntimeException.class, () -> fooBarBaz.resolve("unknown"));
+    assertSame(unknown, fooBarBaz.resolve("unknown"));
+  }
+
+  @Test
+  public void validateSchemaRetrievalByFullName() {
+    assertSame(fooRecord, fooBarBaz.resolve(fooRecord.getFullName()));
+  }
+
+  @Test
+  public void validateSchemaRetrievalByNameAndInheritedNamespace() {
+    assertSame(fooRecord, fooBarBaz.namespace(fooRecord.getNamespace()).resolve(fooRecord.getName()));
+  }
+
+  @Test
+  public void verifyPutIsIdempotent() {
+    ParseContext context = new ParseContext();
+    assertFalse(context.contains(fooRecord));
+
+    context.put(fooRecord);
+    assertTrue(context.contains(fooRecord));
+
+    context.put(fooRecord);
+    assertTrue(context.contains(fooRecord));

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [ParseContext.contains](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/3144)



##########
lang/java/avro/src/test/java/org/apache/avro/ParseContextTest.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.EnumSet;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ParseContextTest {
+  Schema fooRecord, fooRecordCopy, barEnum, bazFixed, mehRecord;
+  ParseContext fooBarBaz;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+    fooRecord = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    fooRecordCopy = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    barEnum = SchemaBuilder.enumeration("ns.Bar").symbols();
+    bazFixed = SchemaBuilder.fixed("ns.Baz").size(8);
+    mehRecord = SchemaBuilder.record("ns.Meh").fields().endRecord();
+
+    fooBarBaz = new ParseContext();
+    fooBarBaz.put(fooRecord);
+    fooBarBaz.put(barEnum);
+    fooBarBaz.put(bazFixed);
+  }
+
+  @Test
+  public void checkNewNameContextContainsPrimitives() {
+    EnumSet<Schema.Type> complexTypes = EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM, Schema.Type.FIXED,
+        Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP);
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(complexTypes);
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : complexTypes) {
+      assertFalse(context.contains(type.getName()));
+    }
+    for (Schema.Type type : primitives) {
+      assertTrue(context.contains(type.getName()));
+    }
+  }
+
+  @Test
+  public void primitivesAreNotCached() {
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM,
+        Schema.Type.FIXED, Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP));
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : primitives) {
+      Schema first = context.resolve(type.getName());
+      Schema second = context.resolve(type.getName());
+      assertEquals(first, second);
+      assertNotSame(first, second);
+
+      first.addProp("logicalType", "brick");
+      assertNotEquals(first, second);
+    }
+  }
+
+  @Test
+  public void validateSchemaTests() {
+    assertTrue(fooBarBaz.contains(fooRecord));
+    assertTrue(fooBarBaz.contains(barEnum));
+    assertTrue(fooBarBaz.contains(bazFixed));
+    assertFalse(fooBarBaz.contains(mehRecord));
+
+    assertTrue(fooBarBaz.contains(fooRecord.getFullName()));
+    assertTrue(fooBarBaz.contains(barEnum.getFullName()));
+    assertTrue(fooBarBaz.contains(bazFixed.getFullName()));
+    assertFalse(fooBarBaz.contains(mehRecord.getFullName()));
+  }
+
+  @Test
+  public void validateNameResolvingAgainstDefaultNamespace() {
+    ParseContext context = new ParseContext("");
+    assertEquals("Bar", context.resolveName("Bar", ""));
+    assertEquals("Bar", context.resolveName("Bar", null));
+    assertEquals("foo.Bar", context.resolveName("Bar", "foo"));
+  }
+
+  @Test
+  public void validateNameResolvingAgainstSetNamespace() {
+    ParseContext context = new ParseContext("ns");
+    assertEquals("ns.Bar", context.resolveName("Bar", ""));
+    assertEquals("ns.Bar", context.resolveName("Bar", null));
+    assertEquals("foo.Bar", context.resolveName("Bar", "foo"));
+  }
+
+  @Test
+  public void validateSchemaRetrievalFailure() {
+    Schema unknown = Schema.createFixed("unknown", null, null, 0);
+
+    assertThrows(AvroRuntimeException.class, () -> fooBarBaz.resolve("unknown"));
+    assertSame(unknown, fooBarBaz.resolve("unknown"));
+  }
+
+  @Test
+  public void validateSchemaRetrievalByFullName() {
+    assertSame(fooRecord, fooBarBaz.resolve(fooRecord.getFullName()));
+  }
+
+  @Test
+  public void validateSchemaRetrievalByNameAndInheritedNamespace() {
+    assertSame(fooRecord, fooBarBaz.namespace(fooRecord.getNamespace()).resolve(fooRecord.getName()));
+  }
+
+  @Test
+  public void verifyPutIsIdempotent() {
+    ParseContext context = new ParseContext();
+    assertFalse(context.contains(fooRecord));

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [ParseContext.contains](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/3142)



##########
lang/java/avro/src/test/java/org/apache/avro/ParseContextTest.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.EnumSet;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ParseContextTest {
+  Schema fooRecord, fooRecordCopy, barEnum, bazFixed, mehRecord;
+  ParseContext fooBarBaz;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+    fooRecord = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    fooRecordCopy = SchemaBuilder.record("ns.Foo").fields().endRecord();
+    barEnum = SchemaBuilder.enumeration("ns.Bar").symbols();
+    bazFixed = SchemaBuilder.fixed("ns.Baz").size(8);
+    mehRecord = SchemaBuilder.record("ns.Meh").fields().endRecord();
+
+    fooBarBaz = new ParseContext();
+    fooBarBaz.put(fooRecord);
+    fooBarBaz.put(barEnum);
+    fooBarBaz.put(bazFixed);
+  }
+
+  @Test
+  public void checkNewNameContextContainsPrimitives() {
+    EnumSet<Schema.Type> complexTypes = EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM, Schema.Type.FIXED,
+        Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP);
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(complexTypes);
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : complexTypes) {
+      assertFalse(context.contains(type.getName()));
+    }
+    for (Schema.Type type : primitives) {
+      assertTrue(context.contains(type.getName()));
+    }
+  }
+
+  @Test
+  public void primitivesAreNotCached() {
+    EnumSet<Schema.Type> primitives = EnumSet.complementOf(EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM,
+        Schema.Type.FIXED, Schema.Type.UNION, Schema.Type.ARRAY, Schema.Type.MAP));
+
+    ParseContext context = new ParseContext();
+    for (Schema.Type type : primitives) {
+      Schema first = context.resolve(type.getName());
+      Schema second = context.resolve(type.getName());
+      assertEquals(first, second);
+      assertNotSame(first, second);
+
+      first.addProp("logicalType", "brick");
+      assertNotEquals(first, second);
+    }
+  }
+
+  @Test
+  public void validateSchemaTests() {
+    assertTrue(fooBarBaz.contains(fooRecord));
+    assertTrue(fooBarBaz.contains(barEnum));
+    assertTrue(fooBarBaz.contains(bazFixed));

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [ParseContext.contains](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/3140)



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] AVRO-3666: [JAVA] Separate parsing from Schema class [avro]

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #2513:
URL: https://github.com/apache/avro/pull/2513#discussion_r1412775493


##########
lang/java/avro/src/test/java/org/apache/avro/util/TestSchemaResolver.java:
##########
@@ -13,32 +31,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.avro.idl;
+package org.apache.avro.util;
 
-import org.apache.avro.Protocol;
 import org.apache.avro.Schema;
 import org.apache.avro.SchemaBuilder;
-import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
 
 public class TestSchemaResolver {
 
   @Test
   public void testResolving() throws IOException {
-    Path testIdl = Paths.get(".", "src", "test", "idl", "cycle.avdl").toAbsolutePath();
-    IdlReader parser = new IdlReader();
-    IdlFile idlFile = parser.parse(testIdl);
-    Protocol protocol = idlFile.getProtocol();
-    System.out.println(protocol);
-    Assert.assertEquals(5, protocol.getTypes().size());
+    // Path testIdl = Paths.get(".", "src", "test", "idl",

Review Comment:
   Should we remove this test if it isn't being used?



##########
lang/java/avro/src/test/java/org/apache/avro/util/TestSchemaResolver.java:
##########
@@ -1,3 +1,21 @@
+/*

Review Comment:
   Double header?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] AVRO-3666: [JAVA] Separate parsing from Schema class [avro]

Posted by "clesaec (via GitHub)" <gi...@apache.org>.
clesaec commented on code in PR #2513:
URL: https://github.com/apache/avro/pull/2513#discussion_r1399296721


##########
lang/java/avro/src/main/java/org/apache/avro/ParseContext.java:
##########
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.apache.avro.util.SchemaResolver;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Class to define a name context, useful to reference schemata with. This
+ * allows for the following:
+ *
+ * <ul>
+ * <li>Provide a default namespace for nested contexts, as found for example in
+ * JSON based schema definitions.</li>
+ * <li>Find schemata by name, including primitives.</li>
+ * <li>Collect new named schemata.</li>
+ * </ul>
+ *
+ * <p>
+ * Note: this class has no use for most Avro users, but is a key component when
+ * implementing a schema parser.
+ * </p>
+ *
+ * @see <a href="https://avro.apache.org/docs/current/specification/">JSON based
+ *      schema definition</a>
+ **/
+public class ParseContext {
+  private static final Map<String, Schema.Type> PRIMITIVES = new HashMap<>();
+
+  static {
+    PRIMITIVES.put("string", Schema.Type.STRING);
+    PRIMITIVES.put("bytes", Schema.Type.BYTES);
+    PRIMITIVES.put("int", Schema.Type.INT);
+    PRIMITIVES.put("long", Schema.Type.LONG);
+    PRIMITIVES.put("float", Schema.Type.FLOAT);
+    PRIMITIVES.put("double", Schema.Type.DOUBLE);
+    PRIMITIVES.put("boolean", Schema.Type.BOOLEAN);
+    PRIMITIVES.put("null", Schema.Type.NULL);
+  }
+
+  private static final Set<Schema.Type> NAMED_SCHEMA_TYPES = EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM,
+      Schema.Type.FIXED);
+  private final Map<String, Schema> oldSchemas;
+  private final Map<String, Schema> newSchemas;
+  // Visible for use in JsonSchemaParser
+  final NameValidator nameValidator;
+  private final String namespace;

Review Comment:
   Current parsing function (`Schema parseNamesDeclared(JsonNode schema, Names names, String currentNameSpace)` embed current namespace in method signature; allowing record declaration to omit mention namespace. So, it easy to stack different level of namespaces;
   `ns1 (ns11 ... ns12) .. ns2 ...` (after ns11 is unused, back to ns1, then ns12 ...)`
   With default namespace as a simple attribute in ParseContext, i find it more difficult 
   (And this seems used only for IDLReader which has its own stack)



##########
lang/java/avro/src/main/java/org/apache/avro/ParseContext.java:
##########
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.apache.avro.util.SchemaResolver;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Class to define a name context, useful to reference schemata with. This
+ * allows for the following:
+ *
+ * <ul>
+ * <li>Provide a default namespace for nested contexts, as found for example in
+ * JSON based schema definitions.</li>
+ * <li>Find schemata by name, including primitives.</li>
+ * <li>Collect new named schemata.</li>
+ * </ul>
+ *
+ * <p>
+ * Note: this class has no use for most Avro users, but is a key component when
+ * implementing a schema parser.
+ * </p>
+ *
+ * @see <a href="https://avro.apache.org/docs/current/specification/">JSON based
+ *      schema definition</a>
+ **/
+public class ParseContext {
+  private static final Map<String, Schema.Type> PRIMITIVES = new HashMap<>();
+
+  static {
+    PRIMITIVES.put("string", Schema.Type.STRING);
+    PRIMITIVES.put("bytes", Schema.Type.BYTES);
+    PRIMITIVES.put("int", Schema.Type.INT);
+    PRIMITIVES.put("long", Schema.Type.LONG);
+    PRIMITIVES.put("float", Schema.Type.FLOAT);
+    PRIMITIVES.put("double", Schema.Type.DOUBLE);
+    PRIMITIVES.put("boolean", Schema.Type.BOOLEAN);
+    PRIMITIVES.put("null", Schema.Type.NULL);
+  }
+
+  private static final Set<Schema.Type> NAMED_SCHEMA_TYPES = EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM,
+      Schema.Type.FIXED);
+  private final Map<String, Schema> oldSchemas;
+  private final Map<String, Schema> newSchemas;
+  // Visible for use in JsonSchemaParser
+  final NameValidator nameValidator;
+  private final String namespace;
+
+  /**
+   * Create a {@code ParseContext} for the default/{@code null} namespace, using
+   * default name validation for new schemata.
+   */
+  public ParseContext() {
+    this(NameValidator.UTF_VALIDATOR, null);
+  }
+
+  /**
+   * Create a {@code ParseContext} for the specified namespace, using default name
+   * validation for new schemata.
+   */
+  public ParseContext(String namespace) {
+    this(NameValidator.UTF_VALIDATOR, namespace);
+  }
+
+  /**
+   * Create a {@code ParseContext} for the default/{@code null} namespace, using
+   * the specified name validation for new schemata.
+   */
+  public ParseContext(NameValidator nameValidator) {
+    this(nameValidator, null);
+  }
+
+  /**
+   * Create a {@code ParseContext} for the specified namespace, using the
+   * specified name validation for new schemata.
+   */
+  public ParseContext(NameValidator nameValidator, String namespace) {
+    this(nameValidator, new LinkedHashMap<>(), new LinkedHashMap<>(), namespace);
+  }
+
+  private ParseContext(NameValidator nameValidator, Map<String, Schema> oldSchemas, Map<String, Schema> newSchemas,
+      String namespace) {
+    this.nameValidator = nameValidator;
+    this.oldSchemas = oldSchemas;
+    this.newSchemas = newSchemas;
+    this.namespace = notEmpty(namespace) ? namespace : null;
+  }
+
+  /**
+   * Create a derived context using a different fallback namespace.
+   *
+   * @param namespace the fallback namespace to resolve names with
+   * @return a new context
+   */
+  public ParseContext namespace(String namespace) {
+    return new ParseContext(nameValidator, oldSchemas, newSchemas, namespace);
+  }
+
+  /**
+   * Return the fallback namespace.
+   *
+   * @return the namespace
+   */
+  public String namespace() {
+    return namespace;
+  }
+
+  /**
+   * Tell whether this context contains a schema with the given name.
+   *
+   * @param name a schema name
+   * @return {@code true} if the context contains a schema with this name,
+   *         {@code false} otherwise
+   */
+  public boolean contains(String name) {
+    return PRIMITIVES.containsKey(name) || oldSchemas.containsKey(name) || newSchemas.containsKey(name);
+  }
+
+  /**
+   * Resolve a schema by name. That is:
+   *
+   * <ul>
+   * <li>If {@code name} is a primitive name, return a (new) schema for it</li>
+   * <li>If {@code name} contains a dot, resolve the schema by full name only</li>
+   * <li>Otherwise: resolve the schema in the current and in the null namespace
+   * (the former takes precedence)</li>
+   * </ul>
+   *
+   * Resolving means that the schema is returned if known, and otherwise an
+   * unresolved schema (a reference) is returned.
+   *
+   * @param name the schema name to resolve
+   * @return the schema
+   * @throws SchemaParseException when the schema does not exist
+   */
+  public Schema resolve(String name) {
+    Schema.Type type = PRIMITIVES.get(name);
+    if (type != null) {
+      return Schema.create(type);
+    }
+
+    String fullName = fullName(name, namespace);
+    Schema schema = getSchema(fullName);
+    if (schema == null) {
+      schema = getSchema(name);
+    }
+
+    return schema != null ? schema : SchemaResolver.unresolvedSchema(fullName);
+  }
+
+  private Schema getSchema(String fullName) {
+    Schema schema = oldSchemas.get(fullName);
+    if (schema == null) {
+      schema = newSchemas.get(fullName);
+    }
+    return schema;
+  }
+
+  // Visible for testing
+  String fullName(String name, String space) {
+    int lastDot = name.lastIndexOf('.');
+    if (lastDot < 0) { // short name
+      if (!notEmpty(space)) {
+        space = namespace;

Review Comment:
   As this is called from line 162 on function resolv, with this.namespace as space ... 
   Should resolve function include namespace in signature ? as `public Schema resolve(Name name)` for example ?
   Combine with first message, does ParseContext.namespace field could be removed ?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] AVRO-3666: [JAVA] Separate parsing from Schema class [avro]

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #2513:
URL: https://github.com/apache/avro/pull/2513#discussion_r1412775467


##########
lang/java/avro/src/test/java/org/apache/avro/util/TestSchemas.java:
##########
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 /*

Review Comment:
   There seems to be a double license 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.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] AVRO-3666: [JAVA] Separate parsing from Schema class [avro]

Posted by "clesaec (via GitHub)" <gi...@apache.org>.
clesaec commented on code in PR #2513:
URL: https://github.com/apache/avro/pull/2513#discussion_r1400281529


##########
lang/java/avro/src/main/java/org/apache/avro/ParseContext.java:
##########
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.apache.avro.util.SchemaResolver;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Class to define a name context, useful to reference schemata with. This
+ * allows for the following:
+ *
+ * <ul>
+ * <li>Provide a default namespace for nested contexts, as found for example in
+ * JSON based schema definitions.</li>
+ * <li>Find schemata by name, including primitives.</li>
+ * <li>Collect new named schemata.</li>
+ * </ul>
+ *
+ * <p>
+ * Note: this class has no use for most Avro users, but is a key component when
+ * implementing a schema parser.
+ * </p>
+ *
+ * @see <a href="https://avro.apache.org/docs/current/specification/">JSON based
+ *      schema definition</a>
+ **/
+public class ParseContext {
+  private static final Map<String, Schema.Type> PRIMITIVES = new HashMap<>();
+
+  static {
+    PRIMITIVES.put("string", Schema.Type.STRING);
+    PRIMITIVES.put("bytes", Schema.Type.BYTES);
+    PRIMITIVES.put("int", Schema.Type.INT);
+    PRIMITIVES.put("long", Schema.Type.LONG);
+    PRIMITIVES.put("float", Schema.Type.FLOAT);
+    PRIMITIVES.put("double", Schema.Type.DOUBLE);
+    PRIMITIVES.put("boolean", Schema.Type.BOOLEAN);
+    PRIMITIVES.put("null", Schema.Type.NULL);
+  }
+
+  private static final Set<Schema.Type> NAMED_SCHEMA_TYPES = EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM,
+      Schema.Type.FIXED);
+  private final Map<String, Schema> oldSchemas;
+  private final Map<String, Schema> newSchemas;
+  // Visible for use in JsonSchemaParser
+  final NameValidator nameValidator;
+  private final String namespace;

Review Comment:
   Yes, i think it's possible to remove "namespace" field from ParseContext class even without changing much current code as both Schema.parseNames... and IDLReader have their own "default namespace" stack (with function call stack for the first and  `private final Deque<String> namespaces;` field for second)



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] AVRO-3666: [JAVA] Separate parsing from Schema class [avro]

Posted by "opwvhk (via GitHub)" <gi...@apache.org>.
opwvhk merged PR #2513:
URL: https://github.com/apache/avro/pull/2513


-- 
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.

To unsubscribe, e-mail: dev-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] AVRO-3666: [JAVA] Separate parsing from Schema class [avro]

Posted by "opwvhk (via GitHub)" <gi...@apache.org>.
opwvhk commented on code in PR #2513:
URL: https://github.com/apache/avro/pull/2513#discussion_r1404487009


##########
lang/java/avro/src/main/java/org/apache/avro/ParseContext.java:
##########
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro;
+
+import org.apache.avro.util.SchemaResolver;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Class to define a name context, useful to reference schemata with. This
+ * allows for the following:
+ *
+ * <ul>
+ * <li>Provide a default namespace for nested contexts, as found for example in
+ * JSON based schema definitions.</li>
+ * <li>Find schemata by name, including primitives.</li>
+ * <li>Collect new named schemata.</li>
+ * </ul>
+ *
+ * <p>
+ * Note: this class has no use for most Avro users, but is a key component when
+ * implementing a schema parser.
+ * </p>
+ *
+ * @see <a href="https://avro.apache.org/docs/current/specification/">JSON based
+ *      schema definition</a>
+ **/
+public class ParseContext {
+  private static final Map<String, Schema.Type> PRIMITIVES = new HashMap<>();
+
+  static {
+    PRIMITIVES.put("string", Schema.Type.STRING);
+    PRIMITIVES.put("bytes", Schema.Type.BYTES);
+    PRIMITIVES.put("int", Schema.Type.INT);
+    PRIMITIVES.put("long", Schema.Type.LONG);
+    PRIMITIVES.put("float", Schema.Type.FLOAT);
+    PRIMITIVES.put("double", Schema.Type.DOUBLE);
+    PRIMITIVES.put("boolean", Schema.Type.BOOLEAN);
+    PRIMITIVES.put("null", Schema.Type.NULL);
+  }
+
+  private static final Set<Schema.Type> NAMED_SCHEMA_TYPES = EnumSet.of(Schema.Type.RECORD, Schema.Type.ENUM,
+      Schema.Type.FIXED);
+  private final Map<String, Schema> oldSchemas;
+  private final Map<String, Schema> newSchemas;
+  // Visible for use in JsonSchemaParser
+  final NameValidator nameValidator;
+  private final String namespace;

Review Comment:
   Fair enough. I've removed 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.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org