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 2022/08/22 00:27:32 UTC

[GitHub] [avro] rayokota opened a new pull request, #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

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

   Add JSON encoder/decoder for C#.
   
   This PR is a straightforward port of the corresponding classes from Java.  The tests for the JSON encoder/decoder have also been ported to C#.
   
   
   ### Jira
   
   - [ x] 
     - https://issues.apache.org/jira/browse/AVRO-3001
     - https://issues.apache.org/jira/browse/AVRO-3024
   
   
   ### Tests
   
   - [ x] This PR also ports the tests for the JSON encoder/decoder from Java to C#.
   
   ### Commits
   
   - [ x] My commits all reference Jira issues in their subject lines. In addition, my 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"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes how to use it.
     - All the public functions and the classes in the PR contain Javadoc that explain what it does
   


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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r951058993


##########
lang/csharp/src/apache/main/IO/ParsingDecoder.cs:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.
+ */
+
+using Avro.IO.Parsing;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// Base class for <a href="parsing/package-summary.html">parser</a>-based

Review Comment:
   The C# compiler supports linking to a namespace with `<see cref="Parsing"/>`. I don't know whether Doxygen supports it too.



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


[GitHub] [avro] martin-g commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
martin-g commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r958398630


##########
lang/csharp/src/apache/main/IO/Encoder.cs:
##########
@@ -187,5 +187,10 @@ public interface Encoder
         /// <param name="start">Position within data where the contents start.</param>
         /// <param name="len">Number of bytes to write.</param>
         void WriteFixed(byte[] data, int start, int len);
+
+        /// <summary>
+        /// Flushes the encoder.
+        /// </summary>
+        void Flush();

Review Comment:
   Done with a60b748c



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


[GitHub] [avro] martin-g commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
martin-g commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1229887817

   > My biggest concern is that user are not able inject own custom logic without changing library. If you add more interfaces and use interface instead of concert classes as dependency, it allows users who want benefit from dependency injection use it.
   
   As far as I understand the code the encoder/decoder code does not use `static` methods/fields and it is easily extendable/overrideable.
   
   `Symbol` class has `static`s indeed! Although they could be explained as an implementation detail of the JsonDecoder I agree that it would be better if their usage is reduced as much as possible. For example adding unit tests for each new class (like JsonDecoderTests.cs, JsonEncoderTests.cs, Symbol.cs) would show us what is well designed and extendable, and what is problematic due to the usage of `static`s.
   At the moment we have only JsonCodecTests.cs which is rather higher level / functional test.
   
   To summarize my review: I think the PR needs more unit tests, especially ones testing invalid input: wrong JSON, wrong values for the expected schema, etc.


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


[GitHub] [avro] github-code-scanning[bot] commented on a diff in pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r950928308


##########
lang/csharp/src/apache/test/IO/JsonCodecTests.cs:
##########
@@ -0,0 +1,226 @@
+/**
+ * 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.
+ */
+
+using System;
+using NUnit.Framework;
+using System.IO;
+using System.Text;
+using Avro.Generic;
+using Avro.IO;
+using Newtonsoft.Json.Linq;
+
+namespace Avro.Test
+{
+    using Decoder = Avro.IO.Decoder;
+    using Encoder = Avro.IO.Encoder;
+
+    /// <summary>
+    /// Tests the JsonEncoder and JsonDecoder.
+    /// </summary>
+    [TestFixture]
+    public class JsonCodecTests
+    {
+        [TestCase]
+        public void TestJsonEncoderWhenIncludeNamespaceOptionIsFalse()
+        {
+            string value = "{\"b\": {\"string\":\"myVal\"}, \"a\": 1}";
+            string schemaStr = "{\"type\": \"record\", \"name\": \"ab\", \"fields\": [" +
+                               "{\"name\": \"a\", \"type\": \"int\"}, {\"name\": \"b\", \"type\": [\"null\", \"string\"]}" +
+                               "]}";
+            Schema schema = Schema.Parse(schemaStr);
+            byte[] avroBytes = fromJsonToAvro(value, schema);
+
+            Assert.IsTrue(JToken.DeepEquals(JObject.Parse("{\"b\":\"myVal\",\"a\":1}"),
+                JObject.Parse(fromAvroToJson(avroBytes, schema, false))));
+        }
+
+        [TestCase]
+        public void TestJsonEncoderWhenIncludeNamespaceOptionIsTrue()
+        {
+            string value = "{\"b\": {\"string\":\"myVal\"}, \"a\": 1}";
+            string schemaStr = "{\"type\": \"record\", \"name\": \"ab\", \"fields\": [" +
+                               "{\"name\": \"a\", \"type\": \"int\"}, {\"name\": \"b\", \"type\": [\"null\", \"string\"]}" +
+                               "]}";
+            Schema schema = Schema.Parse(schemaStr);
+            byte[] avroBytes = fromJsonToAvro(value, schema);
+
+            Assert.IsTrue(JToken.DeepEquals(JObject.Parse("{\"b\":{\"string\":\"myVal\"},\"a\":1}"),
+                JObject.Parse(fromAvroToJson(avroBytes, schema, true))));
+        }
+
+        [TestCase]
+        public void TestJsonRecordOrdering()
+        {
+            string value = "{\"b\": 2, \"a\": 1}";
+            Schema schema = Schema.Parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [" +
+                                         "{\"name\": \"a\", \"type\": \"int\"}, {\"name\": \"b\", \"type\": \"int\"}" +
+                                         "]}");
+            GenericDatumReader<object> reader = new GenericDatumReader<object>(schema, schema);
+            Decoder decoder = new JsonDecoder(schema, value);
+            object o = reader.Read(null, decoder);
+
+            Assert.AreEqual("{\"a\":1,\"b\":2}", fromDatumToJson(o, schema, false));
+        }
+
+        [TestCase]
+        public void TestJsonRecordOrdering2()
+        {
+            string value = "{\"b\": { \"b3\": 1.4, \"b2\": 3.14, \"b1\": \"h\"}, \"a\": {\"a2\":true, \"a1\": null}}";
+            Schema schema = Schema.Parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n" +
+                                         "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n" +
+                                         "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}},\n" +
+                                         "{\"name\": \"b\", \"type\": {\"type\":\"record\",\"name\":\"B\",\"fields\":\n" +
+                                         "[{\"name\":\"b1\", \"type\":\"string\"}, {\"name\":\"b2\", \"type\":\"float\"}, {\"name\":\"b3\", \"type\":\"double\"}]}}\n" +
+                                         "]}");
+            GenericDatumReader<object> reader = new GenericDatumReader<object>(schema, schema);
+            Decoder decoder = new JsonDecoder(schema, value);
+            object o = reader.Read(null, decoder);
+
+            Assert.AreEqual("{\"a\":{\"a1\":null,\"a2\":true},\"b\":{\"b1\":\"h\",\"b2\":3.14,\"b3\":1.4}}",
+                fromDatumToJson(o, schema, false));
+        }
+
+        [TestCase]
+        public void TestJsonRecordOrderingWithProjection()
+        {
+            String value = "{\"b\": { \"b3\": 1.4, \"b2\": 3.14, \"b1\": \"h\"}, \"a\": {\"a2\":true, \"a1\": null}}";
+            Schema writerSchema = Schema.Parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n"
+                                               + "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n"
+                                               + "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}},\n"
+                                               + "{\"name\": \"b\", \"type\": {\"type\":\"record\",\"name\":\"B\",\"fields\":\n"
+                                               + "[{\"name\":\"b1\", \"type\":\"string\"}, {\"name\":\"b2\", \"type\":\"float\"}, {\"name\":\"b3\", \"type\":\"double\"}]}}\n"
+                                               + "]}");
+            Schema readerSchema = Schema.Parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n"
+                                               + "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n"
+                                               + "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}}\n" +
+                                               "]}");
+            GenericDatumReader<object> reader = new GenericDatumReader<object>(writerSchema, readerSchema);
+            Decoder decoder = new JsonDecoder(writerSchema, value);
+            Object o = reader.Read(null, decoder);
+
+            Assert.AreEqual("{\"a\":{\"a1\":null,\"a2\":true}}",
+                fromDatumToJson(o, readerSchema, false));
+        }
+
+
+        [TestCase]
+        public void testJsonRecordOrderingWithProjection2()
+        {
+            String value =
+                "{\"b\": { \"b1\": \"h\", \"b2\": [3.14, 3.56], \"b3\": 1.4}, \"a\": {\"a2\":true, \"a1\": null}}";
+            Schema writerSchema = Schema.Parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n"
+                                               + "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n"
+                                               + "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}},\n"
+                                               + "{\"name\": \"b\", \"type\": {\"type\":\"record\",\"name\":\"B\",\"fields\":\n"
+                                               + "[{\"name\":\"b1\", \"type\":\"string\"}, {\"name\":\"b2\", \"type\":{\"type\":\"array\", \"items\":\"float\"}}, {\"name\":\"b3\", \"type\":\"double\"}]}}\n"
+                                               + "]}");
+
+            Schema readerSchema = Schema.Parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n"
+                                               + "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n"
+                                               + "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}}\n" +
+                                               "]}");
+
+            GenericDatumReader<object> reader = new GenericDatumReader<object>(writerSchema, readerSchema);
+            Decoder decoder = new JsonDecoder(writerSchema, value);
+            object o = reader.Read(null, decoder);
+
+            Assert.AreEqual("{\"a\":{\"a1\":null,\"a2\":true}}",
+                fromDatumToJson(o, readerSchema, false));
+        }
+
+        [TestCase("int", 1)]
+        [TestCase("long", 1L)]
+        [TestCase("float", 1.0F)]
+        [TestCase("double", 1.0)]
+        public void TestJsonDecoderNumeric(string type, object value)
+        {
+            string def = "{\"type\":\"record\",\"name\":\"X\",\"fields\":" + "[{\"type\":\"" + type +
+                         "\",\"name\":\"n\"}]}";
+            Schema schema = Schema.Parse(def);
+            DatumReader<GenericRecord> reader = new GenericDatumReader<GenericRecord>(schema, schema);
+
+            string[] records = { "{\"n\":1}", "{\"n\":1.0}" };
+
+            foreach (string record in records)
+            {
+                Decoder decoder = new JsonDecoder(schema, record);
+                GenericRecord r = reader.Read(null, decoder);
+                Assert.AreEqual(value, r["n"]);
+            }

Review Comment:
   ## Missed opportunity to use Select
   
   This foreach loop immediately maps its iteration variable to another variable [here](1) - consider mapping the sequence explicitly using '.Select(...)'.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2897)



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,778 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        protected class Fixup
+        {
+            public readonly Symbol[] Symbols;
+            public readonly int Pos;
+
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.Symbols = symbols;
+                this.Pos = pos;
+            }
+        }
+
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="in">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="out">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] @in, int start, Symbol[] @out, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < @in.Length; i++)
+            {
+                Symbol s = @in[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l = map2.ContainsKey((Sequence)s) ? map2[(Sequence)s] : null;
+                    if (l == null)
+                    {
+                        Array.Copy(p, 0, @out, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, @out, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(@out, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    @out[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] @out, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(@out, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        protected class Terminal : Symbol
+        {
+            public readonly string PrintName;
+
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <seealso cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;
+            }
+        }
+
+        protected class Root : Symbol
+        {
+            public Root(params Symbol[] symbols) : base(Kind.Root, makeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            public virtual Symbol Get(int index)
+            {
+                return Production[index];
+            }
+
+            public virtual int Size()
+            {
+                return Production.Length;
+            }
+
+            public IEnumerator<Symbol> GetEnumerator()
+            {
+                return Enumerable.Reverse(Production).GetEnumerator();
+            }
+
+            IEnumerator IEnumerable.GetEnumerator()
+            {
+                return this.GetEnumerator();
+            }
+
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Sequence result = map.ContainsKey(this) ? map[this] : null;

Review Comment:
   ## Inefficient use of ContainsKey
   
   Inefficient use of 'ContainsKey' and [indexer](1).
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2900)



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,778 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        protected class Fixup
+        {
+            public readonly Symbol[] Symbols;
+            public readonly int Pos;
+
+            public Fixup(Symbol[] symbols, int pos)

Review Comment:
   ## Exposing internal representation
   
   'Fixup' exposes the internal representation stored in field 'Symbols'. The value may be modified [through the variable out](1).
   'Fixup' exposes the internal representation stored in field 'Symbols'. The value may be modified [through the variable out](2).
   'Fixup' exposes the internal representation stored in field 'Symbols'. The value may be modified [through the variable out](3).
   'Fixup' exposes the internal representation stored in field 'Symbols'. The value may be modified [through the variable out](4).
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2895)



##########
lang/csharp/src/apache/main/IO/Parsing/JsonGrammarGenerator.cs:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// The class that generates a grammar suitable to parse Avro data in JSON
+    /// format.
+    /// </summary>
+    public class JsonGrammarGenerator : ValidatingGrammarGenerator
+    {
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// grammar for the given schema <tt>sc</tt>.
+        /// </summary>
+        public override Symbol Generate(Schema schema)
+        {
+            return Symbol.NewRoot(Generate(schema, new Dictionary<LitS, Symbol>()));
+        }
+
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for grammar of the given
+        /// schema <tt>sc</tt>. If there is already an entry for the given schema in the
+        /// given map <tt>seen</tt> then that entry is returned. Otherwise a new symbol
+        /// is generated and an entry is inserted into the map.
+        /// </summary>
+        /// <param name="sc">   The schema for which the start symbol is required </param>
+        /// <param name="seen"> A map of schema to symbol mapping done so far. </param>
+        /// <returns> The start symbol for the schema </returns>
+        protected override Symbol Generate(Schema sc, IDictionary<LitS, Symbol> seen)
+        {
+            switch (sc.Tag)
+            {
+                case Schema.Type.Null:
+                case Schema.Type.Boolean:
+                case Schema.Type.Int:
+                case Schema.Type.Long:
+                case Schema.Type.Float:
+                case Schema.Type.Double:
+                case Schema.Type.String:
+                case Schema.Type.Bytes:
+                case Schema.Type.Fixed:
+                case Schema.Type.Union:
+                    return base.Generate(sc, seen);
+                case Schema.Type.Enumeration:
+                    return Symbol.NewSeq(new Symbol.EnumLabelsAction(((EnumSchema)sc).Symbols), Symbol.Enum);
+                case Schema.Type.Array:
+                    return Symbol.NewSeq(
+                        Symbol.NewRepeat(Symbol.ArrayEnd, Symbol.ItemEnd, Generate(((ArraySchema)sc).ItemSchema, seen)),
+                        Symbol.ArrayStart);
+                case Schema.Type.Map:
+                    return Symbol.NewSeq(
+                        Symbol.NewRepeat(Symbol.MapEnd, Symbol.ItemEnd, Generate(((MapSchema)sc).ValueSchema, seen),
+                            Symbol.MapKeyMarker, Symbol.String), Symbol.MapStart);
+                case Schema.Type.Record:
+                    {
+                        LitS wsc = new LitS(sc);
+                        Symbol rresult = seen.ContainsKey(wsc) ? seen[wsc] : null;

Review Comment:
   ## Inefficient use of ContainsKey
   
   Inefficient use of 'ContainsKey' and [indexer](1).
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2898)



##########
lang/csharp/src/apache/main/IO/Parsing/ValidatingGrammarGenerator.cs:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// The class that generates validating grammar.
+    /// </summary>
+    public class ValidatingGrammarGenerator
+    {
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// given schema <tt>sc</tt>.
+        /// </summary>
+        public virtual Symbol Generate(Schema schema)
+        {
+            return Symbol.NewRoot(Generate(schema, new Dictionary<LitS, Symbol>()));
+        }
+
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// given schema <tt>sc</tt>. If there is already an entry for the given schema
+        /// in the given map <tt>seen</tt> then that entry is returned. Otherwise a new
+        /// symbol is generated and an entry is inserted into the map.
+        /// </summary>
+        /// <param name="sc">   The schema for which the start symbol is required </param>
+        /// <param name="seen"> A map of schema to symbol mapping done so far. </param>
+        /// <returns> The start symbol for the schema </returns>
+        protected virtual Symbol Generate(Schema sc, IDictionary<LitS, Symbol> seen)
+        {
+            switch (sc.Tag)
+            {
+                case Schema.Type.Null:
+                    return Symbol.Null;
+                case Schema.Type.Boolean:
+                    return Symbol.Boolean;
+                case Schema.Type.Int:
+                    return Symbol.Int;
+                case Schema.Type.Long:
+                    return Symbol.Long;
+                case Schema.Type.Float:
+                    return Symbol.Float;
+                case Schema.Type.Double:
+                    return Symbol.Double;
+                case Schema.Type.String:
+                    return Symbol.String;
+                case Schema.Type.Bytes:
+                    return Symbol.Bytes;
+                case Schema.Type.Fixed:
+                    return Symbol.NewSeq(new Symbol.IntCheckAction(((FixedSchema)sc).Size), Symbol.Fixed);
+                case Schema.Type.Enumeration:
+                    return Symbol.NewSeq(new Symbol.IntCheckAction(((EnumSchema)sc).Symbols.Count), Symbol.Enum);
+                case Schema.Type.Array:
+                    return Symbol.NewSeq(
+                        Symbol.NewRepeat(Symbol.ArrayEnd, Generate(((ArraySchema)sc).ItemSchema, seen)),
+                        Symbol.ArrayStart);
+                case Schema.Type.Map:
+                    return Symbol.NewSeq(
+                        Symbol.NewRepeat(Symbol.MapEnd, Generate(((MapSchema)sc).ValueSchema, seen), Symbol.String),
+                        Symbol.MapStart);
+                case Schema.Type.Record:
+                    {
+                        LitS wsc = new LitS(sc);
+                        Symbol rresult = seen.ContainsKey(wsc) ? seen[wsc] : null;
+                        if (rresult == null)
+                        {
+                            Symbol[] production = new Symbol[((RecordSchema)sc).Fields.Count];
+
+                            // We construct a symbol without filling the array. Please see
+                            // <seealso cref="Symbol.production"/> for the reason.
+                            rresult = Symbol.NewSeq(production);
+                            seen[wsc] = rresult;
+
+                            int j = production.Length;
+                            foreach (Field f in ((RecordSchema)sc).Fields)
+                            {
+                                production[--j] = Generate(f.Schema, seen);
+                            }
+                        }
+
+                        return rresult;
+                    }
+                case Schema.Type.Union:
+                    IList<Schema> subs = ((UnionSchema)sc).Schemas;
+                    Symbol[] symbols = new Symbol[subs.Count];
+                    string[] labels = new string[subs.Count];
+
+                    int i = 0;
+                    foreach (Schema b in ((UnionSchema)sc).Schemas)
+                    {
+                        symbols[i] = Generate(b, seen);
+                        labels[i] = b.Fullname;
+                        i++;
+                    }
+
+                    return Symbol.NewSeq(Symbol.NewAlt(symbols, labels), Symbol.Union);
+
+                default:
+                    throw new Exception("Unexpected schema type");
+            }
+        }
+
+        /// <summary>
+        /// A wrapper around Schema that does "==" equality. </summary>
+        protected class LitS
+        {
+            private readonly Schema actual;
+
+            public LitS(Schema actual)
+            {
+                this.actual = actual;
+            }
+
+            /// <summary>
+            /// Two LitS are equal if and only if their underlying schema is the same (not
+            /// merely equal).
+            /// </summary>
+            public override bool Equals(object o)
+            {
+                if (!(o is LitS))

Review Comment:
   ## Equals should not apply "is"
   
   LitS.Equals(object) should not use "is" on its parameter, as it will not work properly for subclasses of LitS.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2902)



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,778 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        protected class Fixup
+        {
+            public readonly Symbol[] Symbols;
+            public readonly int Pos;
+
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.Symbols = symbols;
+                this.Pos = pos;
+            }
+        }
+
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="in">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="out">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] @in, int start, Symbol[] @out, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < @in.Length; i++)
+            {
+                Symbol s = @in[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l = map2.ContainsKey((Sequence)s) ? map2[(Sequence)s] : null;

Review Comment:
   ## Inefficient use of ContainsKey
   
   Inefficient use of 'ContainsKey' and [indexer](1).
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2899)



##########
lang/csharp/src/apache/main/IO/Parsing/ValidatingGrammarGenerator.cs:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// The class that generates validating grammar.
+    /// </summary>
+    public class ValidatingGrammarGenerator
+    {
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// given schema <tt>sc</tt>.
+        /// </summary>
+        public virtual Symbol Generate(Schema schema)
+        {
+            return Symbol.NewRoot(Generate(schema, new Dictionary<LitS, Symbol>()));
+        }
+
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// given schema <tt>sc</tt>. If there is already an entry for the given schema
+        /// in the given map <tt>seen</tt> then that entry is returned. Otherwise a new
+        /// symbol is generated and an entry is inserted into the map.
+        /// </summary>
+        /// <param name="sc">   The schema for which the start symbol is required </param>
+        /// <param name="seen"> A map of schema to symbol mapping done so far. </param>
+        /// <returns> The start symbol for the schema </returns>
+        protected virtual Symbol Generate(Schema sc, IDictionary<LitS, Symbol> seen)
+        {
+            switch (sc.Tag)
+            {
+                case Schema.Type.Null:
+                    return Symbol.Null;
+                case Schema.Type.Boolean:
+                    return Symbol.Boolean;
+                case Schema.Type.Int:
+                    return Symbol.Int;
+                case Schema.Type.Long:
+                    return Symbol.Long;
+                case Schema.Type.Float:
+                    return Symbol.Float;
+                case Schema.Type.Double:
+                    return Symbol.Double;
+                case Schema.Type.String:
+                    return Symbol.String;
+                case Schema.Type.Bytes:
+                    return Symbol.Bytes;
+                case Schema.Type.Fixed:
+                    return Symbol.NewSeq(new Symbol.IntCheckAction(((FixedSchema)sc).Size), Symbol.Fixed);
+                case Schema.Type.Enumeration:
+                    return Symbol.NewSeq(new Symbol.IntCheckAction(((EnumSchema)sc).Symbols.Count), Symbol.Enum);
+                case Schema.Type.Array:
+                    return Symbol.NewSeq(
+                        Symbol.NewRepeat(Symbol.ArrayEnd, Generate(((ArraySchema)sc).ItemSchema, seen)),
+                        Symbol.ArrayStart);
+                case Schema.Type.Map:
+                    return Symbol.NewSeq(
+                        Symbol.NewRepeat(Symbol.MapEnd, Generate(((MapSchema)sc).ValueSchema, seen), Symbol.String),
+                        Symbol.MapStart);
+                case Schema.Type.Record:
+                    {
+                        LitS wsc = new LitS(sc);
+                        Symbol rresult = seen.ContainsKey(wsc) ? seen[wsc] : null;

Review Comment:
   ## Inefficient use of ContainsKey
   
   Inefficient use of 'ContainsKey' and [indexer](1).
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2901)



##########
lang/csharp/src/apache/main/IO/Parsing/JsonGrammarGenerator.cs:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// The class that generates a grammar suitable to parse Avro data in JSON
+    /// format.
+    /// </summary>
+    public class JsonGrammarGenerator : ValidatingGrammarGenerator
+    {
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// grammar for the given schema <tt>sc</tt>.
+        /// </summary>
+        public override Symbol Generate(Schema schema)
+        {
+            return Symbol.NewRoot(Generate(schema, new Dictionary<LitS, Symbol>()));
+        }
+
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for grammar of the given
+        /// schema <tt>sc</tt>. If there is already an entry for the given schema in the
+        /// given map <tt>seen</tt> then that entry is returned. Otherwise a new symbol
+        /// is generated and an entry is inserted into the map.
+        /// </summary>
+        /// <param name="sc">   The schema for which the start symbol is required </param>
+        /// <param name="seen"> A map of schema to symbol mapping done so far. </param>
+        /// <returns> The start symbol for the schema </returns>
+        protected override Symbol Generate(Schema sc, IDictionary<LitS, Symbol> seen)
+        {
+            switch (sc.Tag)
+            {
+                case Schema.Type.Null:
+                case Schema.Type.Boolean:
+                case Schema.Type.Int:
+                case Schema.Type.Long:
+                case Schema.Type.Float:
+                case Schema.Type.Double:
+                case Schema.Type.String:
+                case Schema.Type.Bytes:
+                case Schema.Type.Fixed:
+                case Schema.Type.Union:
+                    return base.Generate(sc, seen);
+                case Schema.Type.Enumeration:
+                    return Symbol.NewSeq(new Symbol.EnumLabelsAction(((EnumSchema)sc).Symbols), Symbol.Enum);
+                case Schema.Type.Array:
+                    return Symbol.NewSeq(
+                        Symbol.NewRepeat(Symbol.ArrayEnd, Symbol.ItemEnd, Generate(((ArraySchema)sc).ItemSchema, seen)),
+                        Symbol.ArrayStart);
+                case Schema.Type.Map:
+                    return Symbol.NewSeq(
+                        Symbol.NewRepeat(Symbol.MapEnd, Symbol.ItemEnd, Generate(((MapSchema)sc).ValueSchema, seen),
+                            Symbol.MapKeyMarker, Symbol.String), Symbol.MapStart);
+                case Schema.Type.Record:
+                    {
+                        LitS wsc = new LitS(sc);
+                        Symbol rresult = seen.ContainsKey(wsc) ? seen[wsc] : null;
+                        if (rresult == null)
+                        {
+                            Symbol[] production = new Symbol[((RecordSchema)sc).Fields.Count * 3 + 2];
+                            rresult = Symbol.NewSeq(production);
+                            seen[wsc] = rresult;
+
+                            int i = production.Length;
+                            int n = 0;
+                            production[--i] = Symbol.RecordStart;
+                            foreach (Field f in ((RecordSchema)sc).Fields)
+                            {
+                                production[--i] = Symbol.fieldAdjustAction(n, f.Name, f.Aliases);
+                                production[--i] = Generate(f.Schema, seen);
+                                production[--i] = Symbol.FieldEnd;
+                                n++;
+                            }
+
+                            production[--i] = Symbol.RecordEnd;

Review Comment:
   ## Useless assignment to local variable
   
   This assignment to [i](1) is useless, since its value is never read.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2896)



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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r953338043


##########
lang/csharp/src/apache/main/IO/JsonEncoder.cs:
##########
@@ -0,0 +1,356 @@
+/*
+ * 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.
+ */
+
+using Avro.IO.Parsing;
+using System.Collections;
+using System.IO;
+using System.Text;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// An <see cref="Encoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonEncoder buffers output, and data may not appear on the output until
+    /// <see cref="Encoder.Flush()"/> is called.
+    ///
+    /// JsonEncoder is not thread-safe.
+    /// </summary>
+    public class JsonEncoder : ParsingEncoder, Parser.IActionHandler
+    {
+        private readonly Parser parser;
+        private JsonWriter writer;
+        private bool includeNamespace = true;
+
+        // Has anything been written into the collections?
+        private readonly BitArray isEmpty = new BitArray(64);
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream) : this(sc, getJsonWriter(stream, false))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream, bool pretty) : this(sc, getJsonWriter(stream, pretty))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, JsonWriter writer)
+        {
+            Configure(writer);
+            this.parser = new Parser((new JsonGrammarGenerator()).Generate(sc), this);
+        }
+
+        /// <inheritdoc />
+        public override void Flush()
+        {
+            parser.ProcessImplicitActions();
+            if (writer != null)
+            {
+                writer.Flush();
+            }
+        }
+
+        // by default, one object per line.
+        // with pretty option use default pretty printer with root line separator.
+        private static JsonWriter getJsonWriter(Stream stream, bool pretty)
+        {
+            JsonWriter writer = new JsonTextWriter(new StreamWriter(stream));
+            if (pretty)
+            {
+                writer.Formatting = Formatting.Indented;
+            }
+
+            return writer;
+        }
+
+        /// <summary>
+        /// Whether to include the namespace.
+        /// </summary>
+        public virtual bool IncludeNamespace

Review Comment:
   Done



##########
lang/csharp/src/apache/main/IO/Parsing/JsonGrammarGenerator.cs:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// The class that generates a grammar suitable to parse Avro data in JSON
+    /// format.
+    /// </summary>
+    public class JsonGrammarGenerator : ValidatingGrammarGenerator
+    {
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// grammar for the given schema <tt>sc</tt>.

Review Comment:
   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.

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

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


[GitHub] [avro] KalleOlaviNiemitalo commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1225140325

   Ok so I should file a separate issue for name vs. fullname in JSON encoding of unions. Not sure which one is better but surely the spec and the implementations should be consistent with each other.


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


[GitHub] [avro] KhrystynaPopadyuk commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KhrystynaPopadyuk commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1229103595

   Hi @martin-g ,
   
   Please find my 5 cents below.
   
   As was mentioned in description this is port from Java and it looks exactly as port. Of course this code will work. But it would be very difficult maintain, extend and provide sufficient unit test coverage. 
   This is new functionality and it is easy refactor it now, and would be very difficult update if later.
   
   Please find below few suggestion and I believe should be address:
   
   1. Folder and file structure. For example:
    - Why do not create separate folder for JsonParser and placed all related classes there
    - "ParsingDecoder" (https://github.com/apache/avro/blob/c13a45debf4594dc7bc3c90aef64cee5ae115a5d/lang/csharp/src/apache/main/IO/ParsingDecoder.cs) It stated that this is "Base class for a <see cref="Parsing.Parser"/>-based" but Parser (https://github.com/apache/avro/blob/c13a45debf4594dc7bc3c90aef64cee5ae115a5d/lang/csharp/src/apache/main/IO/Parsing/Parser.cs) does no have base class. Moreover ParsingDecoder is base class for JsonDecoder. So is "ParsingDecoder " correct name? What is the purpose of having this class? Why we need this additional layer of abstraction?
    - there are a lot of child classes and interfaces. For example file Symbol.cs contains not only abstract class Symbol but 17 child classes and one enum, And all definitions of fields, properties, child classes, enums, methods are without any structure and order. 
    - there are a lot of other things that need attention
   2. Static method and classes is very controversial tool. It has own benefits but also it can brink a lot of negative. Overall that should be use them carefully and with attention. I have founded 38 static items in Symbol class. That looks very suspicious. 
   3. Poor usage of interfaces but a lot of static and even hidden dependencies. That should be reviewed in direction to use interfaces as dependency instead of classes (even abstract). Dependency inversion/ dependency injection can bring a lot of benefit and flexibility to code.
   4. switch operator - is there way to reduce usage of switch/case?
   
   
   


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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r954605087


##########
lang/csharp/src/apache/main/IO/Encoder.cs:
##########
@@ -187,5 +187,10 @@ public interface Encoder
         /// <param name="start">Position within data where the contents start.</param>
         /// <param name="len">Number of bytes to write.</param>
         void WriteFixed(byte[] data, int start, int len);
+
+        /// <summary>
+        /// Flushes the encoder.
+        /// </summary>
+        void Flush();

Review Comment:
   No, I think it is OK to keep. But if you need the JSON encoder in branch-1.11, then Flush should be omitted from that 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.

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

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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r951039578


##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,787 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <seealso cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// <p/>
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>

Review Comment:
   Not important: The [para](https://docs.microsoft.com/dotnet/csharp/language-reference/xmldoc/recommended-tags#para) element would be more standard in C# XML documentation comments, and is apparently [supported by Doxygen](https://doxygen.nl/manual/xmlcmds.html).



##########
lang/csharp/src/apache/main/IO/Parsing/Parser.cs:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.
+ */
+
+using System;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Parser is the class that maintains the stack for parsing. This class is used
+    /// by encoders, which are not required to skip.
+    /// </summary>
+    public class Parser
+    {
+        /// <summary>
+        /// The parser knows how to handle the terminal and non-terminal symbols. But it
+        /// needs help from outside to handle implicit and explicit actions. The clients
+        /// implement this interface to provide this help.
+        /// </summary>
+        public interface ActionHandler

Review Comment:
   Please name interfaces starting with "I" according to the .NET convention, at least if they are visible outside the assembly. Likewise in SkipParser.SkipHandler.



##########
lang/csharp/src/apache/main/IO/JsonEncoder.cs:
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.
+ */
+
+using Avro.IO.Parsing;
+using System.Collections;
+using System.IO;
+using System.Text;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// An <seealso cref="Encoder"/> for Avro's JSON data encoding.

Review Comment:
   For inline links, please use [see](https://docs.microsoft.com/en-us/dotnet/csharp/language-reference/xmldoc/recommended-tags#see) rather than seealso.



##########
lang/csharp/src/apache/main/IO/JsonEncoder.cs:
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.
+ */
+
+using Avro.IO.Parsing;
+using System.Collections;
+using System.IO;
+using System.Text;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// An <seealso cref="Encoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonEncoder buffers output, and data may not appear on the output until
+    /// <seealso cref="Encoder.Flush()"/> is called.
+    ///
+    /// JsonEncoder is not thread-safe.
+    /// </summary>
+    public class JsonEncoder : ParsingEncoder, Parser.ActionHandler
+    {
+        private readonly Parser parser;
+        private JsonWriter writer;
+        private bool includeNamespace = true;
+
+        // Has anything been written into the collections?
+        private readonly BitArray isEmpty = new BitArray(64);
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream) : this(sc, getJsonWriter(stream, false))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream, bool pretty) : this(sc, getJsonWriter(stream, pretty))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, JsonWriter writer)
+        {
+            Configure(writer);
+            this.parser = new Parser((new JsonGrammarGenerator()).Generate(sc), this);
+        }
+
+        /// <inheritdoc />
+        public override void Flush()
+        {
+            parser.ProcessImplicitActions();
+            if (writer != null)
+            {
+                writer.Flush();
+            }
+        }
+
+        // by default, one object per line.
+        // with pretty option use default pretty printer with root line separator.
+        private static JsonWriter getJsonWriter(Stream stream, bool pretty)
+        {
+            JsonWriter writer = new JsonTextWriter(new StreamWriter(stream));
+            if (pretty)
+            {
+                writer.Formatting = Formatting.Indented;
+            }
+
+            return writer;
+        }
+
+        /// <summary>
+        /// Whether to include the namespace.
+        /// </summary>
+        public virtual bool IncludeNamespace
+        {
+            get { return includeNamespace; }
+            set { this.includeNamespace = value; }
+        }
+
+
+        /// <summary>
+        /// Reconfigures this JsonEncoder to use the output stream provided.
+        /// <p/>
+        /// Otherwise, this JsonEncoder will flush its current output and then
+        /// reconfigure its output to use a default UTF8 JsonWriter that writes to the
+        /// provided OutputStream.

Review Comment:
   OutputStream is a Java thing



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,787 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <seealso cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// <p/>
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// <p/>
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);
+                int n = top.FindLabel(label);
+                if (n >= 0)
+                {
+                    reader.Read();
+                    return n;
+                }
+
+                throw new AvroTypeException("Unknown symbol in enum " + label);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayStart()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Read();
+                return doArrayNext();
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doArrayNext();
+        }
+
+        private long doArrayNext()
+        {
+            if (reader.TokenType == JsonToken.EndArray)
+            {
+                Parser.Advance(Symbol.ArrayEnd);
+                reader.Read();
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipArray()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.ArrayEnd);
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapStart()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                return doMapNext();
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doMapNext();
+        }
+
+        private long doMapNext()
+        {
+            if (reader.TokenType == JsonToken.EndObject)
+            {
+                reader.Read();
+                advance(Symbol.MapEnd);
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipMap()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.MapEnd);
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadUnionIndex()
+        {
+            advance(Symbol.Union);
+            Symbol.Alternative a = (Symbol.Alternative)Parser.PopSymbol();
+
+            string label;
+            if (reader.TokenType == JsonToken.Null)
+            {
+                label = "null";
+            }
+            else if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    label = Convert.ToString(reader.Value);
+                    reader.Read();
+                    Parser.PushSymbol(Symbol.UnionEnd);
+                }
+                else
+                {
+                    throw error("start-union");
+                }
+            }
+            else
+            {
+                throw error("start-union");
+            }
+
+            int n = a.FindLabel(label);
+            if (n < 0)
+            {
+                throw new AvroTypeException("Unknown union branch " + label);
+            }
+
+            Parser.PushSymbol(a.GetSymbol(n));
+            return n;
+        }
+
+        /// <inheritdoc />
+        public override void SkipNull()
+        {
+            ReadNull();
+        }
+
+        /// <inheritdoc />
+        public override void SkipBoolean()
+        {
+            ReadBoolean();
+        }
+
+        /// <inheritdoc />
+        public override void SkipInt()
+        {
+            ReadInt();
+        }
+
+        /// <inheritdoc />
+        public override void SkipLong()
+        {
+            ReadLong();
+        }
+
+        /// <inheritdoc />
+        public override void SkipFloat()
+        {
+            ReadFloat();
+        }
+
+        /// <inheritdoc />
+        public override void SkipDouble()
+        {
+            ReadDouble();
+        }
+
+        /// <inheritdoc />
+        public override void SkipEnum()
+        {
+            ReadEnum();
+        }
+
+        /// <inheritdoc />
+        public override void SkipUnionIndex()
+        {
+            ReadUnionIndex();
+        }
+
+        /// <inheritdoc />
+        public override Symbol DoAction(Symbol input, Symbol top)
+        {
+            if (top is Symbol.FieldAdjustAction)
+            {
+                Symbol.FieldAdjustAction fa = (Symbol.FieldAdjustAction)top;
+                string name = fa.FName;
+                if (currentReorderBuffer != null)
+                {
+                    IList<JsonElement> node = currentReorderBuffer.SavedFields[name];
+                    if (node != null)
+                    {
+                        currentReorderBuffer.SavedFields.Remove(name);
+                        currentReorderBuffer.OrigParser = reader;
+                        reader = makeParser(node);
+                        return null;
+                    }
+                }
+
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    do
+                    {
+                        string fn = Convert.ToString(reader.Value);
+                        reader.Read();
+                        if (name.Equals(fn) || (fa.Aliases != null && fa.Aliases.Contains(fn)))
+                        {
+                            return null;
+                        }
+                        else
+                        {
+                            if (currentReorderBuffer == null)
+                            {
+                                currentReorderBuffer = new ReorderBuffer();
+                            }
+
+                            currentReorderBuffer.SavedFields[fn] = getValueAsTree(reader);
+                        }
+                    } while (reader.TokenType == JsonToken.PropertyName);
+
+                    throw new AvroTypeException("Expected field name not found: " + fa.FName);
+                }
+            }
+            else if (top == Symbol.FieldEnd)
+            {
+                if (currentReorderBuffer != null && currentReorderBuffer.OrigParser != null)
+                {
+                    reader = currentReorderBuffer.OrigParser;
+                    currentReorderBuffer.OrigParser = null;
+                }
+            }
+            else if (top == Symbol.RecordStart)
+            {
+                if (reader.TokenType == JsonToken.StartObject)
+                {
+                    reader.Read();
+                    reorderBuffers.Push(currentReorderBuffer);
+                    currentReorderBuffer = null;
+                }
+                else
+                {
+                    throw error("record-start");
+                }
+            }
+            else if (top == Symbol.RecordEnd || top == Symbol.UnionEnd)
+            {
+                // AVRO-2034 advance to the end of our object
+                while (reader.TokenType != JsonToken.EndObject)
+                {
+                    reader.Read();
+                }
+
+                if (top == Symbol.RecordEnd)
+                {
+                    if (currentReorderBuffer != null && currentReorderBuffer.SavedFields.Count > 0)
+                    {
+                        throw error("Unknown fields: " + currentReorderBuffer.SavedFields.Keys);
+                    }
+
+                    currentReorderBuffer = reorderBuffers.Pop();
+                }
+
+                // AVRO-2034 advance beyond the end object for the next record.
+                reader.Read();
+            }
+            else
+            {
+                throw new AvroTypeException("Unknown action symbol " + top);
+            }
+
+            return null;
+        }
+
+
+        private class JsonElement
+        {
+            public readonly JsonToken Token;
+            public readonly object Value;
+
+            public JsonElement(JsonToken t, object value)
+            {
+                this.Token = t;
+                this.Value = value;
+            }
+
+            public JsonElement(JsonToken t) : this(t, null)
+            {
+            }
+        }
+
+        private static IList<JsonElement> getValueAsTree(JsonReader reader)
+        {
+            int level = 0;
+            IList<JsonElement> result = new List<JsonElement>();
+            do
+            {
+                JsonToken t = reader.TokenType;
+                switch (t)
+                {
+                    case JsonToken.StartObject:
+                    case JsonToken.StartArray:
+                        level++;
+                        result.Add(new JsonElement(t));
+                        break;
+                    case JsonToken.EndObject:
+                    case JsonToken.EndArray:
+                        level--;
+                        result.Add(new JsonElement(t));
+                        break;
+                    case JsonToken.PropertyName:
+                    case JsonToken.String:
+                    case JsonToken.Integer:
+                    case JsonToken.Float:
+                    case JsonToken.Boolean:
+                    case JsonToken.Null:
+                        result.Add(new JsonElement(t, reader.Value));
+                        break;
+                }
+
+                reader.Read();
+            } while (level != 0);
+
+            result.Add(new JsonElement(JsonToken.None));
+            return result;
+        }
+
+        private JsonReader makeParser(in IList<JsonElement> elements)
+        {
+            return new JsonElementReader(elements);
+        }
+
+        private class JsonElementReader : JsonReader
+        {
+            private readonly IList<JsonElement> elements;
+
+            public JsonElementReader(IList<JsonElement> elements)
+            {
+                this.elements = elements;
+                pos = 0;
+            }
+
+            private int pos;
+
+            public override object Value
+            {
+                get { return elements[pos].Value; }
+            }
+
+            public override JsonToken TokenType
+            {
+                get { return elements[pos].Token; }
+            }
+
+            public override bool Read()
+            {
+                pos++;
+                return true;
+            }
+
+            public new void Skip()

Review Comment:
   Is JsonElementReader.Skip() ever called? JsonDecoder.SkipArray() and JsonDecoder.SkipMap() can call reader.Skip(), but that refers to the field `private JsonReader reader;`, so the call goes to [JsonReader.Skip()](https://www.newtonsoft.com/json/help/html/M_Newtonsoft_Json_JsonReader_Skip.htm), which is not virtual.



##########
lang/csharp/src/apache/main/IO/ParsingDecoder.cs:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.
+ */
+
+using Avro.IO.Parsing;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// Base class for <a href="parsing/package-summary.html">parser</a>-based

Review Comment:
   Not sure what "parsing/package-summary.html" refers to. Likewise in ParsingEncoder.



##########
lang/csharp/src/apache/main/IO/JsonEncoder.cs:
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.
+ */
+
+using Avro.IO.Parsing;
+using System.Collections;
+using System.IO;
+using System.Text;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// An <seealso cref="Encoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonEncoder buffers output, and data may not appear on the output until
+    /// <seealso cref="Encoder.Flush()"/> is called.
+    ///
+    /// JsonEncoder is not thread-safe.
+    /// </summary>
+    public class JsonEncoder : ParsingEncoder, Parser.ActionHandler
+    {
+        private readonly Parser parser;
+        private JsonWriter writer;
+        private bool includeNamespace = true;
+
+        // Has anything been written into the collections?
+        private readonly BitArray isEmpty = new BitArray(64);
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream) : this(sc, getJsonWriter(stream, false))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream, bool pretty) : this(sc, getJsonWriter(stream, pretty))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, JsonWriter writer)
+        {
+            Configure(writer);
+            this.parser = new Parser((new JsonGrammarGenerator()).Generate(sc), this);
+        }
+
+        /// <inheritdoc />
+        public override void Flush()
+        {
+            parser.ProcessImplicitActions();
+            if (writer != null)
+            {
+                writer.Flush();
+            }
+        }
+
+        // by default, one object per line.
+        // with pretty option use default pretty printer with root line separator.
+        private static JsonWriter getJsonWriter(Stream stream, bool pretty)
+        {
+            JsonWriter writer = new JsonTextWriter(new StreamWriter(stream));
+            if (pretty)
+            {
+                writer.Formatting = Formatting.Indented;
+            }
+
+            return writer;
+        }
+
+        /// <summary>
+        /// Whether to include the namespace.
+        /// </summary>
+        public virtual bool IncludeNamespace
+        {
+            get { return includeNamespace; }
+            set { this.includeNamespace = value; }
+        }
+
+
+        /// <summary>
+        /// Reconfigures this JsonEncoder to use the output stream provided.
+        /// <p/>
+        /// Otherwise, this JsonEncoder will flush its current output and then
+        /// reconfigure its output to use a default UTF8 JsonWriter that writes to the
+        /// provided OutputStream.
+        /// </summary>
+        /// <param name="stream"> The OutputStream to direct output to. Cannot be null. </param>
+        /// <returns> this JsonEncoder </returns>
+        public JsonEncoder Configure(Stream stream)
+        {
+            this.Configure(getJsonWriter(stream, false));
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonEncoder to output to the JsonWriter provided.
+        /// <p/>
+        /// Otherwise, this JsonEncoder will flush its current output and then
+        /// reconfigure its output to use the provided JsonWriter.
+        /// </summary>
+        /// <param name="jsonWriter"> The JsonWriter to direct output to. Cannot be null. </param>
+        /// <returns> this JsonEncoder </returns>
+        public JsonEncoder Configure(JsonWriter jsonWriter)
+        {
+            if (null != parser)
+            {
+                Flush();
+            }
+
+            this.writer = jsonWriter;
+            return this;
+        }
+
+        /// <inheritdoc />
+        public override void WriteNull()
+        {
+            parser.Advance(Symbol.Null);
+            writer.WriteNull();
+        }
+
+        /// <inheritdoc />
+        public override void WriteBoolean(bool b)
+        {
+            parser.Advance(Symbol.Boolean);
+            writer.WriteValue(b);
+        }
+
+        /// <inheritdoc />
+        public override void WriteInt(int n)
+        {
+            parser.Advance(Symbol.Int);
+            writer.WriteValue(n);
+        }
+
+        /// <inheritdoc />
+        public override void WriteLong(long n)
+        {
+            parser.Advance(Symbol.Long);
+            writer.WriteValue(n);
+        }
+
+        /// <inheritdoc />
+        public override void WriteFloat(float f)
+        {
+            parser.Advance(Symbol.Float);
+            writer.WriteValue(f);
+        }
+
+        /// <inheritdoc />
+        public override void WriteDouble(double d)
+        {
+            parser.Advance(Symbol.Double);
+            writer.WriteValue(d);
+        }
+
+        /// <inheritdoc />
+        public override void WriteString(string str)
+        {
+            parser.Advance(Symbol.String);
+            if (parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                parser.Advance(Symbol.MapKeyMarker);
+                writer.WritePropertyName(str);
+            }
+            else
+            {
+                writer.WriteValue(str);
+            }
+        }
+
+        /// <inheritdoc />
+        public override void WriteBytes(byte[] bytes)
+        {
+            WriteBytes(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void WriteBytes(byte[] bytes, int start, int len)
+        {
+            parser.Advance(Symbol.Bytes);
+            writeByteArray(bytes, start, len);
+        }
+
+        private void writeByteArray(byte[] bytes, int start, int len)
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            writer.WriteValue(iso.GetString(bytes, start, len));
+        }
+
+        /// <inheritdoc />
+        public override void WriteFixed(byte[] bytes)
+        {
+            WriteFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void WriteFixed(byte[] bytes, int start, int len)
+        {
+            parser.Advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)parser.PopSymbol();
+            if (len != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + len + " bytes.");
+            }
+
+            writeByteArray(bytes, start, len);
+        }
+
+        /// <inheritdoc />
+        public override void WriteEnum(int e)
+        {
+            parser.Advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)parser.PopSymbol();
+            if (e < 0 || e >= top.Size)
+            {
+                throw new AvroTypeException("Enumeration out of range: max is " + top.Size + " but received " + e);
+            }
+
+            writer.WriteValue(top.GetLabel(e));
+        }
+
+        /// <inheritdoc />
+        public override void WriteArrayStart()
+        {
+            parser.Advance(Symbol.ArrayStart);
+            writer.WriteStartArray();
+            Push();
+            if (Depth() >= isEmpty.Length)
+            {
+                isEmpty.Length += isEmpty.Length;
+            }
+
+            isEmpty.Set(Depth(), true);
+        }
+
+        /// <inheritdoc />
+        public override void WriteArrayEnd()
+        {
+            if (!isEmpty.Get(Pos))
+            {
+                parser.Advance(Symbol.ItemEnd);
+            }
+
+            Pop();
+            parser.Advance(Symbol.ArrayEnd);
+            writer.WriteEndArray();
+        }
+
+        /// <inheritdoc />
+        public override void WriteMapStart()
+        {
+            Push();
+            if (Depth() >= isEmpty.Length)
+            {
+                isEmpty.Length += isEmpty.Length;
+            }
+
+            isEmpty.Set(Depth(), true);
+
+            parser.Advance(Symbol.MapStart);
+            writer.WriteStartObject();
+        }
+
+        /// <inheritdoc />
+        public override void WriteMapEnd()
+        {
+            if (!isEmpty.Get(Pos))
+            {
+                parser.Advance(Symbol.ItemEnd);
+            }
+
+            Pop();
+
+            parser.Advance(Symbol.MapEnd);
+            writer.WriteEndObject();
+        }
+
+        /// <summary>
+        /// Start an array item.
+        /// </summary>
+        public new void StartItem()

Review Comment:
   Is JsonEncoder.StartItem() ever called?



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


[GitHub] [avro] KalleOlaviNiemitalo commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1223533178

   > I've fixed that issue in this PR by overriding the `Fullname` property in `LogicalSchema`.
   
   That might fix [AVRO-3568](https://issues.apache.org/jira/browse/AVRO-3568) as well.


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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1225083274

   > How about a union with a member that is a record with a namespace? The [JSON Encoding](https://avro.apache.org/docs/1.11.1/specification/#json-encoding) specification says
   > 
   > > otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
   > 
   > which refers to the record _name_ rather than _fullname_.
   
   @KalleOlaviNiemitalo , I added a test for a union containing a record with a namespace as you requested.  The Java code uses the fullname for the union label [here](https://github.com/apache/avro/blob/master/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ValidatingGrammarGenerator.java#L104).   Since the C# code is a straight port of the Java code, it uses the fullname for the union label [here](https://github.com/apache/avro/blob/1133acda686fb1ff96fcbb80ca606c20e261f343/lang/csharp/src/apache/main/IO/Parsing/ValidatingGrammarGenerator.cs#L111)


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


[GitHub] [avro] KyleSchoonover commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KyleSchoonover commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r955370460


##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;

Review Comment:
   I am aware it is a private class.  The shorter method is fine.



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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r955439815


##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));

Review Comment:
   No, because `reader.Value` returns the current token, whereas `reader.ReadAsString()` reads the next token and converts it to a string.



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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r953303802


##########
lang/csharp/src/apache/main/IO/Parsing/JsonGrammarGenerator.cs:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// The class that generates a grammar suitable to parse Avro data in JSON
+    /// format.
+    /// </summary>
+    public class JsonGrammarGenerator : ValidatingGrammarGenerator
+    {
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// grammar for the given schema <tt>sc</tt>.

Review Comment:
   The parameter is named `schema`, not `sc`.



##########
lang/csharp/src/apache/main/IO/JsonEncoder.cs:
##########
@@ -0,0 +1,356 @@
+/*
+ * 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.
+ */
+
+using Avro.IO.Parsing;
+using System.Collections;
+using System.IO;
+using System.Text;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// An <see cref="Encoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonEncoder buffers output, and data may not appear on the output until
+    /// <see cref="Encoder.Flush()"/> is called.
+    ///
+    /// JsonEncoder is not thread-safe.
+    /// </summary>
+    public class JsonEncoder : ParsingEncoder, Parser.IActionHandler
+    {
+        private readonly Parser parser;
+        private JsonWriter writer;
+        private bool includeNamespace = true;
+
+        // Has anything been written into the collections?
+        private readonly BitArray isEmpty = new BitArray(64);
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream) : this(sc, getJsonWriter(stream, false))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream, bool pretty) : this(sc, getJsonWriter(stream, pretty))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, JsonWriter writer)
+        {
+            Configure(writer);
+            this.parser = new Parser((new JsonGrammarGenerator()).Generate(sc), this);
+        }
+
+        /// <inheritdoc />
+        public override void Flush()
+        {
+            parser.ProcessImplicitActions();
+            if (writer != null)
+            {
+                writer.Flush();
+            }
+        }
+
+        // by default, one object per line.
+        // with pretty option use default pretty printer with root line separator.
+        private static JsonWriter getJsonWriter(Stream stream, bool pretty)
+        {
+            JsonWriter writer = new JsonTextWriter(new StreamWriter(stream));
+            if (pretty)
+            {
+                writer.Formatting = Formatting.Indented;
+            }
+
+            return writer;
+        }
+
+        /// <summary>
+        /// Whether to include the namespace.
+        /// </summary>
+        public virtual bool IncludeNamespace

Review Comment:
   Can you please change this documentation to say what it actually does, if it is not related to namespaces used in the schema.



##########
lang/csharp/src/apache/main/IO/Parsing/ValidatingGrammarGenerator.cs:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using Avro.Generic;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// The class that generates validating grammar.
+    /// </summary>
+    public class ValidatingGrammarGenerator
+    {
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// given schema <tt>sc</tt>.
+        /// </summary>
+        public virtual Symbol Generate(Schema schema)
+        {
+            return Symbol.NewRoot(Generate(schema, new Dictionary<LitS, Symbol>()));
+        }
+
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// given schema <tt>sc</tt>. If there is already an entry for the given schema
+        /// in the given map <tt>seen</tt> then that entry is returned. Otherwise a new
+        /// symbol is generated and an entry is inserted into the map.
+        /// </summary>
+        /// <param name="sc">   The schema for which the start symbol is required </param>
+        /// <param name="seen"> A map of schema to symbol mapping done so far. </param>
+        /// <returns> The start symbol for the schema </returns>
+        protected virtual Symbol Generate(Schema sc, IDictionary<LitS, Symbol> seen)
+        {
+            switch (sc.Tag)
+            {
+                case Schema.Type.Null:
+                    return Symbol.Null;
+                case Schema.Type.Boolean:
+                    return Symbol.Boolean;
+                case Schema.Type.Int:
+                    return Symbol.Int;
+                case Schema.Type.Long:
+                    return Symbol.Long;
+                case Schema.Type.Float:
+                    return Symbol.Float;
+                case Schema.Type.Double:
+                    return Symbol.Double;
+                case Schema.Type.String:
+                    return Symbol.String;
+                case Schema.Type.Bytes:
+                    return Symbol.Bytes;
+                case Schema.Type.Fixed:
+                    return Symbol.NewSeq(new Symbol.IntCheckAction(((FixedSchema)sc).Size), Symbol.Fixed);
+                case Schema.Type.Enumeration:
+                    return Symbol.NewSeq(new Symbol.IntCheckAction(((EnumSchema)sc).Symbols.Count), Symbol.Enum);

Review Comment:
   I guess this is where EnumAdjustAction would be used, if anywhere.



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1049 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, makeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index)
+            {
+                return Production[index];
+            }
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Production.Length;
+            }
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator()
+            {
+                return Enumerable.Reverse(Production).GetEnumerator();
+            }
+
+            IEnumerator IEnumerable.GetEnumerator()
+            {
+                return this.GetEnumerator();
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Sequence result;
+                if (!map.TryGetValue(this, out result))
+                {
+                    result = new Sequence(new Symbol[FlattenedSize()]);
+                    map[this] = result;
+                    IList<Fixup> l = new List<Fixup>();
+                    map2[result] = l;
+
+                    Flatten(Production, 0, result.Production, 0, map, map2);
+                    foreach (Fixup f in l)
+                    {
+                        Array.Copy(result.Production, 0, f.Symbols, f.Pos, result.Production.Length);
+                    }
+
+                    map2.Remove(result);
+                }
+
+                return result;
+            }
+
+            /// <inheritdoc />
+            public override int FlattenedSize()
+            {
+                return FlattenedSize(Production, 0);
+            }
+        }
+
+        /// <summary>
+        /// Repeater symbol.
+        /// </summary>
+        public class Repeater : Symbol
+        {
+            /// <summary>
+            /// The end symbol.
+            /// </summary>
+            public readonly Symbol End;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Repeater"/> class.
+            /// </summary>
+            public Repeater(Symbol end, params Symbol[] sequenceToRepeat) : base(Kind.Repeater,
+                makeProduction(sequenceToRepeat))
+            {
+                this.End = end;
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] p)
+            {
+                Symbol[] result = new Symbol[p.Length + 1];
+                Array.Copy(p, 0, result, 1, p.Length);
+                return result;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Repeater result = new Repeater(End, new Symbol[FlattenedSize(Production, 1)]);
+                Flatten(Production, 1, result.Production, 1, map, map2);
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Returns true if the Parser contains any Error symbol, indicating that it may
+        /// fail for some inputs.
+        /// </summary>
+        private static bool hasErrors(Symbol symbol)
+        {
+            return hasErrors(symbol, new HashSet<Symbol>());
+        }
+
+        private static bool hasErrors(Symbol symbol, ISet<Symbol> visited)
+        {
+            // avoid infinite recursion
+            if (visited.Contains(symbol))
+            {
+                return false;
+            }
+
+            visited.Add(symbol);
+
+            switch (symbol.SymKind)
+            {
+                case Kind.Alternative:
+                    return hasErrors(symbol, ((Alternative)symbol).Symbols, visited);
+                case Kind.ExplicitAction:
+                    return false;
+                case Kind.ImplicitAction:
+                    if (symbol is ErrorAction)
+                    {
+                        return true;
+                    }
+
+                    if (symbol is UnionAdjustAction)
+                    {
+                        return hasErrors(((UnionAdjustAction)symbol).SymToParse, visited);
+                    }
+
+                    return false;
+                case Kind.Repeater:
+                    Repeater r = (Repeater)symbol;
+                    return hasErrors(r.End, visited) || hasErrors(symbol, r.Production, visited);
+                case Kind.Root:
+                case Kind.Sequence:
+                    return hasErrors(symbol, symbol.Production, visited);
+                case Kind.Terminal:
+                    return false;
+                default:
+                    throw new Exception("unknown symbol kind: " + symbol.SymKind);
+            }
+        }
+
+        private static bool hasErrors(Symbol root, Symbol[] symbols, ISet<Symbol> visited)
+        {
+            if (null != symbols)
+            {
+                foreach (Symbol s in symbols)
+                {
+                    if (s == root)
+                    {
+                        continue;
+                    }
+
+                    if (hasErrors(s, visited))
+                    {
+                        return true;
+                    }
+                }
+            }
+
+            return false;
+        }
+
+        /// <summary>
+        /// Alternative symbol.
+        /// </summary>
+        public class Alternative : Symbol
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public readonly Symbol[] Symbols;
+            /// <summary>
+            /// The labels.
+            /// </summary>
+            public readonly string[] Labels;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Alternative"/> class.
+            /// </summary>
+            public Alternative(Symbol[] symbols, string[] labels) : base(Kind.Alternative)
+            {
+                this.Symbols = symbols;
+                this.Labels = labels;
+            }
+
+            /// <summary>
+            /// Returns the symbol at the given index.
+            /// </summary>
+            public virtual Symbol GetSymbol(int index)
+            {
+                return Symbols[index];
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int index)
+            {
+                return Labels[index];
+            }
+
+            /// <summary>
+            /// Returns the size.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Symbols.Length;
+            }
+
+            /// <summary>
+            /// Returns the index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string label)
+            {
+                if (!ReferenceEquals(label, null))
+                {
+                    for (int i = 0; i < Labels.Length; i++)
+                    {
+                        if (label.Equals(Labels[i]))
+                        {
+                            return i;
+                        }
+                    }
+                }
+
+                return -1;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Symbol[] ss = new Symbol[Symbols.Length];
+                for (int i = 0; i < ss.Length; i++)
+                {
+                    ss[i] = Symbols[i].Flatten(map, map2);
+                }
+
+                return new Alternative(ss, Labels);
+            }
+        }
+
+        /// <summary>
+        /// The error action.
+        /// </summary>
+        public class ErrorAction : ImplicitAction
+        {
+            /// <summary>
+            /// The error message.
+            /// </summary>
+            public readonly string Msg;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ErrorAction"/> class.
+            /// </summary>
+            public ErrorAction(string msg)
+            {
+                this.Msg = msg;
+            }
+        }
+
+        /// <summary>
+        /// Int check action.
+        /// </summary>
+        public class IntCheckAction : Symbol
+        {
+            /// <summary>
+            /// The size.
+            /// </summary>
+            public readonly int Size;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.IntCheckAction"/> class.
+            /// </summary>
+            public IntCheckAction(int size) : base(Kind.ExplicitAction)
+            {
+                this.Size = size;
+            }
+        }
+
+        /// <summary>
+        /// The enum adjust action.
+        /// </summary>
+        public class EnumAdjustAction : IntCheckAction

Review Comment:
   Is an instance of EnumAdjustAction created somewhere? EnumLabelsAction is used but I don't see where EnumAdjustAction is.



##########
lang/csharp/src/apache/main/IO/Parsing/Parser.cs:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.
+ */
+
+using System;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Parser is the class that maintains the stack for parsing. This class is used
+    /// by encoders, which are not required to skip.
+    /// </summary>
+    public class Parser
+    {
+        /// <summary>
+        /// The parser knows how to handle the terminal and non-terminal symbols. But it
+        /// needs help from outside to handle implicit and explicit actions. The clients
+        /// implement this interface to provide this help.
+        /// </summary>
+        public interface IActionHandler
+        {
+            /// <summary>
+            /// Handle the action symbol <tt>top</tt> when the <tt>input</tt> is sought to be
+            /// taken off the stack.
+            /// </summary>
+            /// <param name="input"> The input symbol from the caller of advance </param>
+            /// <param name="top">   The symbol at the top the stack. </param>
+            /// <returns> <tt>null</tt> if advance() is to continue processing the stack. If
+            ///         not <tt>null</tt> the return value will be returned by advance(). </returns>

Review Comment:
   I guess these refer to Parser.Advance with upper-case A.



##########
lang/csharp/src/apache/main/IO/Parsing/SkipParser.cs:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.
+ */
+
+using System.Diagnostics;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// A parser that capable of skipping as well read and write. This class is used
+    /// by decoders who (unlink encoders) are required to implement methods to skip.

Review Comment:
   Unlike



##########
lang/csharp/src/apache/main/IO/Parsing/ValidatingGrammarGenerator.cs:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using Avro.Generic;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// The class that generates validating grammar.
+    /// </summary>
+    public class ValidatingGrammarGenerator
+    {
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// given schema <tt>sc</tt>.
+        /// </summary>
+        public virtual Symbol Generate(Schema schema)
+        {
+            return Symbol.NewRoot(Generate(schema, new Dictionary<LitS, Symbol>()));
+        }
+
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// given schema <tt>sc</tt>. If there is already an entry for the given schema
+        /// in the given map <tt>seen</tt> then that entry is returned. Otherwise a new
+        /// symbol is generated and an entry is inserted into the map.
+        /// </summary>
+        /// <param name="sc">   The schema for which the start symbol is required </param>
+        /// <param name="seen"> A map of schema to symbol mapping done so far. </param>
+        /// <returns> The start symbol for the schema </returns>
+        protected virtual Symbol Generate(Schema sc, IDictionary<LitS, Symbol> seen)
+        {
+            switch (sc.Tag)
+            {
+                case Schema.Type.Null:
+                    return Symbol.Null;
+                case Schema.Type.Boolean:
+                    return Symbol.Boolean;
+                case Schema.Type.Int:
+                    return Symbol.Int;
+                case Schema.Type.Long:
+                    return Symbol.Long;
+                case Schema.Type.Float:
+                    return Symbol.Float;
+                case Schema.Type.Double:
+                    return Symbol.Double;
+                case Schema.Type.String:
+                    return Symbol.String;
+                case Schema.Type.Bytes:
+                    return Symbol.Bytes;
+                case Schema.Type.Fixed:
+                    return Symbol.NewSeq(new Symbol.IntCheckAction(((FixedSchema)sc).Size), Symbol.Fixed);
+                case Schema.Type.Enumeration:
+                    return Symbol.NewSeq(new Symbol.IntCheckAction(((EnumSchema)sc).Symbols.Count), Symbol.Enum);
+                case Schema.Type.Array:
+                    return Symbol.NewSeq(
+                        Symbol.NewRepeat(Symbol.ArrayEnd, Generate(((ArraySchema)sc).ItemSchema, seen)),
+                        Symbol.ArrayStart);
+                case Schema.Type.Map:
+                    return Symbol.NewSeq(
+                        Symbol.NewRepeat(Symbol.MapEnd, Generate(((MapSchema)sc).ValueSchema, seen), Symbol.String),
+                        Symbol.MapStart);
+                case Schema.Type.Record:
+                    {
+                        LitS wsc = new LitS(sc);
+                        Symbol rresult;
+                        if (!seen.TryGetValue(wsc, out rresult))
+                        {
+                            Symbol[] production = new Symbol[((RecordSchema)sc).Fields.Count];
+
+                            // We construct a symbol without filling the array. Please see
+                            // <see cref="Symbol.production"/> for the reason.
+                            rresult = Symbol.NewSeq(production);
+                            seen[wsc] = rresult;
+
+                            int j = production.Length;
+                            foreach (Field f in ((RecordSchema)sc).Fields)
+                            {
+                                production[--j] = Generate(f.Schema, seen);
+                            }
+                        }
+
+                        return rresult;
+                    }
+                case Schema.Type.Union:
+                    IList<Schema> subs = ((UnionSchema)sc).Schemas;
+                    Symbol[] symbols = new Symbol[subs.Count];
+                    string[] labels = new string[subs.Count];
+
+                    int i = 0;
+                    foreach (Schema b in ((UnionSchema)sc).Schemas)
+                    {
+                        symbols[i] = Generate(b, seen);
+                        labels[i] = b.Fullname;
+                        i++;
+                    }
+
+                    return Symbol.NewSeq(Symbol.NewAlt(symbols, labels), Symbol.Union);
+                case Schema.Type.Logical:
+                    return Generate((sc as LogicalSchema).BaseSchema, seen);
+                default:
+                    throw new Exception("Unexpected schema type");
+            }
+        }
+
+        /// <summary>
+        /// A wrapper around Schema that does "==" equality.
+        /// </summary>
+        protected class LitS

Review Comment:
   For Dictionary\<TKey, TValue>, I normally use a reference-equality IEqualityComparer\<T> rather than a wrapper, but OK.



##########
lang/csharp/src/apache/main/IO/Parsing/Parser.cs:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.
+ */
+
+using System;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Parser is the class that maintains the stack for parsing. This class is used
+    /// by encoders, which are not required to skip.
+    /// </summary>
+    public class Parser
+    {
+        /// <summary>
+        /// The parser knows how to handle the terminal and non-terminal symbols. But it
+        /// needs help from outside to handle implicit and explicit actions. The clients
+        /// implement this interface to provide this help.
+        /// </summary>
+        public interface IActionHandler
+        {
+            /// <summary>
+            /// Handle the action symbol <tt>top</tt> when the <tt>input</tt> is sought to be
+            /// taken off the stack.
+            /// </summary>
+            /// <param name="input"> The input symbol from the caller of advance </param>
+            /// <param name="top">   The symbol at the top the stack. </param>
+            /// <returns> <tt>null</tt> if advance() is to continue processing the stack. If
+            ///         not <tt>null</tt> the return value will be returned by advance(). </returns>
+            Symbol DoAction(Symbol input, Symbol top);
+        }
+
+        private readonly IActionHandler symbolHandler;
+        /// <summary>
+        /// Stack of symbols.
+        /// </summary>
+        protected Symbol[] Stack;
+        /// <summary>
+        /// Position of the stack.
+        /// </summary>
+        protected int Pos;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Parser"/> class.
+        /// </summary>
+        public Parser(Symbol root, IActionHandler symbolHandler)
+        {
+            this.symbolHandler = symbolHandler;
+            this.Stack = new Symbol[5]; // Start small to make sure expansion code works
+            this.Stack[0] = root;
+            this.Pos = 1;
+        }
+
+        /// <summary>
+        /// If there is no sufficient room in the stack, use this expand it.
+        /// </summary>
+        private void expandStack()
+        {
+            Array.Resize(ref Stack, Stack.Length + Math.Max(Stack.Length, 1024));
+        }
+
+        /// <summary>
+        /// Recursively replaces the symbol at the top of the stack with its production,
+        /// until the top is a terminal. Then checks if the top symbol matches the
+        /// terminal symbol supplied <tt>terminal</tt>.

Review Comment:
   Should that be `input` rather than `terminal`?



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1049 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, makeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index)
+            {
+                return Production[index];
+            }
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Production.Length;
+            }
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator()
+            {
+                return Enumerable.Reverse(Production).GetEnumerator();
+            }
+
+            IEnumerator IEnumerable.GetEnumerator()
+            {
+                return this.GetEnumerator();
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Sequence result;
+                if (!map.TryGetValue(this, out result))
+                {
+                    result = new Sequence(new Symbol[FlattenedSize()]);
+                    map[this] = result;
+                    IList<Fixup> l = new List<Fixup>();
+                    map2[result] = l;
+
+                    Flatten(Production, 0, result.Production, 0, map, map2);
+                    foreach (Fixup f in l)
+                    {
+                        Array.Copy(result.Production, 0, f.Symbols, f.Pos, result.Production.Length);
+                    }
+
+                    map2.Remove(result);
+                }
+
+                return result;
+            }
+
+            /// <inheritdoc />
+            public override int FlattenedSize()
+            {
+                return FlattenedSize(Production, 0);
+            }
+        }
+
+        /// <summary>
+        /// Repeater symbol.
+        /// </summary>
+        public class Repeater : Symbol
+        {
+            /// <summary>
+            /// The end symbol.
+            /// </summary>
+            public readonly Symbol End;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Repeater"/> class.
+            /// </summary>
+            public Repeater(Symbol end, params Symbol[] sequenceToRepeat) : base(Kind.Repeater,
+                makeProduction(sequenceToRepeat))
+            {
+                this.End = end;
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] p)
+            {
+                Symbol[] result = new Symbol[p.Length + 1];
+                Array.Copy(p, 0, result, 1, p.Length);
+                return result;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Repeater result = new Repeater(End, new Symbol[FlattenedSize(Production, 1)]);
+                Flatten(Production, 1, result.Production, 1, map, map2);
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Returns true if the Parser contains any Error symbol, indicating that it may
+        /// fail for some inputs.
+        /// </summary>
+        private static bool hasErrors(Symbol symbol)
+        {
+            return hasErrors(symbol, new HashSet<Symbol>());
+        }
+
+        private static bool hasErrors(Symbol symbol, ISet<Symbol> visited)
+        {
+            // avoid infinite recursion
+            if (visited.Contains(symbol))
+            {
+                return false;
+            }
+
+            visited.Add(symbol);
+
+            switch (symbol.SymKind)
+            {
+                case Kind.Alternative:
+                    return hasErrors(symbol, ((Alternative)symbol).Symbols, visited);
+                case Kind.ExplicitAction:
+                    return false;
+                case Kind.ImplicitAction:
+                    if (symbol is ErrorAction)
+                    {
+                        return true;
+                    }
+
+                    if (symbol is UnionAdjustAction)
+                    {
+                        return hasErrors(((UnionAdjustAction)symbol).SymToParse, visited);
+                    }
+
+                    return false;
+                case Kind.Repeater:
+                    Repeater r = (Repeater)symbol;
+                    return hasErrors(r.End, visited) || hasErrors(symbol, r.Production, visited);
+                case Kind.Root:
+                case Kind.Sequence:
+                    return hasErrors(symbol, symbol.Production, visited);
+                case Kind.Terminal:
+                    return false;
+                default:
+                    throw new Exception("unknown symbol kind: " + symbol.SymKind);
+            }
+        }
+
+        private static bool hasErrors(Symbol root, Symbol[] symbols, ISet<Symbol> visited)
+        {
+            if (null != symbols)
+            {
+                foreach (Symbol s in symbols)
+                {
+                    if (s == root)
+                    {
+                        continue;
+                    }
+
+                    if (hasErrors(s, visited))
+                    {
+                        return true;
+                    }
+                }
+            }
+
+            return false;
+        }
+
+        /// <summary>
+        /// Alternative symbol.
+        /// </summary>
+        public class Alternative : Symbol
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public readonly Symbol[] Symbols;
+            /// <summary>
+            /// The labels.
+            /// </summary>
+            public readonly string[] Labels;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Alternative"/> class.
+            /// </summary>
+            public Alternative(Symbol[] symbols, string[] labels) : base(Kind.Alternative)
+            {
+                this.Symbols = symbols;
+                this.Labels = labels;
+            }
+
+            /// <summary>
+            /// Returns the symbol at the given index.
+            /// </summary>
+            public virtual Symbol GetSymbol(int index)
+            {
+                return Symbols[index];
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int index)
+            {
+                return Labels[index];
+            }
+
+            /// <summary>
+            /// Returns the size.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Symbols.Length;
+            }
+
+            /// <summary>
+            /// Returns the index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string label)
+            {
+                if (!ReferenceEquals(label, null))
+                {
+                    for (int i = 0; i < Labels.Length; i++)
+                    {
+                        if (label.Equals(Labels[i]))
+                        {
+                            return i;
+                        }
+                    }
+                }
+
+                return -1;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Symbol[] ss = new Symbol[Symbols.Length];
+                for (int i = 0; i < ss.Length; i++)
+                {
+                    ss[i] = Symbols[i].Flatten(map, map2);
+                }
+
+                return new Alternative(ss, Labels);
+            }
+        }
+
+        /// <summary>
+        /// The error action.
+        /// </summary>
+        public class ErrorAction : ImplicitAction
+        {
+            /// <summary>
+            /// The error message.
+            /// </summary>
+            public readonly string Msg;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ErrorAction"/> class.
+            /// </summary>
+            public ErrorAction(string msg)
+            {
+                this.Msg = msg;
+            }
+        }
+
+        /// <summary>
+        /// Int check action.
+        /// </summary>
+        public class IntCheckAction : Symbol
+        {
+            /// <summary>
+            /// The size.
+            /// </summary>
+            public readonly int Size;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.IntCheckAction"/> class.
+            /// </summary>
+            public IntCheckAction(int size) : base(Kind.ExplicitAction)
+            {
+                this.Size = size;
+            }
+        }
+
+        /// <summary>
+        /// The enum adjust action.
+        /// </summary>
+        public class EnumAdjustAction : IntCheckAction
+        {
+            /// <summary>
+            /// Whether no adjustments are needed.
+            /// </summary>
+            public readonly bool NoAdjustments;
+            /// <summary>
+            /// The adjustments.
+            /// </summary>
+            public readonly object[] Adjustments;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.EnumAdjustAction"/> class.
+            /// </summary>
+            public EnumAdjustAction(int rsymCount, object[] adjustments) : base(rsymCount)

Review Comment:
   `int?[]` might be better here, if EnumAdjustAction is actually used.



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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1225084782

   > I see JsonCodecTests.cs has tests with true and false as JsonEncoder.IncludeNamespace; but I don't see any namespaces in the schemas that the tests use.
   
   The `includeNamespace` is just used to determine whether to emit the union label (its name is somewhat misleading).  The Java code is [here](https://github.com/apache/avro/blob/master/lang/java/avro/src/main/java/org/apache/avro/io/JsonEncoder.java#L297.).  Since the C# is a straight port, the corresponding code is [here](https://github.com/apache/avro/blob/1133acda686fb1ff96fcbb80ca606c20e261f343/lang/csharp/src/apache/main/IO/JsonEncoder.cs#L320).


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


[GitHub] [avro] martin-g commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
martin-g commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1231580851

   Thank you for the contribution, @rayokota !
   Thanks to all reviews too!


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


[GitHub] [avro] KyleSchoonover commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KyleSchoonover commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r956225566


##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,983 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public Kind SymKind { get; private set; }
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public Symbol[] Production { get; private set; }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            Production = production;
+            SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols) => new Root(symbols);
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production) => new Sequence(production);
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat) =>
+            new Repeater(endSymbol, symsToRepeat);
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels) => new Alternative(symbols, labels);
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e) => new ErrorAction(e);
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r) => new ResolvingAction(w, r);
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public int Pos { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2) => this;
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize() => 1;
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    if (!map2.TryGetValue((Sequence)s, out IList<Fixup> l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            CopyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void CopyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public string PrintName { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString() => PrintName;
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public bool IsTrailing { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, MakeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] MakeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol this[int index] => Production[index];
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index) => Production[index];
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size() => Production.Length;
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator() => Enumerable.Reverse(Production).GetEnumerator();
+
+            IEnumerator IEnumerable.GetEnumerator() => GetEnumerator();
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                if (!map.TryGetValue(this, out Sequence result))
+                {
+                    result = new Sequence(new Symbol[FlattenedSize()]);
+                    map[this] = result;
+                    IList<Fixup> l = new List<Fixup>();
+                    map2[result] = l;
+
+                    Flatten(Production, 0, result.Production, 0, map, map2);
+                    foreach (Fixup f in l)
+                    {
+                        Array.Copy(result.Production, 0, f.Symbols, f.Pos, result.Production.Length);
+                    }
+
+                    map2.Remove(result);
+                }
+
+                return result;
+            }
+
+            /// <inheritdoc />
+            public override int FlattenedSize() => FlattenedSize(Production, 0);
+        }
+
+        /// <summary>
+        /// Repeater symbol.
+        /// </summary>
+        public class Repeater : Symbol
+        {
+            /// <summary>
+            /// The end symbol.
+            /// </summary>
+            public Symbol End { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Repeater"/> class.
+            /// </summary>
+            public Repeater(Symbol end, params Symbol[] sequenceToRepeat) : base(Kind.Repeater,
+                MakeProduction(sequenceToRepeat))
+            {
+                End = end;
+                Production[0] = this;
+            }
+
+            private static Symbol[] MakeProduction(Symbol[] p)
+            {
+                Symbol[] result = new Symbol[p.Length + 1];
+                Array.Copy(p, 0, result, 1, p.Length);
+                return result;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Repeater result = new Repeater(End, new Symbol[FlattenedSize(Production, 1)]);
+                Flatten(Production, 1, result.Production, 1, map, map2);
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Returns true if the Parser contains any Error symbol, indicating that it may
+        /// fail for some inputs.
+        /// </summary>
+        private static bool HasErrors(Symbol symbol)
+        {
+            return HasErrors(symbol, new HashSet<Symbol>());
+        }
+
+        private static bool HasErrors(Symbol symbol, ISet<Symbol> visited)
+        {
+            // avoid infinite recursion
+            if (visited.Contains(symbol))
+            {
+                return false;
+            }
+
+            visited.Add(symbol);
+
+            switch (symbol.SymKind)
+            {
+                case Kind.Alternative:
+                    return HasErrors(symbol, ((Alternative)symbol).Symbols, visited);
+                case Kind.ExplicitAction:
+                    return false;
+                case Kind.ImplicitAction:
+                    if (symbol is ErrorAction)
+                    {
+                        return true;
+                    }
+
+                    if (symbol is UnionAdjustAction)
+                    {
+                        return HasErrors(((UnionAdjustAction)symbol).SymToParse, visited);
+                    }
+
+                    return false;
+                case Kind.Repeater:
+                    Repeater r = (Repeater)symbol;
+                    return HasErrors(r.End, visited) || HasErrors(symbol, r.Production, visited);
+                case Kind.Root:
+                case Kind.Sequence:
+                    return HasErrors(symbol, symbol.Production, visited);
+                case Kind.Terminal:
+                    return false;
+                default:
+                    throw new Exception("unknown symbol kind: " + symbol.SymKind);
+            }
+        }
+
+        private static bool HasErrors(Symbol root, Symbol[] symbols, ISet<Symbol> visited)
+        {
+            if (null != symbols)
+            {
+                foreach (Symbol s in symbols)
+                {
+                    if (s == root)
+                    {
+                        continue;
+                    }
+
+                    if (HasErrors(s, visited))
+                    {
+                        return true;
+                    }
+                }
+            }
+
+            return false;
+        }
+
+        /// <summary>
+        /// Alternative symbol.
+        /// </summary>
+        public class Alternative : Symbol
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols { get; private set; }
+
+            /// <summary>
+            /// The labels.
+            /// </summary>
+            public string[] Labels { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Alternative"/> class.
+            /// </summary>
+            public Alternative(Symbol[] symbols, string[] labels) : base(Kind.Alternative)
+            {
+                Symbols = symbols;
+                Labels = labels;
+            }
+
+            /// <summary>
+            /// Returns the symbol at the given index.
+            /// </summary>
+            public virtual Symbol GetSymbol(int index)
+            {
+                return Symbols[index];
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int index)
+            {
+                return Labels[index];
+            }
+
+            /// <summary>
+            /// Returns the size.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Symbols.Length;
+            }
+
+            /// <summary>
+            /// Returns the index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string label)
+            {
+                if (!ReferenceEquals(label, null))

Review Comment:
   `label != null` unless we don't want to allow empty string use `!string.IsNullOrEmpty(label)`



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,983 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public Kind SymKind { get; private set; }
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public Symbol[] Production { get; private set; }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            Production = production;
+            SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols) => new Root(symbols);
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production) => new Sequence(production);
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat) =>
+            new Repeater(endSymbol, symsToRepeat);
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels) => new Alternative(symbols, labels);
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e) => new ErrorAction(e);
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r) => new ResolvingAction(w, r);
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public int Pos { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2) => this;
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize() => 1;
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    if (!map2.TryGetValue((Sequence)s, out IList<Fixup> l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            CopyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void CopyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public string PrintName { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString() => PrintName;
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public bool IsTrailing { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, MakeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] MakeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol this[int index] => Production[index];
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index) => Production[index];
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size() => Production.Length;
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator() => Enumerable.Reverse(Production).GetEnumerator();
+
+            IEnumerator IEnumerable.GetEnumerator() => GetEnumerator();
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                if (!map.TryGetValue(this, out Sequence result))
+                {
+                    result = new Sequence(new Symbol[FlattenedSize()]);
+                    map[this] = result;
+                    IList<Fixup> l = new List<Fixup>();
+                    map2[result] = l;
+
+                    Flatten(Production, 0, result.Production, 0, map, map2);
+                    foreach (Fixup f in l)
+                    {
+                        Array.Copy(result.Production, 0, f.Symbols, f.Pos, result.Production.Length);
+                    }
+
+                    map2.Remove(result);
+                }
+
+                return result;
+            }
+
+            /// <inheritdoc />
+            public override int FlattenedSize() => FlattenedSize(Production, 0);
+        }
+
+        /// <summary>
+        /// Repeater symbol.
+        /// </summary>
+        public class Repeater : Symbol
+        {
+            /// <summary>
+            /// The end symbol.
+            /// </summary>
+            public Symbol End { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Repeater"/> class.
+            /// </summary>
+            public Repeater(Symbol end, params Symbol[] sequenceToRepeat) : base(Kind.Repeater,
+                MakeProduction(sequenceToRepeat))
+            {
+                End = end;
+                Production[0] = this;
+            }
+
+            private static Symbol[] MakeProduction(Symbol[] p)
+            {
+                Symbol[] result = new Symbol[p.Length + 1];
+                Array.Copy(p, 0, result, 1, p.Length);
+                return result;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Repeater result = new Repeater(End, new Symbol[FlattenedSize(Production, 1)]);
+                Flatten(Production, 1, result.Production, 1, map, map2);
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Returns true if the Parser contains any Error symbol, indicating that it may
+        /// fail for some inputs.
+        /// </summary>
+        private static bool HasErrors(Symbol symbol)
+        {
+            return HasErrors(symbol, new HashSet<Symbol>());
+        }
+
+        private static bool HasErrors(Symbol symbol, ISet<Symbol> visited)
+        {
+            // avoid infinite recursion
+            if (visited.Contains(symbol))
+            {
+                return false;
+            }
+
+            visited.Add(symbol);
+
+            switch (symbol.SymKind)
+            {
+                case Kind.Alternative:
+                    return HasErrors(symbol, ((Alternative)symbol).Symbols, visited);
+                case Kind.ExplicitAction:
+                    return false;
+                case Kind.ImplicitAction:
+                    if (symbol is ErrorAction)
+                    {
+                        return true;
+                    }
+
+                    if (symbol is UnionAdjustAction)
+                    {
+                        return HasErrors(((UnionAdjustAction)symbol).SymToParse, visited);
+                    }
+
+                    return false;
+                case Kind.Repeater:
+                    Repeater r = (Repeater)symbol;
+                    return HasErrors(r.End, visited) || HasErrors(symbol, r.Production, visited);
+                case Kind.Root:
+                case Kind.Sequence:
+                    return HasErrors(symbol, symbol.Production, visited);
+                case Kind.Terminal:
+                    return false;
+                default:
+                    throw new Exception("unknown symbol kind: " + symbol.SymKind);
+            }
+        }
+
+        private static bool HasErrors(Symbol root, Symbol[] symbols, ISet<Symbol> visited)
+        {
+            if (null != symbols)
+            {
+                foreach (Symbol s in symbols)
+                {
+                    if (s == root)
+                    {
+                        continue;
+                    }
+
+                    if (HasErrors(s, visited))
+                    {
+                        return true;
+                    }
+                }
+            }
+
+            return false;
+        }
+
+        /// <summary>
+        /// Alternative symbol.
+        /// </summary>
+        public class Alternative : Symbol
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols { get; private set; }
+
+            /// <summary>
+            /// The labels.
+            /// </summary>
+            public string[] Labels { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Alternative"/> class.
+            /// </summary>
+            public Alternative(Symbol[] symbols, string[] labels) : base(Kind.Alternative)
+            {
+                Symbols = symbols;
+                Labels = labels;
+            }
+
+            /// <summary>
+            /// Returns the symbol at the given index.
+            /// </summary>
+            public virtual Symbol GetSymbol(int index)
+            {
+                return Symbols[index];
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int index)
+            {
+                return Labels[index];
+            }
+
+            /// <summary>
+            /// Returns the size.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Symbols.Length;
+            }
+
+            /// <summary>
+            /// Returns the index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string label)
+            {
+                if (!ReferenceEquals(label, null))
+                {
+                    for (int i = 0; i < Labels.Length; i++)
+                    {
+                        if (label.Equals(Labels[i]))
+                        {
+                            return i;
+                        }
+                    }
+                }
+
+                return -1;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Symbol[] ss = new Symbol[Symbols.Length];
+                for (int i = 0; i < ss.Length; i++)
+                {
+                    ss[i] = Symbols[i].Flatten(map, map2);
+                }
+
+                return new Alternative(ss, Labels);
+            }
+        }
+
+        /// <summary>
+        /// The error action.
+        /// </summary>
+        public class ErrorAction : ImplicitAction
+        {
+            /// <summary>
+            /// The error message.
+            /// </summary>
+            public string Msg { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ErrorAction"/> class.
+            /// </summary>
+            public ErrorAction(string msg)
+            {
+                Msg = msg;
+            }
+        }
+
+        /// <summary>
+        /// Int check action.
+        /// </summary>
+        public class IntCheckAction : Symbol
+        {
+            /// <summary>
+            /// The size.
+            /// </summary>
+            public int Size { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.IntCheckAction"/> class.
+            /// </summary>
+            public IntCheckAction(int size) : base(Kind.ExplicitAction)
+            {
+                Size = size;
+            }
+        }
+
+        /// <summary>
+        /// The writer union action.
+        /// </summary>
+        public class WriterUnionAction : ImplicitAction
+        {
+        }
+
+        /// <summary>
+        /// The resolving action.
+        /// </summary>
+        public class ResolvingAction : ImplicitAction
+        {
+            /// <summary>
+            /// The writer.
+            /// </summary>
+            public Symbol Writer { get; private set; }
+
+            /// <summary>
+            /// The reader.
+            /// </summary>
+            public Symbol Reader { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ResolvingAction"/> class.
+            /// </summary>
+            public ResolvingAction(Symbol writer, Symbol reader)
+            {
+                Writer = writer;
+                Reader = reader;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new ResolvingAction(Writer.Flatten(map, map2), Reader.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The skip action.
+        /// </summary>
+        public class SkipAction : ImplicitAction
+        {
+            /// <summary>
+            /// The symbol to skip.
+            /// </summary>
+            public Symbol SymToSkip { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.SkipAction"/> class.
+            /// </summary>
+            public SkipAction(Symbol symToSkip) : base(true)
+            {
+                SymToSkip = symToSkip;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new SkipAction(SymToSkip.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The field adjust action.
+        /// </summary>
+        public class FieldAdjustAction : ImplicitAction
+        {
+            /// <summary>
+            /// The index.
+            /// </summary>
+            public int RIndex { get; private set; }
+
+            /// <summary>
+            /// The field name.
+            /// </summary>
+            public string FName { get; private set; }
+
+            /// <summary>
+            /// The field aliases.
+            /// </summary>
+            public IList<string> Aliases { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.FieldAdjustAction"/> class.
+            /// </summary>
+            public FieldAdjustAction(int rindex, string fname, IList<string> aliases)
+            {
+                RIndex = rindex;
+                FName = fname;
+                Aliases = aliases;
+            }
+        }
+
+        /// <summary>
+        /// THe field order action.
+        /// </summary>
+        public sealed class FieldOrderAction : ImplicitAction
+        {
+            /// <summary>
+            /// Whether no reorder is needed.
+            /// </summary>
+            public bool NoReorder { get; private set; }
+
+            /// <summary>
+            /// The fields.
+            /// </summary>
+            public Field[] Fields { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.FieldOrderAction"/> class.
+            /// </summary>
+            public FieldOrderAction(Field[] fields)
+            {
+                Fields = fields;
+                bool noReorder = true;
+                for (int i = 0; noReorder && i < fields.Length; i++)
+                {
+                    noReorder &= (i == fields[i].Pos);
+                }
+
+                NoReorder = noReorder;
+            }
+        }
+
+        /// <summary>
+        /// The default start action.
+        /// </summary>
+        public class DefaultStartAction : ImplicitAction
+        {
+            /// <summary>
+            /// The contents.
+            /// </summary>
+            public byte[] Contents { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.DefaultStartAction"/> class.
+            /// </summary>
+            public DefaultStartAction(byte[] contents)
+            {
+                Contents = contents;
+            }
+        }
+
+        /// <summary>
+        /// The union adjust action.
+        /// </summary>
+        public class UnionAdjustAction : ImplicitAction
+        {
+            /// <summary>
+            /// The index.
+            /// </summary>
+            public int RIndex { get; private set; }
+
+            /// <summary>
+            /// The symbol to parser.
+            /// </summary>
+            public Symbol SymToParse { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.UnionAdjustAction"/> class.
+            /// </summary>
+            public UnionAdjustAction(int rindex, Symbol symToParse)
+            {
+                RIndex = rindex;
+                SymToParse = symToParse;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new UnionAdjustAction(RIndex, SymToParse.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The enum labels action.
+        /// </summary>
+        public class EnumLabelsAction : IntCheckAction
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public IList<string> Symbols { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.EnumLabelsAction"/> class.
+            /// </summary>
+            public EnumLabelsAction(IList<string> symbols) : base(symbols.Count)
+            {
+                Symbols = symbols;
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int n)
+            {
+                return Symbols[n];
+            }
+
+            /// <summary>
+            /// Returns index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string l)
+            {
+                if (!ReferenceEquals(l, null))
+                {
+                    for (int i = 0; i < Symbols.Count; i++)
+                    {
+                        if (l.Equals(Symbols[i]))
+                        {
+                            return i;
+                        }
+                    }
+                }
+
+                return -1;
+            }
+        }
+
+        /// <summary>
+        /// The terminal symbols for the grammar.
+        /// </summary>
+        public static readonly Symbol Null = new Terminal("null");

Review Comment:
   `public static Symbol Null => new Terminal("null");`
   We should change these to static getters.  This is more of a common implementation in C# that is expected.



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,983 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public Kind SymKind { get; private set; }
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public Symbol[] Production { get; private set; }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            Production = production;
+            SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols) => new Root(symbols);
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production) => new Sequence(production);
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat) =>
+            new Repeater(endSymbol, symsToRepeat);
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels) => new Alternative(symbols, labels);
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e) => new ErrorAction(e);
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r) => new ResolvingAction(w, r);
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public int Pos { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2) => this;
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize() => 1;
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    if (!map2.TryGetValue((Sequence)s, out IList<Fixup> l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            CopyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void CopyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public string PrintName { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString() => PrintName;
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public bool IsTrailing { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, MakeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] MakeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol this[int index] => Production[index];
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index) => Production[index];
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size() => Production.Length;
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator() => Enumerable.Reverse(Production).GetEnumerator();
+
+            IEnumerator IEnumerable.GetEnumerator() => GetEnumerator();
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                if (!map.TryGetValue(this, out Sequence result))
+                {
+                    result = new Sequence(new Symbol[FlattenedSize()]);
+                    map[this] = result;
+                    IList<Fixup> l = new List<Fixup>();
+                    map2[result] = l;
+
+                    Flatten(Production, 0, result.Production, 0, map, map2);
+                    foreach (Fixup f in l)
+                    {
+                        Array.Copy(result.Production, 0, f.Symbols, f.Pos, result.Production.Length);
+                    }
+
+                    map2.Remove(result);
+                }
+
+                return result;
+            }
+
+            /// <inheritdoc />
+            public override int FlattenedSize() => FlattenedSize(Production, 0);
+        }
+
+        /// <summary>
+        /// Repeater symbol.
+        /// </summary>
+        public class Repeater : Symbol
+        {
+            /// <summary>
+            /// The end symbol.
+            /// </summary>
+            public Symbol End { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Repeater"/> class.
+            /// </summary>
+            public Repeater(Symbol end, params Symbol[] sequenceToRepeat) : base(Kind.Repeater,
+                MakeProduction(sequenceToRepeat))
+            {
+                End = end;
+                Production[0] = this;
+            }
+
+            private static Symbol[] MakeProduction(Symbol[] p)
+            {
+                Symbol[] result = new Symbol[p.Length + 1];
+                Array.Copy(p, 0, result, 1, p.Length);
+                return result;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Repeater result = new Repeater(End, new Symbol[FlattenedSize(Production, 1)]);
+                Flatten(Production, 1, result.Production, 1, map, map2);
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Returns true if the Parser contains any Error symbol, indicating that it may
+        /// fail for some inputs.
+        /// </summary>
+        private static bool HasErrors(Symbol symbol)
+        {
+            return HasErrors(symbol, new HashSet<Symbol>());
+        }
+
+        private static bool HasErrors(Symbol symbol, ISet<Symbol> visited)
+        {
+            // avoid infinite recursion
+            if (visited.Contains(symbol))
+            {
+                return false;
+            }
+
+            visited.Add(symbol);
+
+            switch (symbol.SymKind)
+            {
+                case Kind.Alternative:
+                    return HasErrors(symbol, ((Alternative)symbol).Symbols, visited);
+                case Kind.ExplicitAction:
+                    return false;
+                case Kind.ImplicitAction:
+                    if (symbol is ErrorAction)
+                    {
+                        return true;
+                    }
+
+                    if (symbol is UnionAdjustAction)
+                    {
+                        return HasErrors(((UnionAdjustAction)symbol).SymToParse, visited);
+                    }
+
+                    return false;
+                case Kind.Repeater:
+                    Repeater r = (Repeater)symbol;
+                    return HasErrors(r.End, visited) || HasErrors(symbol, r.Production, visited);
+                case Kind.Root:
+                case Kind.Sequence:
+                    return HasErrors(symbol, symbol.Production, visited);
+                case Kind.Terminal:
+                    return false;
+                default:
+                    throw new Exception("unknown symbol kind: " + symbol.SymKind);
+            }
+        }
+
+        private static bool HasErrors(Symbol root, Symbol[] symbols, ISet<Symbol> visited)
+        {
+            if (null != symbols)
+            {
+                foreach (Symbol s in symbols)
+                {
+                    if (s == root)
+                    {
+                        continue;
+                    }
+
+                    if (HasErrors(s, visited))
+                    {
+                        return true;
+                    }
+                }
+            }
+
+            return false;
+        }
+
+        /// <summary>
+        /// Alternative symbol.
+        /// </summary>
+        public class Alternative : Symbol
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols { get; private set; }
+
+            /// <summary>
+            /// The labels.
+            /// </summary>
+            public string[] Labels { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Alternative"/> class.
+            /// </summary>
+            public Alternative(Symbol[] symbols, string[] labels) : base(Kind.Alternative)
+            {
+                Symbols = symbols;
+                Labels = labels;
+            }
+
+            /// <summary>
+            /// Returns the symbol at the given index.
+            /// </summary>
+            public virtual Symbol GetSymbol(int index)
+            {
+                return Symbols[index];
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int index)
+            {
+                return Labels[index];
+            }
+
+            /// <summary>
+            /// Returns the size.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Symbols.Length;
+            }
+
+            /// <summary>
+            /// Returns the index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string label)
+            {
+                if (!ReferenceEquals(label, null))
+                {
+                    for (int i = 0; i < Labels.Length; i++)
+                    {
+                        if (label.Equals(Labels[i]))
+                        {
+                            return i;
+                        }
+                    }
+                }
+
+                return -1;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Symbol[] ss = new Symbol[Symbols.Length];
+                for (int i = 0; i < ss.Length; i++)
+                {
+                    ss[i] = Symbols[i].Flatten(map, map2);
+                }
+
+                return new Alternative(ss, Labels);
+            }
+        }
+
+        /// <summary>
+        /// The error action.
+        /// </summary>
+        public class ErrorAction : ImplicitAction
+        {
+            /// <summary>
+            /// The error message.
+            /// </summary>
+            public string Msg { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ErrorAction"/> class.
+            /// </summary>
+            public ErrorAction(string msg)
+            {
+                Msg = msg;
+            }
+        }
+
+        /// <summary>
+        /// Int check action.
+        /// </summary>
+        public class IntCheckAction : Symbol
+        {
+            /// <summary>
+            /// The size.
+            /// </summary>
+            public int Size { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.IntCheckAction"/> class.
+            /// </summary>
+            public IntCheckAction(int size) : base(Kind.ExplicitAction)
+            {
+                Size = size;
+            }
+        }
+
+        /// <summary>
+        /// The writer union action.
+        /// </summary>
+        public class WriterUnionAction : ImplicitAction
+        {
+        }
+
+        /// <summary>
+        /// The resolving action.
+        /// </summary>
+        public class ResolvingAction : ImplicitAction
+        {
+            /// <summary>
+            /// The writer.
+            /// </summary>
+            public Symbol Writer { get; private set; }
+
+            /// <summary>
+            /// The reader.
+            /// </summary>
+            public Symbol Reader { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ResolvingAction"/> class.
+            /// </summary>
+            public ResolvingAction(Symbol writer, Symbol reader)
+            {
+                Writer = writer;
+                Reader = reader;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new ResolvingAction(Writer.Flatten(map, map2), Reader.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The skip action.
+        /// </summary>
+        public class SkipAction : ImplicitAction
+        {
+            /// <summary>
+            /// The symbol to skip.
+            /// </summary>
+            public Symbol SymToSkip { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.SkipAction"/> class.
+            /// </summary>
+            public SkipAction(Symbol symToSkip) : base(true)
+            {
+                SymToSkip = symToSkip;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new SkipAction(SymToSkip.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The field adjust action.
+        /// </summary>
+        public class FieldAdjustAction : ImplicitAction
+        {
+            /// <summary>
+            /// The index.
+            /// </summary>
+            public int RIndex { get; private set; }
+
+            /// <summary>
+            /// The field name.
+            /// </summary>
+            public string FName { get; private set; }
+
+            /// <summary>
+            /// The field aliases.
+            /// </summary>
+            public IList<string> Aliases { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.FieldAdjustAction"/> class.
+            /// </summary>
+            public FieldAdjustAction(int rindex, string fname, IList<string> aliases)
+            {
+                RIndex = rindex;
+                FName = fname;
+                Aliases = aliases;
+            }
+        }
+
+        /// <summary>
+        /// THe field order action.
+        /// </summary>
+        public sealed class FieldOrderAction : ImplicitAction
+        {
+            /// <summary>
+            /// Whether no reorder is needed.
+            /// </summary>
+            public bool NoReorder { get; private set; }
+
+            /// <summary>
+            /// The fields.
+            /// </summary>
+            public Field[] Fields { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.FieldOrderAction"/> class.
+            /// </summary>
+            public FieldOrderAction(Field[] fields)
+            {
+                Fields = fields;
+                bool noReorder = true;
+                for (int i = 0; noReorder && i < fields.Length; i++)
+                {
+                    noReorder &= (i == fields[i].Pos);
+                }
+
+                NoReorder = noReorder;
+            }
+        }
+
+        /// <summary>
+        /// The default start action.
+        /// </summary>
+        public class DefaultStartAction : ImplicitAction
+        {
+            /// <summary>
+            /// The contents.
+            /// </summary>
+            public byte[] Contents { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.DefaultStartAction"/> class.
+            /// </summary>
+            public DefaultStartAction(byte[] contents)
+            {
+                Contents = contents;
+            }
+        }
+
+        /// <summary>
+        /// The union adjust action.
+        /// </summary>
+        public class UnionAdjustAction : ImplicitAction
+        {
+            /// <summary>
+            /// The index.
+            /// </summary>
+            public int RIndex { get; private set; }
+
+            /// <summary>
+            /// The symbol to parser.
+            /// </summary>
+            public Symbol SymToParse { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.UnionAdjustAction"/> class.
+            /// </summary>
+            public UnionAdjustAction(int rindex, Symbol symToParse)
+            {
+                RIndex = rindex;
+                SymToParse = symToParse;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new UnionAdjustAction(RIndex, SymToParse.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The enum labels action.
+        /// </summary>
+        public class EnumLabelsAction : IntCheckAction
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public IList<string> Symbols { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.EnumLabelsAction"/> class.
+            /// </summary>
+            public EnumLabelsAction(IList<string> symbols) : base(symbols.Count)
+            {
+                Symbols = symbols;
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int n)
+            {
+                return Symbols[n];
+            }
+
+            /// <summary>
+            /// Returns index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string l)
+            {
+                if (!ReferenceEquals(l, null))

Review Comment:
   l != null unless we don't want to allow empty string use !string.IsNullOrEmpty(l)
   Also, we should call the string label.  Single letter parameter names should be avoided.



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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r956273784


##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,983 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public Kind SymKind { get; private set; }
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public Symbol[] Production { get; private set; }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            Production = production;
+            SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols) => new Root(symbols);
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production) => new Sequence(production);
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat) =>
+            new Repeater(endSymbol, symsToRepeat);
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels) => new Alternative(symbols, labels);
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e) => new ErrorAction(e);
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r) => new ResolvingAction(w, r);
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public int Pos { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2) => this;
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize() => 1;
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    if (!map2.TryGetValue((Sequence)s, out IList<Fixup> l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            CopyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void CopyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public string PrintName { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString() => PrintName;
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public bool IsTrailing { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, MakeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] MakeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol this[int index] => Production[index];
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index) => Production[index];
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size() => Production.Length;
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator() => Enumerable.Reverse(Production).GetEnumerator();
+
+            IEnumerator IEnumerable.GetEnumerator() => GetEnumerator();
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                if (!map.TryGetValue(this, out Sequence result))
+                {
+                    result = new Sequence(new Symbol[FlattenedSize()]);
+                    map[this] = result;
+                    IList<Fixup> l = new List<Fixup>();
+                    map2[result] = l;
+
+                    Flatten(Production, 0, result.Production, 0, map, map2);
+                    foreach (Fixup f in l)
+                    {
+                        Array.Copy(result.Production, 0, f.Symbols, f.Pos, result.Production.Length);
+                    }
+
+                    map2.Remove(result);
+                }
+
+                return result;
+            }
+
+            /// <inheritdoc />
+            public override int FlattenedSize() => FlattenedSize(Production, 0);
+        }
+
+        /// <summary>
+        /// Repeater symbol.
+        /// </summary>
+        public class Repeater : Symbol
+        {
+            /// <summary>
+            /// The end symbol.
+            /// </summary>
+            public Symbol End { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Repeater"/> class.
+            /// </summary>
+            public Repeater(Symbol end, params Symbol[] sequenceToRepeat) : base(Kind.Repeater,
+                MakeProduction(sequenceToRepeat))
+            {
+                End = end;
+                Production[0] = this;
+            }
+
+            private static Symbol[] MakeProduction(Symbol[] p)
+            {
+                Symbol[] result = new Symbol[p.Length + 1];
+                Array.Copy(p, 0, result, 1, p.Length);
+                return result;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Repeater result = new Repeater(End, new Symbol[FlattenedSize(Production, 1)]);
+                Flatten(Production, 1, result.Production, 1, map, map2);
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Returns true if the Parser contains any Error symbol, indicating that it may
+        /// fail for some inputs.
+        /// </summary>
+        private static bool HasErrors(Symbol symbol)
+        {
+            return HasErrors(symbol, new HashSet<Symbol>());
+        }
+
+        private static bool HasErrors(Symbol symbol, ISet<Symbol> visited)
+        {
+            // avoid infinite recursion
+            if (visited.Contains(symbol))
+            {
+                return false;
+            }
+
+            visited.Add(symbol);
+
+            switch (symbol.SymKind)
+            {
+                case Kind.Alternative:
+                    return HasErrors(symbol, ((Alternative)symbol).Symbols, visited);
+                case Kind.ExplicitAction:
+                    return false;
+                case Kind.ImplicitAction:
+                    if (symbol is ErrorAction)
+                    {
+                        return true;
+                    }
+
+                    if (symbol is UnionAdjustAction)
+                    {
+                        return HasErrors(((UnionAdjustAction)symbol).SymToParse, visited);
+                    }
+
+                    return false;
+                case Kind.Repeater:
+                    Repeater r = (Repeater)symbol;
+                    return HasErrors(r.End, visited) || HasErrors(symbol, r.Production, visited);
+                case Kind.Root:
+                case Kind.Sequence:
+                    return HasErrors(symbol, symbol.Production, visited);
+                case Kind.Terminal:
+                    return false;
+                default:
+                    throw new Exception("unknown symbol kind: " + symbol.SymKind);
+            }
+        }
+
+        private static bool HasErrors(Symbol root, Symbol[] symbols, ISet<Symbol> visited)
+        {
+            if (null != symbols)
+            {
+                foreach (Symbol s in symbols)
+                {
+                    if (s == root)
+                    {
+                        continue;
+                    }
+
+                    if (HasErrors(s, visited))
+                    {
+                        return true;
+                    }
+                }
+            }
+
+            return false;
+        }
+
+        /// <summary>
+        /// Alternative symbol.
+        /// </summary>
+        public class Alternative : Symbol
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols { get; private set; }
+
+            /// <summary>
+            /// The labels.
+            /// </summary>
+            public string[] Labels { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Alternative"/> class.
+            /// </summary>
+            public Alternative(Symbol[] symbols, string[] labels) : base(Kind.Alternative)
+            {
+                Symbols = symbols;
+                Labels = labels;
+            }
+
+            /// <summary>
+            /// Returns the symbol at the given index.
+            /// </summary>
+            public virtual Symbol GetSymbol(int index)
+            {
+                return Symbols[index];
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int index)
+            {
+                return Labels[index];
+            }
+
+            /// <summary>
+            /// Returns the size.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Symbols.Length;
+            }
+
+            /// <summary>
+            /// Returns the index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string label)
+            {
+                if (!ReferenceEquals(label, null))
+                {
+                    for (int i = 0; i < Labels.Length; i++)
+                    {
+                        if (label.Equals(Labels[i]))
+                        {
+                            return i;
+                        }
+                    }
+                }
+
+                return -1;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Symbol[] ss = new Symbol[Symbols.Length];
+                for (int i = 0; i < ss.Length; i++)
+                {
+                    ss[i] = Symbols[i].Flatten(map, map2);
+                }
+
+                return new Alternative(ss, Labels);
+            }
+        }
+
+        /// <summary>
+        /// The error action.
+        /// </summary>
+        public class ErrorAction : ImplicitAction
+        {
+            /// <summary>
+            /// The error message.
+            /// </summary>
+            public string Msg { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ErrorAction"/> class.
+            /// </summary>
+            public ErrorAction(string msg)
+            {
+                Msg = msg;
+            }
+        }
+
+        /// <summary>
+        /// Int check action.
+        /// </summary>
+        public class IntCheckAction : Symbol
+        {
+            /// <summary>
+            /// The size.
+            /// </summary>
+            public int Size { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.IntCheckAction"/> class.
+            /// </summary>
+            public IntCheckAction(int size) : base(Kind.ExplicitAction)
+            {
+                Size = size;
+            }
+        }
+
+        /// <summary>
+        /// The writer union action.
+        /// </summary>
+        public class WriterUnionAction : ImplicitAction
+        {
+        }
+
+        /// <summary>
+        /// The resolving action.
+        /// </summary>
+        public class ResolvingAction : ImplicitAction
+        {
+            /// <summary>
+            /// The writer.
+            /// </summary>
+            public Symbol Writer { get; private set; }
+
+            /// <summary>
+            /// The reader.
+            /// </summary>
+            public Symbol Reader { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ResolvingAction"/> class.
+            /// </summary>
+            public ResolvingAction(Symbol writer, Symbol reader)
+            {
+                Writer = writer;
+                Reader = reader;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new ResolvingAction(Writer.Flatten(map, map2), Reader.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The skip action.
+        /// </summary>
+        public class SkipAction : ImplicitAction
+        {
+            /// <summary>
+            /// The symbol to skip.
+            /// </summary>
+            public Symbol SymToSkip { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.SkipAction"/> class.
+            /// </summary>
+            public SkipAction(Symbol symToSkip) : base(true)
+            {
+                SymToSkip = symToSkip;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new SkipAction(SymToSkip.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The field adjust action.
+        /// </summary>
+        public class FieldAdjustAction : ImplicitAction
+        {
+            /// <summary>
+            /// The index.
+            /// </summary>
+            public int RIndex { get; private set; }
+
+            /// <summary>
+            /// The field name.
+            /// </summary>
+            public string FName { get; private set; }
+
+            /// <summary>
+            /// The field aliases.
+            /// </summary>
+            public IList<string> Aliases { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.FieldAdjustAction"/> class.
+            /// </summary>
+            public FieldAdjustAction(int rindex, string fname, IList<string> aliases)
+            {
+                RIndex = rindex;
+                FName = fname;
+                Aliases = aliases;
+            }
+        }
+
+        /// <summary>
+        /// THe field order action.
+        /// </summary>
+        public sealed class FieldOrderAction : ImplicitAction
+        {
+            /// <summary>
+            /// Whether no reorder is needed.
+            /// </summary>
+            public bool NoReorder { get; private set; }
+
+            /// <summary>
+            /// The fields.
+            /// </summary>
+            public Field[] Fields { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.FieldOrderAction"/> class.
+            /// </summary>
+            public FieldOrderAction(Field[] fields)
+            {
+                Fields = fields;
+                bool noReorder = true;
+                for (int i = 0; noReorder && i < fields.Length; i++)
+                {
+                    noReorder &= (i == fields[i].Pos);
+                }
+
+                NoReorder = noReorder;
+            }
+        }
+
+        /// <summary>
+        /// The default start action.
+        /// </summary>
+        public class DefaultStartAction : ImplicitAction
+        {
+            /// <summary>
+            /// The contents.
+            /// </summary>
+            public byte[] Contents { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.DefaultStartAction"/> class.
+            /// </summary>
+            public DefaultStartAction(byte[] contents)
+            {
+                Contents = contents;
+            }
+        }
+
+        /// <summary>
+        /// The union adjust action.
+        /// </summary>
+        public class UnionAdjustAction : ImplicitAction
+        {
+            /// <summary>
+            /// The index.
+            /// </summary>
+            public int RIndex { get; private set; }
+
+            /// <summary>
+            /// The symbol to parser.
+            /// </summary>
+            public Symbol SymToParse { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.UnionAdjustAction"/> class.
+            /// </summary>
+            public UnionAdjustAction(int rindex, Symbol symToParse)
+            {
+                RIndex = rindex;
+                SymToParse = symToParse;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new UnionAdjustAction(RIndex, SymToParse.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The enum labels action.
+        /// </summary>
+        public class EnumLabelsAction : IntCheckAction
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public IList<string> Symbols { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.EnumLabelsAction"/> class.
+            /// </summary>
+            public EnumLabelsAction(IList<string> symbols) : base(symbols.Count)
+            {
+                Symbols = symbols;
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int n)
+            {
+                return Symbols[n];
+            }
+
+            /// <summary>
+            /// Returns index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string l)
+            {
+                if (!ReferenceEquals(l, null))
+                {
+                    for (int i = 0; i < Symbols.Count; i++)
+                    {
+                        if (l.Equals(Symbols[i]))
+                        {
+                            return i;
+                        }
+                    }
+                }
+
+                return -1;
+            }
+        }
+
+        /// <summary>
+        /// The terminal symbols for the grammar.
+        /// </summary>
+        public static readonly Symbol Null = new Terminal("null");

Review Comment:
   Thanks, I went with `{ get; }` as the code does `==` checks on the symbols.



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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r953338257


##########
lang/csharp/src/apache/main/IO/Parsing/Parser.cs:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.
+ */
+
+using System;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Parser is the class that maintains the stack for parsing. This class is used
+    /// by encoders, which are not required to skip.
+    /// </summary>
+    public class Parser
+    {
+        /// <summary>
+        /// The parser knows how to handle the terminal and non-terminal symbols. But it
+        /// needs help from outside to handle implicit and explicit actions. The clients
+        /// implement this interface to provide this help.
+        /// </summary>
+        public interface IActionHandler
+        {
+            /// <summary>
+            /// Handle the action symbol <tt>top</tt> when the <tt>input</tt> is sought to be
+            /// taken off the stack.
+            /// </summary>
+            /// <param name="input"> The input symbol from the caller of advance </param>
+            /// <param name="top">   The symbol at the top the stack. </param>
+            /// <returns> <tt>null</tt> if advance() is to continue processing the stack. If
+            ///         not <tt>null</tt> the return value will be returned by advance(). </returns>
+            Symbol DoAction(Symbol input, Symbol top);
+        }
+
+        private readonly IActionHandler symbolHandler;
+        /// <summary>
+        /// Stack of symbols.
+        /// </summary>
+        protected Symbol[] Stack;
+        /// <summary>
+        /// Position of the stack.
+        /// </summary>
+        protected int Pos;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Parser"/> class.
+        /// </summary>
+        public Parser(Symbol root, IActionHandler symbolHandler)
+        {
+            this.symbolHandler = symbolHandler;
+            this.Stack = new Symbol[5]; // Start small to make sure expansion code works
+            this.Stack[0] = root;
+            this.Pos = 1;
+        }
+
+        /// <summary>
+        /// If there is no sufficient room in the stack, use this expand it.
+        /// </summary>
+        private void expandStack()
+        {
+            Array.Resize(ref Stack, Stack.Length + Math.Max(Stack.Length, 1024));
+        }
+
+        /// <summary>
+        /// Recursively replaces the symbol at the top of the stack with its production,
+        /// until the top is a terminal. Then checks if the top symbol matches the
+        /// terminal symbol supplied <tt>terminal</tt>.

Review Comment:
   Done



##########
lang/csharp/src/apache/main/IO/Parsing/SkipParser.cs:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.
+ */
+
+using System.Diagnostics;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// A parser that capable of skipping as well read and write. This class is used
+    /// by decoders who (unlink encoders) are required to implement methods to skip.

Review Comment:
   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.

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

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


[GitHub] [avro] martin-g commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
martin-g commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1226836507

   My C# knowledge is too weak to be able to review such big PR :-/
   I'll let the C# devs to manage this 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.

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

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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1225186562

   @KalleOlaviNiemitalo , I've incorporated your latest review feedback, thanks for your thoroughness.


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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1227798783

   Thanks for the review @KyleSchoonover !  I believe I've incorporated all your feedback.  Please let me know if I missed something.  I look forward to the rest of your review! 


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


[GitHub] [avro] KalleOlaviNiemitalo commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1223553494

   How about a union with a member that is a record with a namespace? The [JSON Encoding](https://avro.apache.org/docs/1.11.1/specification/#json-encoding) specification says
   
   > otherwise it is encoded as a JSON object with one name/value pair whose name is the type’s name and whose value is the recursively encoded value. For Avro’s named types (record, fixed or enum) the user-specified name is used, for other types the type name is used.
   
   which refers to the record _name_ rather than _fullname_.


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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r951080378


##########
lang/csharp/src/apache/main/IO/Parsing/Parser.cs:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.
+ */
+
+using System;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Parser is the class that maintains the stack for parsing. This class is used
+    /// by encoders, which are not required to skip.
+    /// </summary>
+    public class Parser
+    {
+        /// <summary>
+        /// The parser knows how to handle the terminal and non-terminal symbols. But it
+        /// needs help from outside to handle implicit and explicit actions. The clients
+        /// implement this interface to provide this help.
+        /// </summary>
+        public interface ActionHandler

Review Comment:
   Fixed



##########
lang/csharp/src/apache/main/IO/JsonEncoder.cs:
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.
+ */
+
+using Avro.IO.Parsing;
+using System.Collections;
+using System.IO;
+using System.Text;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// An <seealso cref="Encoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonEncoder buffers output, and data may not appear on the output until
+    /// <seealso cref="Encoder.Flush()"/> is called.
+    ///
+    /// JsonEncoder is not thread-safe.
+    /// </summary>
+    public class JsonEncoder : ParsingEncoder, Parser.ActionHandler
+    {
+        private readonly Parser parser;
+        private JsonWriter writer;
+        private bool includeNamespace = true;
+
+        // Has anything been written into the collections?
+        private readonly BitArray isEmpty = new BitArray(64);
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream) : this(sc, getJsonWriter(stream, false))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream, bool pretty) : this(sc, getJsonWriter(stream, pretty))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, JsonWriter writer)
+        {
+            Configure(writer);
+            this.parser = new Parser((new JsonGrammarGenerator()).Generate(sc), this);
+        }
+
+        /// <inheritdoc />
+        public override void Flush()
+        {
+            parser.ProcessImplicitActions();
+            if (writer != null)
+            {
+                writer.Flush();
+            }
+        }
+
+        // by default, one object per line.
+        // with pretty option use default pretty printer with root line separator.
+        private static JsonWriter getJsonWriter(Stream stream, bool pretty)
+        {
+            JsonWriter writer = new JsonTextWriter(new StreamWriter(stream));
+            if (pretty)
+            {
+                writer.Formatting = Formatting.Indented;
+            }
+
+            return writer;
+        }
+
+        /// <summary>
+        /// Whether to include the namespace.
+        /// </summary>
+        public virtual bool IncludeNamespace
+        {
+            get { return includeNamespace; }
+            set { this.includeNamespace = value; }
+        }
+
+
+        /// <summary>
+        /// Reconfigures this JsonEncoder to use the output stream provided.
+        /// <p/>
+        /// Otherwise, this JsonEncoder will flush its current output and then
+        /// reconfigure its output to use a default UTF8 JsonWriter that writes to the
+        /// provided OutputStream.

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.

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

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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1221939177

   Thanks for the review @KalleOlaviNiemitalo .  I've fixed the cosmetic issues, and I'll try to add the requested test involving logical types as soon as I get a chance.


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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1229131964

   > Hi @martin-g and @rayokota ,
   
   Hi @KhrystynaPopadyuk ,
   
   > 
   > Please find my 5 cents below.
   > 
   > As was mentioned in description this is port from Java and it looks and feels exactly as port. Of course this code will work. But it would be very difficult maintain, extend and provide sufficient unit test coverage. This is new functionality and it is easy refactor it now, and would be very difficult update if later.
   > 
   > Please find below few suggestion that, I believe, should be addressed:
   > 
   > 1. Folder and file structure. For example:
   > 
   > * Why do not create separate folder for JsonParser and placed all related classes there
   
   All the parsing classes are indeed in a separate `Parsing` package.  The `JsonEncoder` and `JsonDecoder` are siblings of `BinaryEncoder` and `BinaryDecoder` in the `IO` package, which makes sense to me.
   
   > * "ParsingDecoder" (https://github.com/apache/avro/blob/c13a45debf4594dc7bc3c90aef64cee5ae115a5d/lang/csharp/src/apache/main/IO/ParsingDecoder.cs) It states that this is "Base class for a -based" but Parser (https://github.com/apache/avro/blob/c13a45debf4594dc7bc3c90aef64cee5ae115a5d/lang/csharp/src/apache/main/IO/Parsing/Parser.cs) does no have base class. Moreover ParsingDecoder is base class for JsonDecoder. So is "ParsingDecoder " correct name? What is the purpose of having this class? Why we need this additional layer of abstraction?
   
   The `ParsingEncoder` and `ParsingDecoder` are implementations of `Encoder` and `Decoder` that maintain some parsing state.  The `JsonEncoder` and `JsonDecoder` subclass those classes to delegate the maintenance of the parsing state.  In the Java code, there are also `ValidatingEncoder` and `ValidatingDecoder` that subclass `ParsingEncoder` and `ParsingDecoder`.  I did not port those classes over.
   
   > * there are a lot of child classes and interfaces. For example file Symbol.cs contains not only abstract class Symbol but 17 child classes and one enum, And all definitions of fields, properties, child classes, enums, methods are without any structure and order.
   
   The `Symbol` class maintains the terminals and non-terminals of the parsing grammar, so that is why there are a lot of classes and enums in it.
   
   > * there are a lot of other things that need attention
   > 
   > 2. Static method and classes is very controversial tool. It has own benefits but also it can brink a lot of negative. Overall we should be use them carefully and with attention. I have founded 38 static items in Symbol class. That looks very suspicious.
   
   I agree static methods and classes have their advantages and disadvantages, but I believe there is nothing wrong in the use of them in the `Symbol` class.  I'd be happy to incorporate more specific suggestions at particular line numbers.
   
   > 3. Poor usage of interfaces but a lot of static and even hidden dependencies. That should be reviewed in direction to use interfaces as dependency instead of classes (even abstract). Dependency inversion/ dependency injection can bring a lot of benefit and flexibility to code.
   
   Dependency inversion also has its advantages and disadvantages, but I don't think it would really help here, unless you can be more specific.
   
   > 4. switch operator - is there way to reduce usage of switch/case?
   
   I think `switch` can also have its advantages and disadvantages, but I think its use is fine 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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r955440899


##########
lang/csharp/src/apache/main/IO/Parsing/Parser.cs:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.
+ */
+
+using System;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Parser is the class that maintains the stack for parsing. This class is used
+    /// by encoders, which are not required to skip.
+    /// </summary>
+    public class Parser
+    {
+        /// <summary>
+        /// The parser knows how to handle the terminal and non-terminal symbols. But it
+        /// needs help from outside to handle implicit and explicit actions. The clients
+        /// implement this interface to provide this help.
+        /// </summary>
+        public interface IActionHandler
+        {
+            /// <summary>
+            /// Handle the action symbol <tt>top</tt> when the <tt>input</tt> is sought to be
+            /// taken off the stack.
+            /// </summary>
+            /// <param name="input"> The input symbol from the caller of Advance </param>
+            /// <param name="top">   The symbol at the top the stack. </param>
+            /// <returns> <tt>null</tt> if Advance() is to continue processing the stack. If
+            ///         not <tt>null</tt> the return value will be returned by Advance(). </returns>
+            Symbol DoAction(Symbol input, Symbol top);
+        }
+
+        private readonly IActionHandler symbolHandler;
+        /// <summary>
+        /// Stack of symbols.
+        /// </summary>
+        protected Symbol[] Stack;
+        /// <summary>
+        /// Position of the stack.
+        /// </summary>
+        protected int Pos;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Parser"/> class.
+        /// </summary>
+        public Parser(Symbol root, IActionHandler symbolHandler)
+        {
+            this.symbolHandler = symbolHandler;
+            this.Stack = new Symbol[5]; // Start small to make sure expansion code works
+            this.Stack[0] = root;
+            this.Pos = 1;
+        }
+
+        /// <summary>
+        /// If there is no sufficient room in the stack, use this expand it.
+        /// </summary>
+        private void expandStack()
+        {
+            Array.Resize(ref Stack, Stack.Length + Math.Max(Stack.Length, 1024));
+        }
+
+        /// <summary>
+        /// Recursively replaces the symbol at the top of the stack with its production,
+        /// until the top is a terminal. Then checks if the top symbol matches the
+        /// terminal symbol supplied <tt>input</tt>.
+        /// </summary>
+        /// <param name="input"> The symbol to match against the terminal at the top of the
+        ///              stack. </param>
+        /// <returns> The terminal symbol at the top of the stack unless an implicit action
+        ///         resulted in another symbol, in which case that symbol is returned. </returns>
+        public Symbol Advance(Symbol input)
+        {
+            for (;;)
+            {
+                Symbol top = Stack[--Pos];
+                if (top == input)
+                {
+                    return top; // A common case
+                }
+
+                Symbol.Kind k = top.SymKind;
+                if (k == Symbol.Kind.ImplicitAction)
+                {
+                    Symbol result = symbolHandler.DoAction(input, top);
+                    if (result != null)
+                    {
+                        return result;
+                    }
+                }
+                else if (k == Symbol.Kind.Terminal)
+                {
+                    throw new AvroTypeException("Attempt to process a " + input + " when a " + top + " was expected.");
+                }
+                else if (k == Symbol.Kind.Repeater && input == ((Symbol.Repeater)top).End)
+                {
+                    return input;
+                }
+                else
+                {
+                    PushProduction(top);

Review Comment:
   The second branch does not return if `result` is null.



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


[GitHub] [avro] KalleOlaviNiemitalo commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1225380917

   > Ok so I should file a separate issue for name vs. fullname in JSON encoding of unions.
   
   Filed as [AVRO-3620](https://issues.apache.org/jira/browse/AVRO-3620).


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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1226872623

   > My C# knowledge is too weak to be able to review such big PR :-/
   > I'll let the C# devs to manage this one.
   
   Thanks @martin-g .  This PR is mostly a line-for-line port of the following Java classes:
   
   - JsonGrammarGenerator.java
   - Parser.java
   - SkipParser.java
   - Symbol.java
   - ValidatingGrammarGenerator.java
   - ParsingEncoder.java
   - ParsingDecoder.java
   - JsonEncoder.java
   - JsonDecoder.java
   
   In any case, who are the C# devs who would be able 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.

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

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


[GitHub] [avro] martin-g merged pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
martin-g merged PR #1833:
URL: https://github.com/apache/avro/pull/1833


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


[GitHub] [avro] github-code-scanning[bot] commented on a diff in pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r950941158


##########
lang/csharp/src/apache/main/IO/Parsing/ValidatingGrammarGenerator.cs:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// The class that generates validating grammar.
+    /// </summary>
+    public class ValidatingGrammarGenerator
+    {
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// given schema <tt>sc</tt>.
+        /// </summary>
+        public virtual Symbol Generate(Schema schema)
+        {
+            return Symbol.NewRoot(Generate(schema, new Dictionary<LitS, Symbol>()));
+        }
+
+        /// <summary>
+        /// Returns the non-terminal that is the start symbol for the grammar for the
+        /// given schema <tt>sc</tt>. If there is already an entry for the given schema
+        /// in the given map <tt>seen</tt> then that entry is returned. Otherwise a new
+        /// symbol is generated and an entry is inserted into the map.
+        /// </summary>
+        /// <param name="sc">   The schema for which the start symbol is required </param>
+        /// <param name="seen"> A map of schema to symbol mapping done so far. </param>
+        /// <returns> The start symbol for the schema </returns>
+        protected virtual Symbol Generate(Schema sc, IDictionary<LitS, Symbol> seen)
+        {
+            switch (sc.Tag)
+            {
+                case Schema.Type.Null:
+                    return Symbol.Null;
+                case Schema.Type.Boolean:
+                    return Symbol.Boolean;
+                case Schema.Type.Int:
+                    return Symbol.Int;
+                case Schema.Type.Long:
+                    return Symbol.Long;
+                case Schema.Type.Float:
+                    return Symbol.Float;
+                case Schema.Type.Double:
+                    return Symbol.Double;
+                case Schema.Type.String:
+                    return Symbol.String;
+                case Schema.Type.Bytes:
+                    return Symbol.Bytes;
+                case Schema.Type.Fixed:
+                    return Symbol.NewSeq(new Symbol.IntCheckAction(((FixedSchema)sc).Size), Symbol.Fixed);
+                case Schema.Type.Enumeration:
+                    return Symbol.NewSeq(new Symbol.IntCheckAction(((EnumSchema)sc).Symbols.Count), Symbol.Enum);
+                case Schema.Type.Array:
+                    return Symbol.NewSeq(
+                        Symbol.NewRepeat(Symbol.ArrayEnd, Generate(((ArraySchema)sc).ItemSchema, seen)),
+                        Symbol.ArrayStart);
+                case Schema.Type.Map:
+                    return Symbol.NewSeq(
+                        Symbol.NewRepeat(Symbol.MapEnd, Generate(((MapSchema)sc).ValueSchema, seen), Symbol.String),
+                        Symbol.MapStart);
+                case Schema.Type.Record:
+                    {
+                        LitS wsc = new LitS(sc);
+                        Symbol rresult = seen.ContainsKey(wsc) ? seen[wsc] : null;
+                        if (rresult == null)
+                        {
+                            Symbol[] production = new Symbol[((RecordSchema)sc).Fields.Count];
+
+                            // We construct a symbol without filling the array. Please see
+                            // <seealso cref="Symbol.production"/> for the reason.
+                            rresult = Symbol.NewSeq(production);
+                            seen[wsc] = rresult;
+
+                            int j = production.Length;
+                            foreach (Field f in ((RecordSchema)sc).Fields)
+                            {
+                                production[--j] = Generate(f.Schema, seen);
+                            }
+                        }
+
+                        return rresult;
+                    }
+                case Schema.Type.Union:
+                    IList<Schema> subs = ((UnionSchema)sc).Schemas;
+                    Symbol[] symbols = new Symbol[subs.Count];
+                    string[] labels = new string[subs.Count];
+
+                    int i = 0;
+                    foreach (Schema b in ((UnionSchema)sc).Schemas)
+                    {
+                        symbols[i] = Generate(b, seen);
+                        labels[i] = b.Fullname;
+                        i++;
+                    }
+
+                    return Symbol.NewSeq(Symbol.NewAlt(symbols, labels), Symbol.Union);
+
+                default:
+                    throw new Exception("Unexpected schema type");
+            }
+        }
+
+        /// <summary>
+        /// A wrapper around Schema that does "==" equality.
+        /// </summary>
+        protected class LitS
+        {
+            private readonly Schema actual;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="LitS"/> class.
+            /// </summary>
+            public LitS(Schema actual)
+            {
+                this.actual = actual;
+            }
+
+            /// <summary>
+            /// Two LitS are equal if and only if their underlying schema is the same (not
+            /// merely equal).
+            /// </summary>
+            public override bool Equals(object o)
+            {
+                if (!(o is LitS))

Review Comment:
   ## Equals should not apply "is"
   
   LitS.Equals(object) should not use "is" on its parameter, as it will not work properly for subclasses of LitS.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2904)



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1044 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public readonly Symbol[] Symbols;
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)

Review Comment:
   ## Exposing internal representation
   
   'Fixup' exposes the internal representation stored in field 'Symbols'. The value may be modified [through the variable out](1).
   'Fixup' exposes the internal representation stored in field 'Symbols'. The value may be modified [through the variable out](2).
   'Fixup' exposes the internal representation stored in field 'Symbols'. The value may be modified [through the variable out](3).
   'Fixup' exposes the internal representation stored in field 'Symbols'. The value may be modified [through the variable out](4).
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2903)



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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1222815474

   @KalleOlaviNiemitalo , actually the schema above does not parse in C#.  It turns out that the C# implementation only allows one logical type in a union.  I've filed https://issues.apache.org/jira/browse/AVRO-3613.
   
   In any case, I've added a test with one logical type in a union.  Please let me know if there's anything else you'd like to see.  Thanks!


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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r953340246


##########
lang/csharp/src/apache/main/IO/Parsing/Parser.cs:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.
+ */
+
+using System;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Parser is the class that maintains the stack for parsing. This class is used
+    /// by encoders, which are not required to skip.
+    /// </summary>
+    public class Parser
+    {
+        /// <summary>
+        /// The parser knows how to handle the terminal and non-terminal symbols. But it
+        /// needs help from outside to handle implicit and explicit actions. The clients
+        /// implement this interface to provide this help.
+        /// </summary>
+        public interface IActionHandler
+        {
+            /// <summary>
+            /// Handle the action symbol <tt>top</tt> when the <tt>input</tt> is sought to be
+            /// taken off the stack.
+            /// </summary>
+            /// <param name="input"> The input symbol from the caller of advance </param>
+            /// <param name="top">   The symbol at the top the stack. </param>
+            /// <returns> <tt>null</tt> if advance() is to continue processing the stack. If
+            ///         not <tt>null</tt> the return value will be returned by advance(). </returns>

Review Comment:
   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.

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

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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1230744178

   Thanks @martin-g .  I added unit tests to test all Avro types, both with valid and invalid values, as well as tests for malformed JSON.


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


[GitHub] [avro] KyleSchoonover commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KyleSchoonover commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r955394370


##########
lang/csharp/src/apache/main/IO/Parsing/Parser.cs:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.
+ */
+
+using System;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Parser is the class that maintains the stack for parsing. This class is used
+    /// by encoders, which are not required to skip.
+    /// </summary>
+    public class Parser
+    {
+        /// <summary>
+        /// The parser knows how to handle the terminal and non-terminal symbols. But it
+        /// needs help from outside to handle implicit and explicit actions. The clients
+        /// implement this interface to provide this help.
+        /// </summary>
+        public interface IActionHandler
+        {
+            /// <summary>
+            /// Handle the action symbol <tt>top</tt> when the <tt>input</tt> is sought to be
+            /// taken off the stack.
+            /// </summary>
+            /// <param name="input"> The input symbol from the caller of Advance </param>
+            /// <param name="top">   The symbol at the top the stack. </param>
+            /// <returns> <tt>null</tt> if Advance() is to continue processing the stack. If
+            ///         not <tt>null</tt> the return value will be returned by Advance(). </returns>
+            Symbol DoAction(Symbol input, Symbol top);
+        }
+
+        private readonly IActionHandler symbolHandler;
+        /// <summary>
+        /// Stack of symbols.
+        /// </summary>
+        protected Symbol[] Stack;
+        /// <summary>
+        /// Position of the stack.
+        /// </summary>
+        protected int Pos;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Parser"/> class.
+        /// </summary>
+        public Parser(Symbol root, IActionHandler symbolHandler)
+        {
+            this.symbolHandler = symbolHandler;
+            this.Stack = new Symbol[5]; // Start small to make sure expansion code works
+            this.Stack[0] = root;
+            this.Pos = 1;
+        }
+
+        /// <summary>
+        /// If there is no sufficient room in the stack, use this expand it.
+        /// </summary>
+        private void expandStack()
+        {
+            Array.Resize(ref Stack, Stack.Length + Math.Max(Stack.Length, 1024));
+        }
+
+        /// <summary>
+        /// Recursively replaces the symbol at the top of the stack with its production,
+        /// until the top is a terminal. Then checks if the top symbol matches the
+        /// terminal symbol supplied <tt>input</tt>.
+        /// </summary>
+        /// <param name="input"> The symbol to match against the terminal at the top of the
+        ///              stack. </param>
+        /// <returns> The terminal symbol at the top of the stack unless an implicit action
+        ///         resulted in another symbol, in which case that symbol is returned. </returns>
+        public Symbol Advance(Symbol input)
+        {
+            for (;;)
+            {
+                Symbol top = Stack[--Pos];
+                if (top == input)
+                {
+                    return top; // A common case
+                }
+
+                Symbol.Kind k = top.SymKind;
+                if (k == Symbol.Kind.ImplicitAction)
+                {
+                    Symbol result = symbolHandler.DoAction(input, top);
+                    if (result != null)
+                    {
+                        return result;
+                    }
+                }
+                else if (k == Symbol.Kind.Terminal)
+                {
+                    throw new AvroTypeException("Attempt to process a " + input + " when a " + top + " was expected.");
+                }
+                else if (k == Symbol.Kind.Repeater && input == ((Symbol.Repeater)top).End)
+                {
+                    return input;
+                }
+                else
+                {
+                    PushProduction(top);

Review Comment:
   Can we get rid of the last else block and just put `PushProduction(top);`?  This is more for readability that it always will run this action if the conditions are not meat, or returned early.



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)

Review Comment:
   Recommendation to use expression body



##########
lang/csharp/src/apache/main/IO/Parsing/Parser.cs:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.
+ */
+
+using System;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Parser is the class that maintains the stack for parsing. This class is used
+    /// by encoders, which are not required to skip.
+    /// </summary>
+    public class Parser
+    {
+        /// <summary>
+        /// The parser knows how to handle the terminal and non-terminal symbols. But it
+        /// needs help from outside to handle implicit and explicit actions. The clients
+        /// implement this interface to provide this help.
+        /// </summary>
+        public interface IActionHandler
+        {
+            /// <summary>
+            /// Handle the action symbol <tt>top</tt> when the <tt>input</tt> is sought to be
+            /// taken off the stack.
+            /// </summary>
+            /// <param name="input"> The input symbol from the caller of Advance </param>
+            /// <param name="top">   The symbol at the top the stack. </param>
+            /// <returns> <tt>null</tt> if Advance() is to continue processing the stack. If
+            ///         not <tt>null</tt> the return value will be returned by Advance(). </returns>
+            Symbol DoAction(Symbol input, Symbol top);
+        }
+
+        private readonly IActionHandler symbolHandler;
+        /// <summary>
+        /// Stack of symbols.
+        /// </summary>
+        protected Symbol[] Stack;
+        /// <summary>
+        /// Position of the stack.
+        /// </summary>
+        protected int Pos;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Parser"/> class.
+        /// </summary>
+        public Parser(Symbol root, IActionHandler symbolHandler)
+        {
+            this.symbolHandler = symbolHandler;
+            this.Stack = new Symbol[5]; // Start small to make sure expansion code works

Review Comment:
   remove `this.` for Stack and Pos



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;

Review Comment:
   Can remove `this.`



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)

Review Comment:
   This is more of a recommendation to use expression bodies
   `public static Symbol NewRoot(params Symbol[] symbols) => new Root(symbols);`



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)

Review Comment:
   Recommendation to use expression body



##########
lang/csharp/src/apache/main/IO/Parsing/Parser.cs:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.
+ */
+
+using System;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Parser is the class that maintains the stack for parsing. This class is used
+    /// by encoders, which are not required to skip.
+    /// </summary>
+    public class Parser
+    {
+        /// <summary>
+        /// The parser knows how to handle the terminal and non-terminal symbols. But it
+        /// needs help from outside to handle implicit and explicit actions. The clients
+        /// implement this interface to provide this help.
+        /// </summary>
+        public interface IActionHandler
+        {
+            /// <summary>
+            /// Handle the action symbol <tt>top</tt> when the <tt>input</tt> is sought to be
+            /// taken off the stack.
+            /// </summary>
+            /// <param name="input"> The input symbol from the caller of Advance </param>
+            /// <param name="top">   The symbol at the top the stack. </param>
+            /// <returns> <tt>null</tt> if Advance() is to continue processing the stack. If
+            ///         not <tt>null</tt> the return value will be returned by Advance(). </returns>
+            Symbol DoAction(Symbol input, Symbol top);
+        }
+
+        private readonly IActionHandler symbolHandler;
+        /// <summary>
+        /// Stack of symbols.
+        /// </summary>
+        protected Symbol[] Stack;
+        /// <summary>
+        /// Position of the stack.
+        /// </summary>
+        protected int Pos;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Parser"/> class.
+        /// </summary>
+        public Parser(Symbol root, IActionHandler symbolHandler)
+        {
+            this.symbolHandler = symbolHandler;
+            this.Stack = new Symbol[5]; // Start small to make sure expansion code works
+            this.Stack[0] = root;
+            this.Pos = 1;
+        }
+
+        /// <summary>
+        /// If there is no sufficient room in the stack, use this expand it.
+        /// </summary>
+        private void expandStack()

Review Comment:
   Use Pascal Casing



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)

Review Comment:
   Recommendation to use expression body



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)

Review Comment:
   Recommendation to use expression body



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()

Review Comment:
   Recommendation to use expression body, or should this be a property?



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)

Review Comment:
   Recommendation to use expression body



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))

Review Comment:
   Can inline the out variable
   `Symbol[] p = s.Production;`
   `if (!map2.TryGetValue((Sequence)s, out IList<Fixup> l))`



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)

Review Comment:
   Recommendation to use expression body



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;

Review Comment:
   Don't need `this.` for Pos



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;

Review Comment:
   `public string PrintName { get; private set; }`



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;

Review Comment:
   Remove `this.`



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;

Review Comment:
   `public bool IsTrailing { get; private set; }`



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()

Review Comment:
   recommend to make expression body



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, makeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index)
+            {
+                return Production[index];
+            }
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size()

Review Comment:
   Recommend that we use an expression body or make this a property



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;

Review Comment:
   Remove `this.`



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, makeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index)

Review Comment:
   Recommend that we use expression body.  Additionally, should we implement indexers here?



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, makeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index)
+            {
+                return Production[index];
+            }
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Production.Length;
+            }
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator()
+            {
+                return Enumerable.Reverse(Production).GetEnumerator();
+            }
+
+            IEnumerator IEnumerable.GetEnumerator()
+            {
+                return this.GetEnumerator();
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Sequence result;
+                if (!map.TryGetValue(this, out result))

Review Comment:
   Can inline Sequence
   `if (!map.TryGetValue(this, out Sequence result))`



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, makeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index)
+            {
+                return Production[index];
+            }
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Production.Length;
+            }
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator()

Review Comment:
   Recommend that we use an expression body



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, makeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index)
+            {
+                return Production[index];
+            }
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Production.Length;
+            }
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator()
+            {
+                return Enumerable.Reverse(Production).GetEnumerator();
+            }
+
+            IEnumerator IEnumerable.GetEnumerator()

Review Comment:
   Recommend that we use an expression body.
   Don't need to use `this.`



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, makeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index)
+            {
+                return Production[index];
+            }
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Production.Length;
+            }
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator()
+            {
+                return Enumerable.Reverse(Production).GetEnumerator();
+            }
+
+            IEnumerator IEnumerable.GetEnumerator()
+            {
+                return this.GetEnumerator();
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Sequence result;
+                if (!map.TryGetValue(this, out result))
+                {
+                    result = new Sequence(new Symbol[FlattenedSize()]);
+                    map[this] = result;
+                    IList<Fixup> l = new List<Fixup>();
+                    map2[result] = l;
+
+                    Flatten(Production, 0, result.Production, 0, map, map2);
+                    foreach (Fixup f in l)
+                    {
+                        Array.Copy(result.Production, 0, f.Symbols, f.Pos, result.Production.Length);
+                    }
+
+                    map2.Remove(result);
+                }
+
+                return result;
+            }
+
+            /// <inheritdoc />
+            public override int FlattenedSize()
+            {
+                return FlattenedSize(Production, 0);
+            }
+        }
+
+        /// <summary>
+        /// Repeater symbol.
+        /// </summary>
+        public class Repeater : Symbol
+        {
+            /// <summary>
+            /// The end symbol.
+            /// </summary>
+            public readonly Symbol End;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Repeater"/> class.
+            /// </summary>
+            public Repeater(Symbol end, params Symbol[] sequenceToRepeat) : base(Kind.Repeater,
+                makeProduction(sequenceToRepeat))
+            {
+                this.End = end;

Review Comment:
   Remove `this.`



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, makeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index)
+            {
+                return Production[index];
+            }
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Production.Length;
+            }
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator()
+            {
+                return Enumerable.Reverse(Production).GetEnumerator();
+            }
+
+            IEnumerator IEnumerable.GetEnumerator()
+            {
+                return this.GetEnumerator();
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Sequence result;
+                if (!map.TryGetValue(this, out result))
+                {
+                    result = new Sequence(new Symbol[FlattenedSize()]);
+                    map[this] = result;
+                    IList<Fixup> l = new List<Fixup>();
+                    map2[result] = l;
+
+                    Flatten(Production, 0, result.Production, 0, map, map2);
+                    foreach (Fixup f in l)
+                    {
+                        Array.Copy(result.Production, 0, f.Symbols, f.Pos, result.Production.Length);
+                    }
+
+                    map2.Remove(result);
+                }
+
+                return result;
+            }
+
+            /// <inheritdoc />
+            public override int FlattenedSize()
+            {
+                return FlattenedSize(Production, 0);
+            }
+        }
+
+        /// <summary>
+        /// Repeater symbol.
+        /// </summary>
+        public class Repeater : Symbol
+        {
+            /// <summary>
+            /// The end symbol.
+            /// </summary>
+            public readonly Symbol End;

Review Comment:
   `public Symbol End { get; private set; }`



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, makeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index)
+            {
+                return Production[index];
+            }
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Production.Length;
+            }
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator()
+            {
+                return Enumerable.Reverse(Production).GetEnumerator();
+            }
+
+            IEnumerator IEnumerable.GetEnumerator()
+            {
+                return this.GetEnumerator();
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Sequence result;
+                if (!map.TryGetValue(this, out result))
+                {
+                    result = new Sequence(new Symbol[FlattenedSize()]);
+                    map[this] = result;
+                    IList<Fixup> l = new List<Fixup>();
+                    map2[result] = l;
+
+                    Flatten(Production, 0, result.Production, 0, map, map2);
+                    foreach (Fixup f in l)
+                    {
+                        Array.Copy(result.Production, 0, f.Symbols, f.Pos, result.Production.Length);
+                    }
+
+                    map2.Remove(result);
+                }
+
+                return result;
+            }
+
+            /// <inheritdoc />
+            public override int FlattenedSize()

Review Comment:
   Recommend using an expression body



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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r955451590


##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1014 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, makeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index)

Review Comment:
   Thanks, added an indexer.



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


[GitHub] [avro] RyanSkraba commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

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

   Thanks for stepping up here @KyleSchoonover and @KalleOlaviNiemitalo!  When you approve this PR, let's merge it for 1.12.0!
   


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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1228762719

   Thanks for the thorough review @KyleSchoonover , I've incorporated your latest feedback.


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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r954622532


##########
lang/csharp/src/apache/main/IO/Encoder.cs:
##########
@@ -187,5 +187,10 @@ public interface Encoder
         /// <param name="start">Position within data where the contents start.</param>
         /// <param name="len">Number of bytes to write.</param>
         void WriteFixed(byte[] data, int start, int len);
+
+        /// <summary>
+        /// Flushes the encoder.
+        /// </summary>
+        void Flush();

Review Comment:
   JsonEncoder.Flush would be OK for branch-1.11, of course. Only Encoder.Flush is a compatibility problem. I see Avro.IO.BinaryEncoder already has `public void Flush()` in release-1.11.1.



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


[GitHub] [avro] KyleSchoonover commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KyleSchoonover commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r955179534


##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;

Review Comment:
   Update to:
   `private JsonReader origParser;
   
    public JsonReader OrigParser { get => origParser; set => origParser = value; }`



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)

Review Comment:
   Does this have to return "this?"  Seems more like it would be void.



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();

Review Comment:
   remove this.



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));

Review Comment:
   using `this.` is unnecessary



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));

Review Comment:
   Is it possible to use reader.ReadAsString()?



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);

Review Comment:
   Can you use reader.ReadAsString()?



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)

Review Comment:
   Use Pascal Casing



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);
+                int n = top.FindLabel(label);
+                if (n >= 0)
+                {
+                    reader.Read();
+                    return n;
+                }
+
+                throw new AvroTypeException("Unknown symbol in enum " + label);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayStart()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Read();
+                return doArrayNext();
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doArrayNext();
+        }
+
+        private long doArrayNext()
+        {
+            if (reader.TokenType == JsonToken.EndArray)
+            {
+                Parser.Advance(Symbol.ArrayEnd);
+                reader.Read();
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipArray()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.ArrayEnd);
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapStart()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                return doMapNext();
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doMapNext();
+        }
+
+        private long doMapNext()
+        {
+            if (reader.TokenType == JsonToken.EndObject)
+            {
+                reader.Read();
+                advance(Symbol.MapEnd);
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipMap()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.MapEnd);
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadUnionIndex()
+        {
+            advance(Symbol.Union);
+            Symbol.Alternative a = (Symbol.Alternative)Parser.PopSymbol();
+
+            string label;
+            if (reader.TokenType == JsonToken.Null)
+            {
+                label = "null";
+            }
+            else if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    label = Convert.ToString(reader.Value);
+                    reader.Read();
+                    Parser.PushSymbol(Symbol.UnionEnd);
+                }
+                else
+                {
+                    throw error("start-union");
+                }
+            }
+            else
+            {
+                throw error("start-union");
+            }
+
+            int n = a.FindLabel(label);
+            if (n < 0)
+            {
+                throw new AvroTypeException("Unknown union branch " + label);
+            }
+
+            Parser.PushSymbol(a.GetSymbol(n));
+            return n;
+        }
+
+        /// <inheritdoc />
+        public override void SkipNull()
+        {
+            ReadNull();
+        }
+
+        /// <inheritdoc />
+        public override void SkipBoolean()
+        {
+            ReadBoolean();
+        }
+
+        /// <inheritdoc />
+        public override void SkipInt()
+        {
+            ReadInt();
+        }
+
+        /// <inheritdoc />
+        public override void SkipLong()
+        {
+            ReadLong();
+        }
+
+        /// <inheritdoc />
+        public override void SkipFloat()
+        {
+            ReadFloat();
+        }
+
+        /// <inheritdoc />
+        public override void SkipDouble()
+        {
+            ReadDouble();
+        }
+
+        /// <inheritdoc />
+        public override void SkipEnum()
+        {
+            ReadEnum();
+        }
+
+        /// <inheritdoc />
+        public override void SkipUnionIndex()
+        {
+            ReadUnionIndex();
+        }
+
+        /// <inheritdoc />
+        public override Symbol DoAction(Symbol input, Symbol top)
+        {
+            if (top is Symbol.FieldAdjustAction)
+            {
+                Symbol.FieldAdjustAction fa = (Symbol.FieldAdjustAction)top;
+                string name = fa.FName;
+                if (currentReorderBuffer != null)
+                {
+                    IList<JsonElement> node = currentReorderBuffer.SavedFields[name];
+                    if (node != null)
+                    {
+                        currentReorderBuffer.SavedFields.Remove(name);
+                        currentReorderBuffer.OrigParser = reader;
+                        reader = makeParser(node);
+                        return null;
+                    }
+                }
+
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    do
+                    {
+                        string fn = Convert.ToString(reader.Value);
+                        reader.Read();
+                        if (name.Equals(fn) || (fa.Aliases != null && fa.Aliases.Contains(fn)))
+                        {
+                            return null;
+                        }
+                        else
+                        {
+                            if (currentReorderBuffer == null)
+                            {
+                                currentReorderBuffer = new ReorderBuffer();
+                            }
+
+                            currentReorderBuffer.SavedFields[fn] = getValueAsTree(reader);
+                        }
+                    } while (reader.TokenType == JsonToken.PropertyName);
+
+                    throw new AvroTypeException("Expected field name not found: " + fa.FName);
+                }
+            }
+            else if (top == Symbol.FieldEnd)
+            {
+                if (currentReorderBuffer != null && currentReorderBuffer.OrigParser != null)
+                {
+                    reader = currentReorderBuffer.OrigParser;
+                    currentReorderBuffer.OrigParser = null;
+                }
+            }
+            else if (top == Symbol.RecordStart)
+            {
+                if (reader.TokenType == JsonToken.StartObject)
+                {
+                    reader.Read();
+                    reorderBuffers.Push(currentReorderBuffer);
+                    currentReorderBuffer = null;
+                }
+                else
+                {
+                    throw error("record-start");
+                }
+            }
+            else if (top == Symbol.RecordEnd || top == Symbol.UnionEnd)
+            {
+                // AVRO-2034 advance to the end of our object
+                while (reader.TokenType != JsonToken.EndObject)
+                {
+                    reader.Read();
+                }
+
+                if (top == Symbol.RecordEnd)
+                {
+                    if (currentReorderBuffer != null && currentReorderBuffer.SavedFields.Count > 0)
+                    {
+                        throw error("Unknown fields: " + currentReorderBuffer.SavedFields.Keys);
+                    }
+
+                    currentReorderBuffer = reorderBuffers.Pop();
+                }
+
+                // AVRO-2034 advance beyond the end object for the next record.
+                reader.Read();
+            }
+            else
+            {
+                throw new AvroTypeException("Unknown action symbol " + top);
+            }
+
+            return null;
+        }
+
+
+        private class JsonElement
+        {
+            public readonly JsonToken Token;
+            public readonly object Value;
+
+            public JsonElement(JsonToken t, object value)
+            {
+                this.Token = t;
+                this.Value = value;
+            }
+
+            public JsonElement(JsonToken t) : this(t, null)
+            {
+            }
+        }
+
+        private static IList<JsonElement> getValueAsTree(JsonReader reader)

Review Comment:
   Use Pascal Casing



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));

Review Comment:
   using `this.` is unnecessary



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);
+                int n = top.FindLabel(label);
+                if (n >= 0)
+                {
+                    reader.Read();
+                    return n;
+                }
+
+                throw new AvroTypeException("Unknown symbol in enum " + label);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayStart()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Read();
+                return doArrayNext();
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doArrayNext();
+        }
+
+        private long doArrayNext()
+        {
+            if (reader.TokenType == JsonToken.EndArray)
+            {
+                Parser.Advance(Symbol.ArrayEnd);
+                reader.Read();
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipArray()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.ArrayEnd);
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapStart()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                return doMapNext();
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doMapNext();
+        }
+
+        private long doMapNext()
+        {
+            if (reader.TokenType == JsonToken.EndObject)
+            {
+                reader.Read();
+                advance(Symbol.MapEnd);
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipMap()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.MapEnd);
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadUnionIndex()
+        {
+            advance(Symbol.Union);
+            Symbol.Alternative a = (Symbol.Alternative)Parser.PopSymbol();
+
+            string label;
+            if (reader.TokenType == JsonToken.Null)
+            {
+                label = "null";
+            }
+            else if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    label = Convert.ToString(reader.Value);
+                    reader.Read();
+                    Parser.PushSymbol(Symbol.UnionEnd);
+                }
+                else
+                {
+                    throw error("start-union");
+                }
+            }
+            else
+            {
+                throw error("start-union");
+            }
+
+            int n = a.FindLabel(label);
+            if (n < 0)
+            {
+                throw new AvroTypeException("Unknown union branch " + label);
+            }
+
+            Parser.PushSymbol(a.GetSymbol(n));
+            return n;
+        }
+
+        /// <inheritdoc />
+        public override void SkipNull()
+        {
+            ReadNull();
+        }
+
+        /// <inheritdoc />
+        public override void SkipBoolean()
+        {
+            ReadBoolean();
+        }
+
+        /// <inheritdoc />
+        public override void SkipInt()
+        {
+            ReadInt();
+        }
+
+        /// <inheritdoc />
+        public override void SkipLong()
+        {
+            ReadLong();
+        }
+
+        /// <inheritdoc />
+        public override void SkipFloat()
+        {
+            ReadFloat();
+        }
+
+        /// <inheritdoc />
+        public override void SkipDouble()
+        {
+            ReadDouble();
+        }
+
+        /// <inheritdoc />
+        public override void SkipEnum()
+        {
+            ReadEnum();
+        }
+
+        /// <inheritdoc />
+        public override void SkipUnionIndex()
+        {
+            ReadUnionIndex();
+        }
+
+        /// <inheritdoc />
+        public override Symbol DoAction(Symbol input, Symbol top)
+        {
+            if (top is Symbol.FieldAdjustAction)
+            {
+                Symbol.FieldAdjustAction fa = (Symbol.FieldAdjustAction)top;
+                string name = fa.FName;
+                if (currentReorderBuffer != null)
+                {
+                    IList<JsonElement> node = currentReorderBuffer.SavedFields[name];
+                    if (node != null)
+                    {
+                        currentReorderBuffer.SavedFields.Remove(name);
+                        currentReorderBuffer.OrigParser = reader;
+                        reader = makeParser(node);
+                        return null;
+                    }
+                }
+
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    do
+                    {
+                        string fn = Convert.ToString(reader.Value);
+                        reader.Read();
+                        if (name.Equals(fn) || (fa.Aliases != null && fa.Aliases.Contains(fn)))
+                        {
+                            return null;
+                        }
+                        else
+                        {
+                            if (currentReorderBuffer == null)
+                            {
+                                currentReorderBuffer = new ReorderBuffer();
+                            }
+
+                            currentReorderBuffer.SavedFields[fn] = getValueAsTree(reader);
+                        }
+                    } while (reader.TokenType == JsonToken.PropertyName);
+
+                    throw new AvroTypeException("Expected field name not found: " + fa.FName);
+                }
+            }
+            else if (top == Symbol.FieldEnd)
+            {
+                if (currentReorderBuffer != null && currentReorderBuffer.OrigParser != null)
+                {
+                    reader = currentReorderBuffer.OrigParser;
+                    currentReorderBuffer.OrigParser = null;
+                }
+            }
+            else if (top == Symbol.RecordStart)
+            {
+                if (reader.TokenType == JsonToken.StartObject)
+                {
+                    reader.Read();
+                    reorderBuffers.Push(currentReorderBuffer);
+                    currentReorderBuffer = null;
+                }
+                else
+                {
+                    throw error("record-start");
+                }
+            }
+            else if (top == Symbol.RecordEnd || top == Symbol.UnionEnd)
+            {
+                // AVRO-2034 advance to the end of our object
+                while (reader.TokenType != JsonToken.EndObject)
+                {
+                    reader.Read();
+                }
+
+                if (top == Symbol.RecordEnd)
+                {
+                    if (currentReorderBuffer != null && currentReorderBuffer.SavedFields.Count > 0)
+                    {
+                        throw error("Unknown fields: " + currentReorderBuffer.SavedFields.Keys);
+                    }
+
+                    currentReorderBuffer = reorderBuffers.Pop();
+                }
+
+                // AVRO-2034 advance beyond the end object for the next record.
+                reader.Read();
+            }
+            else
+            {
+                throw new AvroTypeException("Unknown action symbol " + top);
+            }
+
+            return null;
+        }
+
+
+        private class JsonElement
+        {
+            public readonly JsonToken Token;
+            public readonly object Value;

Review Comment:
   `private readonly object value;`
   `public object Value => value;`



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()

Review Comment:
   Use Pascal Casing



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)

Review Comment:
   Does this have to return "this?"  Seems more like it would be void.



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)

Review Comment:
   Should be Pascal Casing for method name.



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);
+                int n = top.FindLabel(label);
+                if (n >= 0)
+                {
+                    reader.Read();
+                    return n;
+                }
+
+                throw new AvroTypeException("Unknown symbol in enum " + label);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayStart()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Read();
+                return doArrayNext();
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doArrayNext();
+        }
+
+        private long doArrayNext()
+        {
+            if (reader.TokenType == JsonToken.EndArray)
+            {
+                Parser.Advance(Symbol.ArrayEnd);
+                reader.Read();
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipArray()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.ArrayEnd);
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapStart()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                return doMapNext();
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doMapNext();
+        }
+
+        private long doMapNext()
+        {
+            if (reader.TokenType == JsonToken.EndObject)
+            {
+                reader.Read();
+                advance(Symbol.MapEnd);
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipMap()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.MapEnd);
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadUnionIndex()
+        {
+            advance(Symbol.Union);
+            Symbol.Alternative a = (Symbol.Alternative)Parser.PopSymbol();
+
+            string label;
+            if (reader.TokenType == JsonToken.Null)
+            {
+                label = "null";
+            }
+            else if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    label = Convert.ToString(reader.Value);
+                    reader.Read();
+                    Parser.PushSymbol(Symbol.UnionEnd);
+                }
+                else
+                {
+                    throw error("start-union");
+                }
+            }
+            else
+            {
+                throw error("start-union");
+            }
+
+            int n = a.FindLabel(label);
+            if (n < 0)
+            {
+                throw new AvroTypeException("Unknown union branch " + label);
+            }
+
+            Parser.PushSymbol(a.GetSymbol(n));
+            return n;
+        }
+
+        /// <inheritdoc />
+        public override void SkipNull()
+        {
+            ReadNull();
+        }
+
+        /// <inheritdoc />
+        public override void SkipBoolean()
+        {
+            ReadBoolean();
+        }
+
+        /// <inheritdoc />
+        public override void SkipInt()
+        {
+            ReadInt();
+        }
+
+        /// <inheritdoc />
+        public override void SkipLong()
+        {
+            ReadLong();
+        }
+
+        /// <inheritdoc />
+        public override void SkipFloat()
+        {
+            ReadFloat();
+        }
+
+        /// <inheritdoc />
+        public override void SkipDouble()
+        {
+            ReadDouble();
+        }
+
+        /// <inheritdoc />
+        public override void SkipEnum()
+        {
+            ReadEnum();
+        }
+
+        /// <inheritdoc />
+        public override void SkipUnionIndex()
+        {
+            ReadUnionIndex();
+        }
+
+        /// <inheritdoc />
+        public override Symbol DoAction(Symbol input, Symbol top)
+        {
+            if (top is Symbol.FieldAdjustAction)
+            {
+                Symbol.FieldAdjustAction fa = (Symbol.FieldAdjustAction)top;
+                string name = fa.FName;
+                if (currentReorderBuffer != null)
+                {
+                    IList<JsonElement> node = currentReorderBuffer.SavedFields[name];
+                    if (node != null)
+                    {
+                        currentReorderBuffer.SavedFields.Remove(name);
+                        currentReorderBuffer.OrigParser = reader;
+                        reader = makeParser(node);
+                        return null;
+                    }
+                }
+
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    do
+                    {
+                        string fn = Convert.ToString(reader.Value);
+                        reader.Read();
+                        if (name.Equals(fn) || (fa.Aliases != null && fa.Aliases.Contains(fn)))
+                        {
+                            return null;
+                        }
+                        else
+                        {
+                            if (currentReorderBuffer == null)
+                            {
+                                currentReorderBuffer = new ReorderBuffer();
+                            }
+
+                            currentReorderBuffer.SavedFields[fn] = getValueAsTree(reader);
+                        }
+                    } while (reader.TokenType == JsonToken.PropertyName);
+
+                    throw new AvroTypeException("Expected field name not found: " + fa.FName);
+                }
+            }
+            else if (top == Symbol.FieldEnd)
+            {
+                if (currentReorderBuffer != null && currentReorderBuffer.OrigParser != null)
+                {
+                    reader = currentReorderBuffer.OrigParser;
+                    currentReorderBuffer.OrigParser = null;
+                }
+            }
+            else if (top == Symbol.RecordStart)
+            {
+                if (reader.TokenType == JsonToken.StartObject)
+                {
+                    reader.Read();
+                    reorderBuffers.Push(currentReorderBuffer);
+                    currentReorderBuffer = null;
+                }
+                else
+                {
+                    throw error("record-start");
+                }
+            }
+            else if (top == Symbol.RecordEnd || top == Symbol.UnionEnd)
+            {
+                // AVRO-2034 advance to the end of our object
+                while (reader.TokenType != JsonToken.EndObject)
+                {
+                    reader.Read();
+                }
+
+                if (top == Symbol.RecordEnd)
+                {
+                    if (currentReorderBuffer != null && currentReorderBuffer.SavedFields.Count > 0)
+                    {
+                        throw error("Unknown fields: " + currentReorderBuffer.SavedFields.Keys);

Review Comment:
   You will need to build this error message out specifically.  This will not ouput as expected.



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)

Review Comment:
   Use Pascal Casing for method name



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)

Review Comment:
   Use Pascal Casing



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);

Review Comment:
   Can you use reader.ReadAsString()?



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);
+                int n = top.FindLabel(label);
+                if (n >= 0)
+                {
+                    reader.Read();
+                    return n;
+                }
+
+                throw new AvroTypeException("Unknown symbol in enum " + label);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayStart()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Read();
+                return doArrayNext();
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doArrayNext();
+        }
+
+        private long doArrayNext()
+        {
+            if (reader.TokenType == JsonToken.EndArray)
+            {
+                Parser.Advance(Symbol.ArrayEnd);
+                reader.Read();
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipArray()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.ArrayEnd);
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapStart()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                return doMapNext();
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doMapNext();
+        }
+
+        private long doMapNext()

Review Comment:
   Use Pascal Casing



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);
+                int n = top.FindLabel(label);
+                if (n >= 0)
+                {
+                    reader.Read();
+                    return n;
+                }
+
+                throw new AvroTypeException("Unknown symbol in enum " + label);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayStart()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Read();
+                return doArrayNext();
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doArrayNext();
+        }
+
+        private long doArrayNext()
+        {
+            if (reader.TokenType == JsonToken.EndArray)
+            {
+                Parser.Advance(Symbol.ArrayEnd);
+                reader.Read();
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipArray()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.ArrayEnd);
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapStart()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                return doMapNext();
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doMapNext();
+        }
+
+        private long doMapNext()
+        {
+            if (reader.TokenType == JsonToken.EndObject)
+            {
+                reader.Read();
+                advance(Symbol.MapEnd);
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipMap()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.MapEnd);
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadUnionIndex()
+        {
+            advance(Symbol.Union);
+            Symbol.Alternative a = (Symbol.Alternative)Parser.PopSymbol();
+
+            string label;
+            if (reader.TokenType == JsonToken.Null)
+            {
+                label = "null";
+            }
+            else if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    label = Convert.ToString(reader.Value);

Review Comment:
   Can you use reader.ReadAsString()?



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);
+                int n = top.FindLabel(label);
+                if (n >= 0)
+                {
+                    reader.Read();
+                    return n;
+                }
+
+                throw new AvroTypeException("Unknown symbol in enum " + label);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayStart()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Read();
+                return doArrayNext();
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doArrayNext();
+        }
+
+        private long doArrayNext()

Review Comment:
   Use Pascal Casing



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);
+                int n = top.FindLabel(label);
+                if (n >= 0)
+                {
+                    reader.Read();
+                    return n;
+                }
+
+                throw new AvroTypeException("Unknown symbol in enum " + label);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayStart()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Read();
+                return doArrayNext();
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doArrayNext();
+        }
+
+        private long doArrayNext()
+        {
+            if (reader.TokenType == JsonToken.EndArray)
+            {
+                Parser.Advance(Symbol.ArrayEnd);
+                reader.Read();
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipArray()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.ArrayEnd);
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapStart()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                return doMapNext();
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doMapNext();
+        }
+
+        private long doMapNext()
+        {
+            if (reader.TokenType == JsonToken.EndObject)
+            {
+                reader.Read();
+                advance(Symbol.MapEnd);
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipMap()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.MapEnd);
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadUnionIndex()
+        {
+            advance(Symbol.Union);
+            Symbol.Alternative a = (Symbol.Alternative)Parser.PopSymbol();
+
+            string label;
+            if (reader.TokenType == JsonToken.Null)
+            {
+                label = "null";
+            }
+            else if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    label = Convert.ToString(reader.Value);
+                    reader.Read();
+                    Parser.PushSymbol(Symbol.UnionEnd);
+                }
+                else
+                {
+                    throw error("start-union");
+                }
+            }
+            else
+            {
+                throw error("start-union");
+            }
+
+            int n = a.FindLabel(label);
+            if (n < 0)
+            {
+                throw new AvroTypeException("Unknown union branch " + label);
+            }
+
+            Parser.PushSymbol(a.GetSymbol(n));
+            return n;
+        }
+
+        /// <inheritdoc />
+        public override void SkipNull()
+        {
+            ReadNull();
+        }
+
+        /// <inheritdoc />
+        public override void SkipBoolean()
+        {
+            ReadBoolean();
+        }
+
+        /// <inheritdoc />
+        public override void SkipInt()
+        {
+            ReadInt();
+        }
+
+        /// <inheritdoc />
+        public override void SkipLong()
+        {
+            ReadLong();
+        }
+
+        /// <inheritdoc />
+        public override void SkipFloat()
+        {
+            ReadFloat();
+        }
+
+        /// <inheritdoc />
+        public override void SkipDouble()
+        {
+            ReadDouble();
+        }
+
+        /// <inheritdoc />
+        public override void SkipEnum()
+        {
+            ReadEnum();
+        }
+
+        /// <inheritdoc />
+        public override void SkipUnionIndex()
+        {
+            ReadUnionIndex();
+        }
+
+        /// <inheritdoc />
+        public override Symbol DoAction(Symbol input, Symbol top)
+        {
+            if (top is Symbol.FieldAdjustAction)
+            {
+                Symbol.FieldAdjustAction fa = (Symbol.FieldAdjustAction)top;
+                string name = fa.FName;
+                if (currentReorderBuffer != null)
+                {
+                    IList<JsonElement> node = currentReorderBuffer.SavedFields[name];
+                    if (node != null)
+                    {
+                        currentReorderBuffer.SavedFields.Remove(name);
+                        currentReorderBuffer.OrigParser = reader;
+                        reader = makeParser(node);
+                        return null;
+                    }
+                }
+
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    do
+                    {
+                        string fn = Convert.ToString(reader.Value);

Review Comment:
   Can you use reader.ReadAsString()?



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);
+                int n = top.FindLabel(label);
+                if (n >= 0)
+                {
+                    reader.Read();
+                    return n;
+                }
+
+                throw new AvroTypeException("Unknown symbol in enum " + label);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayStart()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Read();
+                return doArrayNext();
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doArrayNext();
+        }
+
+        private long doArrayNext()
+        {
+            if (reader.TokenType == JsonToken.EndArray)
+            {
+                Parser.Advance(Symbol.ArrayEnd);
+                reader.Read();
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipArray()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.ArrayEnd);
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapStart()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                return doMapNext();
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doMapNext();
+        }
+
+        private long doMapNext()
+        {
+            if (reader.TokenType == JsonToken.EndObject)
+            {
+                reader.Read();
+                advance(Symbol.MapEnd);
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipMap()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.MapEnd);
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadUnionIndex()
+        {
+            advance(Symbol.Union);
+            Symbol.Alternative a = (Symbol.Alternative)Parser.PopSymbol();
+
+            string label;
+            if (reader.TokenType == JsonToken.Null)
+            {
+                label = "null";
+            }
+            else if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    label = Convert.ToString(reader.Value);
+                    reader.Read();
+                    Parser.PushSymbol(Symbol.UnionEnd);
+                }
+                else
+                {
+                    throw error("start-union");
+                }
+            }
+            else
+            {
+                throw error("start-union");
+            }
+
+            int n = a.FindLabel(label);
+            if (n < 0)
+            {
+                throw new AvroTypeException("Unknown union branch " + label);
+            }
+
+            Parser.PushSymbol(a.GetSymbol(n));
+            return n;
+        }
+
+        /// <inheritdoc />
+        public override void SkipNull()
+        {
+            ReadNull();
+        }
+
+        /// <inheritdoc />
+        public override void SkipBoolean()
+        {
+            ReadBoolean();
+        }
+
+        /// <inheritdoc />
+        public override void SkipInt()
+        {
+            ReadInt();
+        }
+
+        /// <inheritdoc />
+        public override void SkipLong()
+        {
+            ReadLong();
+        }
+
+        /// <inheritdoc />
+        public override void SkipFloat()
+        {
+            ReadFloat();
+        }
+
+        /// <inheritdoc />
+        public override void SkipDouble()
+        {
+            ReadDouble();
+        }
+
+        /// <inheritdoc />
+        public override void SkipEnum()
+        {
+            ReadEnum();
+        }
+
+        /// <inheritdoc />
+        public override void SkipUnionIndex()
+        {
+            ReadUnionIndex();
+        }
+
+        /// <inheritdoc />
+        public override Symbol DoAction(Symbol input, Symbol top)
+        {
+            if (top is Symbol.FieldAdjustAction)
+            {
+                Symbol.FieldAdjustAction fa = (Symbol.FieldAdjustAction)top;
+                string name = fa.FName;
+                if (currentReorderBuffer != null)
+                {
+                    IList<JsonElement> node = currentReorderBuffer.SavedFields[name];
+                    if (node != null)
+                    {
+                        currentReorderBuffer.SavedFields.Remove(name);
+                        currentReorderBuffer.OrigParser = reader;
+                        reader = makeParser(node);
+                        return null;
+                    }
+                }
+
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    do
+                    {
+                        string fn = Convert.ToString(reader.Value);
+                        reader.Read();
+                        if (name.Equals(fn) || (fa.Aliases != null && fa.Aliases.Contains(fn)))
+                        {
+                            return null;
+                        }
+                        else
+                        {
+                            if (currentReorderBuffer == null)
+                            {
+                                currentReorderBuffer = new ReorderBuffer();
+                            }
+
+                            currentReorderBuffer.SavedFields[fn] = getValueAsTree(reader);
+                        }
+                    } while (reader.TokenType == JsonToken.PropertyName);
+
+                    throw new AvroTypeException("Expected field name not found: " + fa.FName);
+                }
+            }
+            else if (top == Symbol.FieldEnd)
+            {
+                if (currentReorderBuffer != null && currentReorderBuffer.OrigParser != null)
+                {
+                    reader = currentReorderBuffer.OrigParser;
+                    currentReorderBuffer.OrigParser = null;
+                }
+            }
+            else if (top == Symbol.RecordStart)
+            {
+                if (reader.TokenType == JsonToken.StartObject)
+                {
+                    reader.Read();
+                    reorderBuffers.Push(currentReorderBuffer);
+                    currentReorderBuffer = null;
+                }
+                else
+                {
+                    throw error("record-start");
+                }
+            }
+            else if (top == Symbol.RecordEnd || top == Symbol.UnionEnd)
+            {
+                // AVRO-2034 advance to the end of our object
+                while (reader.TokenType != JsonToken.EndObject)
+                {
+                    reader.Read();
+                }
+
+                if (top == Symbol.RecordEnd)
+                {
+                    if (currentReorderBuffer != null && currentReorderBuffer.SavedFields.Count > 0)
+                    {
+                        throw error("Unknown fields: " + currentReorderBuffer.SavedFields.Keys);
+                    }
+
+                    currentReorderBuffer = reorderBuffers.Pop();
+                }
+
+                // AVRO-2034 advance beyond the end object for the next record.
+                reader.Read();
+            }
+            else
+            {
+                throw new AvroTypeException("Unknown action symbol " + top);
+            }
+
+            return null;
+        }
+
+
+        private class JsonElement
+        {
+            public readonly JsonToken Token;

Review Comment:
   `private readonly JsonToken token;`
   `public JsonToken Token => token;`



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);
+                int n = top.FindLabel(label);
+                if (n >= 0)
+                {
+                    reader.Read();
+                    return n;
+                }
+
+                throw new AvroTypeException("Unknown symbol in enum " + label);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayStart()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Read();
+                return doArrayNext();
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doArrayNext();
+        }
+
+        private long doArrayNext()
+        {
+            if (reader.TokenType == JsonToken.EndArray)
+            {
+                Parser.Advance(Symbol.ArrayEnd);
+                reader.Read();
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipArray()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.ArrayEnd);
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapStart()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                return doMapNext();
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doMapNext();
+        }
+
+        private long doMapNext()
+        {
+            if (reader.TokenType == JsonToken.EndObject)
+            {
+                reader.Read();
+                advance(Symbol.MapEnd);
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipMap()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.MapEnd);
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadUnionIndex()
+        {
+            advance(Symbol.Union);
+            Symbol.Alternative a = (Symbol.Alternative)Parser.PopSymbol();
+
+            string label;
+            if (reader.TokenType == JsonToken.Null)
+            {
+                label = "null";
+            }
+            else if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    label = Convert.ToString(reader.Value);
+                    reader.Read();
+                    Parser.PushSymbol(Symbol.UnionEnd);
+                }
+                else
+                {
+                    throw error("start-union");
+                }
+            }
+            else
+            {
+                throw error("start-union");
+            }
+
+            int n = a.FindLabel(label);
+            if (n < 0)
+            {
+                throw new AvroTypeException("Unknown union branch " + label);
+            }
+
+            Parser.PushSymbol(a.GetSymbol(n));
+            return n;
+        }
+
+        /// <inheritdoc />
+        public override void SkipNull()
+        {
+            ReadNull();
+        }
+
+        /// <inheritdoc />
+        public override void SkipBoolean()
+        {
+            ReadBoolean();
+        }
+
+        /// <inheritdoc />
+        public override void SkipInt()
+        {
+            ReadInt();
+        }
+
+        /// <inheritdoc />
+        public override void SkipLong()
+        {
+            ReadLong();
+        }
+
+        /// <inheritdoc />
+        public override void SkipFloat()
+        {
+            ReadFloat();
+        }
+
+        /// <inheritdoc />
+        public override void SkipDouble()
+        {
+            ReadDouble();
+        }
+
+        /// <inheritdoc />
+        public override void SkipEnum()
+        {
+            ReadEnum();
+        }
+
+        /// <inheritdoc />
+        public override void SkipUnionIndex()
+        {
+            ReadUnionIndex();
+        }
+
+        /// <inheritdoc />
+        public override Symbol DoAction(Symbol input, Symbol top)
+        {
+            if (top is Symbol.FieldAdjustAction)
+            {
+                Symbol.FieldAdjustAction fa = (Symbol.FieldAdjustAction)top;
+                string name = fa.FName;
+                if (currentReorderBuffer != null)
+                {
+                    IList<JsonElement> node = currentReorderBuffer.SavedFields[name];
+                    if (node != null)
+                    {
+                        currentReorderBuffer.SavedFields.Remove(name);
+                        currentReorderBuffer.OrigParser = reader;
+                        reader = makeParser(node);
+                        return null;
+                    }
+                }
+
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    do
+                    {
+                        string fn = Convert.ToString(reader.Value);
+                        reader.Read();
+                        if (name.Equals(fn) || (fa.Aliases != null && fa.Aliases.Contains(fn)))
+                        {
+                            return null;
+                        }
+                        else
+                        {
+                            if (currentReorderBuffer == null)
+                            {
+                                currentReorderBuffer = new ReorderBuffer();
+                            }
+
+                            currentReorderBuffer.SavedFields[fn] = getValueAsTree(reader);
+                        }
+                    } while (reader.TokenType == JsonToken.PropertyName);
+
+                    throw new AvroTypeException("Expected field name not found: " + fa.FName);
+                }
+            }
+            else if (top == Symbol.FieldEnd)
+            {
+                if (currentReorderBuffer != null && currentReorderBuffer.OrigParser != null)
+                {
+                    reader = currentReorderBuffer.OrigParser;
+                    currentReorderBuffer.OrigParser = null;
+                }
+            }
+            else if (top == Symbol.RecordStart)
+            {
+                if (reader.TokenType == JsonToken.StartObject)
+                {
+                    reader.Read();
+                    reorderBuffers.Push(currentReorderBuffer);
+                    currentReorderBuffer = null;
+                }
+                else
+                {
+                    throw error("record-start");
+                }
+            }
+            else if (top == Symbol.RecordEnd || top == Symbol.UnionEnd)
+            {
+                // AVRO-2034 advance to the end of our object
+                while (reader.TokenType != JsonToken.EndObject)
+                {
+                    reader.Read();
+                }
+
+                if (top == Symbol.RecordEnd)
+                {
+                    if (currentReorderBuffer != null && currentReorderBuffer.SavedFields.Count > 0)
+                    {
+                        throw error("Unknown fields: " + currentReorderBuffer.SavedFields.Keys);
+                    }
+
+                    currentReorderBuffer = reorderBuffers.Pop();
+                }
+
+                // AVRO-2034 advance beyond the end object for the next record.
+                reader.Read();
+            }
+            else
+            {
+                throw new AvroTypeException("Unknown action symbol " + top);
+            }
+
+            return null;
+        }
+
+
+        private class JsonElement
+        {
+            public readonly JsonToken Token;
+            public readonly object Value;
+
+            public JsonElement(JsonToken t, object value)
+            {
+                this.Token = t;
+                this.Value = value;
+            }
+
+            public JsonElement(JsonToken t) : this(t, null)
+            {
+            }
+        }
+
+        private static IList<JsonElement> getValueAsTree(JsonReader reader)
+        {
+            int level = 0;
+            IList<JsonElement> result = new List<JsonElement>();
+            do
+            {
+                JsonToken t = reader.TokenType;
+                switch (t)
+                {
+                    case JsonToken.StartObject:
+                    case JsonToken.StartArray:
+                        level++;
+                        result.Add(new JsonElement(t));
+                        break;
+                    case JsonToken.EndObject:
+                    case JsonToken.EndArray:
+                        level--;
+                        result.Add(new JsonElement(t));
+                        break;
+                    case JsonToken.PropertyName:
+                    case JsonToken.String:
+                    case JsonToken.Integer:
+                    case JsonToken.Float:
+                    case JsonToken.Boolean:
+                    case JsonToken.Null:
+                        result.Add(new JsonElement(t, reader.Value));
+                        break;
+                }
+
+                reader.Read();
+            } while (level != 0);
+
+            result.Add(new JsonElement(JsonToken.None));
+            return result;
+        }
+
+        private JsonReader makeParser(in IList<JsonElement> elements)

Review Comment:
   Use Pascal Casing



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);
+                int n = top.FindLabel(label);
+                if (n >= 0)
+                {
+                    reader.Read();
+                    return n;
+                }
+
+                throw new AvroTypeException("Unknown symbol in enum " + label);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayStart()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Read();
+                return doArrayNext();
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doArrayNext();
+        }
+
+        private long doArrayNext()
+        {
+            if (reader.TokenType == JsonToken.EndArray)
+            {
+                Parser.Advance(Symbol.ArrayEnd);
+                reader.Read();
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipArray()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.ArrayEnd);
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapStart()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                return doMapNext();
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doMapNext();
+        }
+
+        private long doMapNext()
+        {
+            if (reader.TokenType == JsonToken.EndObject)
+            {
+                reader.Read();
+                advance(Symbol.MapEnd);
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipMap()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.MapEnd);
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadUnionIndex()
+        {
+            advance(Symbol.Union);
+            Symbol.Alternative a = (Symbol.Alternative)Parser.PopSymbol();
+
+            string label;
+            if (reader.TokenType == JsonToken.Null)
+            {
+                label = "null";
+            }
+            else if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    label = Convert.ToString(reader.Value);
+                    reader.Read();
+                    Parser.PushSymbol(Symbol.UnionEnd);
+                }
+                else
+                {
+                    throw error("start-union");
+                }
+            }
+            else
+            {
+                throw error("start-union");
+            }
+
+            int n = a.FindLabel(label);
+            if (n < 0)
+            {
+                throw new AvroTypeException("Unknown union branch " + label);
+            }
+
+            Parser.PushSymbol(a.GetSymbol(n));
+            return n;
+        }
+
+        /// <inheritdoc />
+        public override void SkipNull()
+        {
+            ReadNull();
+        }
+
+        /// <inheritdoc />
+        public override void SkipBoolean()
+        {
+            ReadBoolean();
+        }
+
+        /// <inheritdoc />
+        public override void SkipInt()
+        {
+            ReadInt();
+        }
+
+        /// <inheritdoc />
+        public override void SkipLong()
+        {
+            ReadLong();
+        }
+
+        /// <inheritdoc />
+        public override void SkipFloat()
+        {
+            ReadFloat();
+        }
+
+        /// <inheritdoc />
+        public override void SkipDouble()
+        {
+            ReadDouble();
+        }
+
+        /// <inheritdoc />
+        public override void SkipEnum()
+        {
+            ReadEnum();
+        }
+
+        /// <inheritdoc />
+        public override void SkipUnionIndex()
+        {
+            ReadUnionIndex();
+        }
+
+        /// <inheritdoc />
+        public override Symbol DoAction(Symbol input, Symbol top)
+        {
+            if (top is Symbol.FieldAdjustAction)
+            {
+                Symbol.FieldAdjustAction fa = (Symbol.FieldAdjustAction)top;
+                string name = fa.FName;
+                if (currentReorderBuffer != null)
+                {
+                    IList<JsonElement> node = currentReorderBuffer.SavedFields[name];
+                    if (node != null)
+                    {
+                        currentReorderBuffer.SavedFields.Remove(name);
+                        currentReorderBuffer.OrigParser = reader;
+                        reader = makeParser(node);
+                        return null;
+                    }
+                }
+
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    do
+                    {
+                        string fn = Convert.ToString(reader.Value);
+                        reader.Read();
+                        if (name.Equals(fn) || (fa.Aliases != null && fa.Aliases.Contains(fn)))
+                        {
+                            return null;
+                        }
+                        else
+                        {
+                            if (currentReorderBuffer == null)
+                            {
+                                currentReorderBuffer = new ReorderBuffer();
+                            }
+
+                            currentReorderBuffer.SavedFields[fn] = getValueAsTree(reader);
+                        }
+                    } while (reader.TokenType == JsonToken.PropertyName);
+
+                    throw new AvroTypeException("Expected field name not found: " + fa.FName);
+                }
+            }
+            else if (top == Symbol.FieldEnd)
+            {
+                if (currentReorderBuffer != null && currentReorderBuffer.OrigParser != null)
+                {
+                    reader = currentReorderBuffer.OrigParser;
+                    currentReorderBuffer.OrigParser = null;
+                }
+            }
+            else if (top == Symbol.RecordStart)
+            {
+                if (reader.TokenType == JsonToken.StartObject)
+                {
+                    reader.Read();
+                    reorderBuffers.Push(currentReorderBuffer);
+                    currentReorderBuffer = null;
+                }
+                else
+                {
+                    throw error("record-start");
+                }
+            }
+            else if (top == Symbol.RecordEnd || top == Symbol.UnionEnd)
+            {
+                // AVRO-2034 advance to the end of our object
+                while (reader.TokenType != JsonToken.EndObject)
+                {
+                    reader.Read();
+                }
+
+                if (top == Symbol.RecordEnd)
+                {
+                    if (currentReorderBuffer != null && currentReorderBuffer.SavedFields.Count > 0)
+                    {
+                        throw error("Unknown fields: " + currentReorderBuffer.SavedFields.Keys);
+                    }
+
+                    currentReorderBuffer = reorderBuffers.Pop();
+                }
+
+                // AVRO-2034 advance beyond the end object for the next record.
+                reader.Read();
+            }
+            else
+            {
+                throw new AvroTypeException("Unknown action symbol " + top);
+            }
+
+            return null;
+        }
+
+
+        private class JsonElement
+        {
+            public readonly JsonToken Token;
+            public readonly object Value;
+
+            public JsonElement(JsonToken t, object value)
+            {
+                this.Token = t;
+                this.Value = value;
+            }
+
+            public JsonElement(JsonToken t) : this(t, null)
+            {
+            }
+        }
+
+        private static IList<JsonElement> getValueAsTree(JsonReader reader)
+        {
+            int level = 0;
+            IList<JsonElement> result = new List<JsonElement>();
+            do
+            {
+                JsonToken t = reader.TokenType;
+                switch (t)
+                {
+                    case JsonToken.StartObject:
+                    case JsonToken.StartArray:
+                        level++;
+                        result.Add(new JsonElement(t));
+                        break;
+                    case JsonToken.EndObject:
+                    case JsonToken.EndArray:
+                        level--;
+                        result.Add(new JsonElement(t));
+                        break;
+                    case JsonToken.PropertyName:
+                    case JsonToken.String:
+                    case JsonToken.Integer:
+                    case JsonToken.Float:
+                    case JsonToken.Boolean:
+                    case JsonToken.Null:
+                        result.Add(new JsonElement(t, reader.Value));
+                        break;
+                }
+
+                reader.Read();
+            } while (level != 0);
+
+            result.Add(new JsonElement(JsonToken.None));
+            return result;
+        }
+
+        private JsonReader makeParser(in IList<JsonElement> elements)
+        {
+            return new JsonElementReader(elements);
+        }
+
+        private class JsonElementReader : JsonReader
+        {
+            private readonly IList<JsonElement> elements;
+
+            public JsonElementReader(IList<JsonElement> elements)
+            {
+                this.elements = elements;
+                pos = 0;
+            }
+
+            private int pos;
+
+            public override object Value
+            {
+                get { return elements[pos].Value; }
+            }
+
+            public override JsonToken TokenType
+            {
+                get { return elements[pos].Token; }
+            }
+
+            public override bool Read()
+            {
+                pos++;
+                return true;
+            }
+        }
+
+        private AvroTypeException error(string type)

Review Comment:
   Use Pascal Casing, and maybe rename to TypeError



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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r955266867


##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;

Review Comment:
   Note it is in a private class.
   
   If you want to make it a property anyway, I think `public JsonReader OrigParser { get; set; }` would be simpler.



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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r954589056


##########
lang/csharp/src/apache/main/IO/Encoder.cs:
##########
@@ -187,5 +187,10 @@ public interface Encoder
         /// <param name="start">Position within data where the contents start.</param>
         /// <param name="len">Number of bytes to write.</param>
         void WriteFixed(byte[] data, int start, int len);
+
+        /// <summary>
+        /// Flushes the encoder.
+        /// </summary>
+        void Flush();

Review Comment:
   Thanks @KalleOlaviNiemitalo , would you prefer that I remove 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.

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

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


[GitHub] [avro] martin-g commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
martin-g commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1228404830

   @KhrystynaPopadyuk do want to join the reviewing process here ?
   
   It would be really nice if C# contributors help each other for the common good!
   @rayokota You could return the favour by reviewing others' PRs - https://github.com/apache/avro/pulls?q=is%3Aopen+is%3Apr+label%3AC%23


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


[GitHub] [avro] martin-g commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
martin-g commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r958272425


##########
lang/csharp/src/apache/main/IO/Encoder.cs:
##########
@@ -187,5 +187,10 @@ public interface Encoder
         /// <param name="start">Position within data where the contents start.</param>
         /// <param name="len">Number of bytes to write.</param>
         void WriteFixed(byte[] data, int start, int len);
+
+        /// <summary>
+        /// Flushes the encoder.
+        /// </summary>
+        void Flush();

Review Comment:
   So, this PR could be merged as is only in 1.12.0.
   For branch-1.11 we should cherry-pick the squashed commit from `master`, remove this method and add an extra commit, right ?



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


[GitHub] [avro] KalleOlaviNiemitalo commented on pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1221930980

   Because the C# logical type implementation is different from that of Java, I'd like to see an extra test for JSON encoding and decoding of a union with members having logical types.  For example,
   
   ```JSON
   [
       "null",
       { "type": "int", "logicalType": "date" },
       { "type": "long", "logicalType": "timestamp-millis" }
   ]
   ```
   
   AFAIK, this should write "null" or "int" or "long" as the property name in the union object in JSON, and should be able to parse each of those back.


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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r951080180


##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,787 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <seealso cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// <p/>
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>

Review Comment:
   Fixed



##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,787 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <seealso cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// <p/>
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// <p/>
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);
+                int n = top.FindLabel(label);
+                if (n >= 0)
+                {
+                    reader.Read();
+                    return n;
+                }
+
+                throw new AvroTypeException("Unknown symbol in enum " + label);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayStart()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Read();
+                return doArrayNext();
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doArrayNext();
+        }
+
+        private long doArrayNext()
+        {
+            if (reader.TokenType == JsonToken.EndArray)
+            {
+                Parser.Advance(Symbol.ArrayEnd);
+                reader.Read();
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipArray()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.ArrayEnd);
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapStart()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                return doMapNext();
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doMapNext();
+        }
+
+        private long doMapNext()
+        {
+            if (reader.TokenType == JsonToken.EndObject)
+            {
+                reader.Read();
+                advance(Symbol.MapEnd);
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipMap()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.MapEnd);
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadUnionIndex()
+        {
+            advance(Symbol.Union);
+            Symbol.Alternative a = (Symbol.Alternative)Parser.PopSymbol();
+
+            string label;
+            if (reader.TokenType == JsonToken.Null)
+            {
+                label = "null";
+            }
+            else if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    label = Convert.ToString(reader.Value);
+                    reader.Read();
+                    Parser.PushSymbol(Symbol.UnionEnd);
+                }
+                else
+                {
+                    throw error("start-union");
+                }
+            }
+            else
+            {
+                throw error("start-union");
+            }
+
+            int n = a.FindLabel(label);
+            if (n < 0)
+            {
+                throw new AvroTypeException("Unknown union branch " + label);
+            }
+
+            Parser.PushSymbol(a.GetSymbol(n));
+            return n;
+        }
+
+        /// <inheritdoc />
+        public override void SkipNull()
+        {
+            ReadNull();
+        }
+
+        /// <inheritdoc />
+        public override void SkipBoolean()
+        {
+            ReadBoolean();
+        }
+
+        /// <inheritdoc />
+        public override void SkipInt()
+        {
+            ReadInt();
+        }
+
+        /// <inheritdoc />
+        public override void SkipLong()
+        {
+            ReadLong();
+        }
+
+        /// <inheritdoc />
+        public override void SkipFloat()
+        {
+            ReadFloat();
+        }
+
+        /// <inheritdoc />
+        public override void SkipDouble()
+        {
+            ReadDouble();
+        }
+
+        /// <inheritdoc />
+        public override void SkipEnum()
+        {
+            ReadEnum();
+        }
+
+        /// <inheritdoc />
+        public override void SkipUnionIndex()
+        {
+            ReadUnionIndex();
+        }
+
+        /// <inheritdoc />
+        public override Symbol DoAction(Symbol input, Symbol top)
+        {
+            if (top is Symbol.FieldAdjustAction)
+            {
+                Symbol.FieldAdjustAction fa = (Symbol.FieldAdjustAction)top;
+                string name = fa.FName;
+                if (currentReorderBuffer != null)
+                {
+                    IList<JsonElement> node = currentReorderBuffer.SavedFields[name];
+                    if (node != null)
+                    {
+                        currentReorderBuffer.SavedFields.Remove(name);
+                        currentReorderBuffer.OrigParser = reader;
+                        reader = makeParser(node);
+                        return null;
+                    }
+                }
+
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    do
+                    {
+                        string fn = Convert.ToString(reader.Value);
+                        reader.Read();
+                        if (name.Equals(fn) || (fa.Aliases != null && fa.Aliases.Contains(fn)))
+                        {
+                            return null;
+                        }
+                        else
+                        {
+                            if (currentReorderBuffer == null)
+                            {
+                                currentReorderBuffer = new ReorderBuffer();
+                            }
+
+                            currentReorderBuffer.SavedFields[fn] = getValueAsTree(reader);
+                        }
+                    } while (reader.TokenType == JsonToken.PropertyName);
+
+                    throw new AvroTypeException("Expected field name not found: " + fa.FName);
+                }
+            }
+            else if (top == Symbol.FieldEnd)
+            {
+                if (currentReorderBuffer != null && currentReorderBuffer.OrigParser != null)
+                {
+                    reader = currentReorderBuffer.OrigParser;
+                    currentReorderBuffer.OrigParser = null;
+                }
+            }
+            else if (top == Symbol.RecordStart)
+            {
+                if (reader.TokenType == JsonToken.StartObject)
+                {
+                    reader.Read();
+                    reorderBuffers.Push(currentReorderBuffer);
+                    currentReorderBuffer = null;
+                }
+                else
+                {
+                    throw error("record-start");
+                }
+            }
+            else if (top == Symbol.RecordEnd || top == Symbol.UnionEnd)
+            {
+                // AVRO-2034 advance to the end of our object
+                while (reader.TokenType != JsonToken.EndObject)
+                {
+                    reader.Read();
+                }
+
+                if (top == Symbol.RecordEnd)
+                {
+                    if (currentReorderBuffer != null && currentReorderBuffer.SavedFields.Count > 0)
+                    {
+                        throw error("Unknown fields: " + currentReorderBuffer.SavedFields.Keys);
+                    }
+
+                    currentReorderBuffer = reorderBuffers.Pop();
+                }
+
+                // AVRO-2034 advance beyond the end object for the next record.
+                reader.Read();
+            }
+            else
+            {
+                throw new AvroTypeException("Unknown action symbol " + top);
+            }
+
+            return null;
+        }
+
+
+        private class JsonElement
+        {
+            public readonly JsonToken Token;
+            public readonly object Value;
+
+            public JsonElement(JsonToken t, object value)
+            {
+                this.Token = t;
+                this.Value = value;
+            }
+
+            public JsonElement(JsonToken t) : this(t, null)
+            {
+            }
+        }
+
+        private static IList<JsonElement> getValueAsTree(JsonReader reader)
+        {
+            int level = 0;
+            IList<JsonElement> result = new List<JsonElement>();
+            do
+            {
+                JsonToken t = reader.TokenType;
+                switch (t)
+                {
+                    case JsonToken.StartObject:
+                    case JsonToken.StartArray:
+                        level++;
+                        result.Add(new JsonElement(t));
+                        break;
+                    case JsonToken.EndObject:
+                    case JsonToken.EndArray:
+                        level--;
+                        result.Add(new JsonElement(t));
+                        break;
+                    case JsonToken.PropertyName:
+                    case JsonToken.String:
+                    case JsonToken.Integer:
+                    case JsonToken.Float:
+                    case JsonToken.Boolean:
+                    case JsonToken.Null:
+                        result.Add(new JsonElement(t, reader.Value));
+                        break;
+                }
+
+                reader.Read();
+            } while (level != 0);
+
+            result.Add(new JsonElement(JsonToken.None));
+            return result;
+        }
+
+        private JsonReader makeParser(in IList<JsonElement> elements)
+        {
+            return new JsonElementReader(elements);
+        }
+
+        private class JsonElementReader : JsonReader
+        {
+            private readonly IList<JsonElement> elements;
+
+            public JsonElementReader(IList<JsonElement> elements)
+            {
+                this.elements = elements;
+                pos = 0;
+            }
+
+            private int pos;
+
+            public override object Value
+            {
+                get { return elements[pos].Value; }
+            }
+
+            public override JsonToken TokenType
+            {
+                get { return elements[pos].Token; }
+            }
+
+            public override bool Read()
+            {
+                pos++;
+                return true;
+            }
+
+            public new void Skip()

Review Comment:
   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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r951082143


##########
lang/csharp/src/apache/main/IO/JsonEncoder.cs:
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.
+ */
+
+using Avro.IO.Parsing;
+using System.Collections;
+using System.IO;
+using System.Text;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// An <seealso cref="Encoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonEncoder buffers output, and data may not appear on the output until
+    /// <seealso cref="Encoder.Flush()"/> is called.
+    ///
+    /// JsonEncoder is not thread-safe.
+    /// </summary>
+    public class JsonEncoder : ParsingEncoder, Parser.ActionHandler
+    {
+        private readonly Parser parser;
+        private JsonWriter writer;
+        private bool includeNamespace = true;
+
+        // Has anything been written into the collections?
+        private readonly BitArray isEmpty = new BitArray(64);
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream) : this(sc, getJsonWriter(stream, false))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream, bool pretty) : this(sc, getJsonWriter(stream, pretty))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, JsonWriter writer)
+        {
+            Configure(writer);
+            this.parser = new Parser((new JsonGrammarGenerator()).Generate(sc), this);
+        }
+
+        /// <inheritdoc />
+        public override void Flush()
+        {
+            parser.ProcessImplicitActions();
+            if (writer != null)
+            {
+                writer.Flush();
+            }
+        }
+
+        // by default, one object per line.
+        // with pretty option use default pretty printer with root line separator.
+        private static JsonWriter getJsonWriter(Stream stream, bool pretty)
+        {
+            JsonWriter writer = new JsonTextWriter(new StreamWriter(stream));
+            if (pretty)
+            {
+                writer.Formatting = Formatting.Indented;
+            }
+
+            return writer;
+        }
+
+        /// <summary>
+        /// Whether to include the namespace.
+        /// </summary>
+        public virtual bool IncludeNamespace
+        {
+            get { return includeNamespace; }
+            set { this.includeNamespace = value; }
+        }
+
+
+        /// <summary>
+        /// Reconfigures this JsonEncoder to use the output stream provided.
+        /// <p/>
+        /// Otherwise, this JsonEncoder will flush its current output and then
+        /// reconfigure its output to use a default UTF8 JsonWriter that writes to the
+        /// provided OutputStream.
+        /// </summary>
+        /// <param name="stream"> The OutputStream to direct output to. Cannot be null. </param>
+        /// <returns> this JsonEncoder </returns>
+        public JsonEncoder Configure(Stream stream)
+        {
+            this.Configure(getJsonWriter(stream, false));
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonEncoder to output to the JsonWriter provided.
+        /// <p/>
+        /// Otherwise, this JsonEncoder will flush its current output and then
+        /// reconfigure its output to use the provided JsonWriter.
+        /// </summary>
+        /// <param name="jsonWriter"> The JsonWriter to direct output to. Cannot be null. </param>
+        /// <returns> this JsonEncoder </returns>
+        public JsonEncoder Configure(JsonWriter jsonWriter)
+        {
+            if (null != parser)
+            {
+                Flush();
+            }
+
+            this.writer = jsonWriter;
+            return this;
+        }
+
+        /// <inheritdoc />
+        public override void WriteNull()
+        {
+            parser.Advance(Symbol.Null);
+            writer.WriteNull();
+        }
+
+        /// <inheritdoc />
+        public override void WriteBoolean(bool b)
+        {
+            parser.Advance(Symbol.Boolean);
+            writer.WriteValue(b);
+        }
+
+        /// <inheritdoc />
+        public override void WriteInt(int n)
+        {
+            parser.Advance(Symbol.Int);
+            writer.WriteValue(n);
+        }
+
+        /// <inheritdoc />
+        public override void WriteLong(long n)
+        {
+            parser.Advance(Symbol.Long);
+            writer.WriteValue(n);
+        }
+
+        /// <inheritdoc />
+        public override void WriteFloat(float f)
+        {
+            parser.Advance(Symbol.Float);
+            writer.WriteValue(f);
+        }
+
+        /// <inheritdoc />
+        public override void WriteDouble(double d)
+        {
+            parser.Advance(Symbol.Double);
+            writer.WriteValue(d);
+        }
+
+        /// <inheritdoc />
+        public override void WriteString(string str)
+        {
+            parser.Advance(Symbol.String);
+            if (parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                parser.Advance(Symbol.MapKeyMarker);
+                writer.WritePropertyName(str);
+            }
+            else
+            {
+                writer.WriteValue(str);
+            }
+        }
+
+        /// <inheritdoc />
+        public override void WriteBytes(byte[] bytes)
+        {
+            WriteBytes(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void WriteBytes(byte[] bytes, int start, int len)
+        {
+            parser.Advance(Symbol.Bytes);
+            writeByteArray(bytes, start, len);
+        }
+
+        private void writeByteArray(byte[] bytes, int start, int len)
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            writer.WriteValue(iso.GetString(bytes, start, len));
+        }
+
+        /// <inheritdoc />
+        public override void WriteFixed(byte[] bytes)
+        {
+            WriteFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void WriteFixed(byte[] bytes, int start, int len)
+        {
+            parser.Advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)parser.PopSymbol();
+            if (len != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + len + " bytes.");
+            }
+
+            writeByteArray(bytes, start, len);
+        }
+
+        /// <inheritdoc />
+        public override void WriteEnum(int e)
+        {
+            parser.Advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)parser.PopSymbol();
+            if (e < 0 || e >= top.Size)
+            {
+                throw new AvroTypeException("Enumeration out of range: max is " + top.Size + " but received " + e);
+            }
+
+            writer.WriteValue(top.GetLabel(e));
+        }
+
+        /// <inheritdoc />
+        public override void WriteArrayStart()
+        {
+            parser.Advance(Symbol.ArrayStart);
+            writer.WriteStartArray();
+            Push();
+            if (Depth() >= isEmpty.Length)
+            {
+                isEmpty.Length += isEmpty.Length;
+            }
+
+            isEmpty.Set(Depth(), true);
+        }
+
+        /// <inheritdoc />
+        public override void WriteArrayEnd()
+        {
+            if (!isEmpty.Get(Pos))
+            {
+                parser.Advance(Symbol.ItemEnd);
+            }
+
+            Pop();
+            parser.Advance(Symbol.ArrayEnd);
+            writer.WriteEndArray();
+        }
+
+        /// <inheritdoc />
+        public override void WriteMapStart()
+        {
+            Push();
+            if (Depth() >= isEmpty.Length)
+            {
+                isEmpty.Length += isEmpty.Length;
+            }
+
+            isEmpty.Set(Depth(), true);
+
+            parser.Advance(Symbol.MapStart);
+            writer.WriteStartObject();
+        }
+
+        /// <inheritdoc />
+        public override void WriteMapEnd()
+        {
+            if (!isEmpty.Get(Pos))
+            {
+                parser.Advance(Symbol.ItemEnd);
+            }
+
+            Pop();
+
+            parser.Advance(Symbol.MapEnd);
+            writer.WriteEndObject();
+        }
+
+        /// <summary>
+        /// Start an array item.
+        /// </summary>
+        public new void StartItem()

Review Comment:
   This is inherited from `Encoder`, and `encoder.StartItem` does seem to be called in several places.



##########
lang/csharp/src/apache/main/IO/ParsingDecoder.cs:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.
+ */
+
+using Avro.IO.Parsing;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// Base class for <a href="parsing/package-summary.html">parser</a>-based

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.

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

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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r951083531


##########
lang/csharp/src/apache/main/IO/JsonEncoder.cs:
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.
+ */
+
+using Avro.IO.Parsing;
+using System.Collections;
+using System.IO;
+using System.Text;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// An <seealso cref="Encoder"/> for Avro's JSON data encoding.

Review Comment:
   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.

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

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


[GitHub] [avro] github-code-scanning[bot] commented on a diff in pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r950951306


##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1049 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)

Review Comment:
   ## Exposing internal representation
   
   'Fixup' exposes the internal representation stored in field 'symbols'. The value may be modified [through the variable output](1).
   'Fixup' exposes the internal representation stored in field 'symbols'. The value may be modified [through the variable output](2).
   'Fixup' exposes the internal representation stored in field 'symbols'. The value may be modified [through the variable output](3).
   'Fixup' exposes the internal representation stored in field 'symbols'. The value may be modified [through the variable output](4).
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2905)



##########
lang/csharp/src/apache/test/IO/JsonCodecTests.cs:
##########
@@ -0,0 +1,226 @@
+/**
+ * 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.
+ */
+
+using System;
+using NUnit.Framework;
+using System.IO;
+using System.Linq;
+using System.Text;
+using Avro.Generic;
+using Avro.IO;
+using Newtonsoft.Json.Linq;
+
+namespace Avro.Test
+{
+    using Decoder = Avro.IO.Decoder;
+    using Encoder = Avro.IO.Encoder;
+
+    /// <summary>
+    /// Tests the JsonEncoder and JsonDecoder.
+    /// </summary>
+    [TestFixture]
+    public class JsonCodecTests
+    {
+        [TestCase]
+        public void TestJsonEncoderWhenIncludeNamespaceOptionIsFalse()
+        {
+            string value = "{\"b\": {\"string\":\"myVal\"}, \"a\": 1}";
+            string schemaStr = "{\"type\": \"record\", \"name\": \"ab\", \"fields\": [" +
+                               "{\"name\": \"a\", \"type\": \"int\"}, {\"name\": \"b\", \"type\": [\"null\", \"string\"]}" +
+                               "]}";
+            Schema schema = Schema.Parse(schemaStr);
+            byte[] avroBytes = fromJsonToAvro(value, schema);
+
+            Assert.IsTrue(JToken.DeepEquals(JObject.Parse("{\"b\":\"myVal\",\"a\":1}"),
+                JObject.Parse(fromAvroToJson(avroBytes, schema, false))));
+        }
+
+        [TestCase]
+        public void TestJsonEncoderWhenIncludeNamespaceOptionIsTrue()
+        {
+            string value = "{\"b\": {\"string\":\"myVal\"}, \"a\": 1}";
+            string schemaStr = "{\"type\": \"record\", \"name\": \"ab\", \"fields\": [" +
+                               "{\"name\": \"a\", \"type\": \"int\"}, {\"name\": \"b\", \"type\": [\"null\", \"string\"]}" +
+                               "]}";
+            Schema schema = Schema.Parse(schemaStr);
+            byte[] avroBytes = fromJsonToAvro(value, schema);
+
+            Assert.IsTrue(JToken.DeepEquals(JObject.Parse("{\"b\":{\"string\":\"myVal\"},\"a\":1}"),
+                JObject.Parse(fromAvroToJson(avroBytes, schema, true))));
+        }
+
+        [TestCase]
+        public void TestJsonRecordOrdering()
+        {
+            string value = "{\"b\": 2, \"a\": 1}";
+            Schema schema = Schema.Parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [" +
+                                         "{\"name\": \"a\", \"type\": \"int\"}, {\"name\": \"b\", \"type\": \"int\"}" +
+                                         "]}");
+            GenericDatumReader<object> reader = new GenericDatumReader<object>(schema, schema);
+            Decoder decoder = new JsonDecoder(schema, value);
+            object o = reader.Read(null, decoder);
+
+            Assert.AreEqual("{\"a\":1,\"b\":2}", fromDatumToJson(o, schema, false));
+        }
+
+        [TestCase]
+        public void TestJsonRecordOrdering2()
+        {
+            string value = "{\"b\": { \"b3\": 1.4, \"b2\": 3.14, \"b1\": \"h\"}, \"a\": {\"a2\":true, \"a1\": null}}";
+            Schema schema = Schema.Parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n" +
+                                         "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n" +
+                                         "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}},\n" +
+                                         "{\"name\": \"b\", \"type\": {\"type\":\"record\",\"name\":\"B\",\"fields\":\n" +
+                                         "[{\"name\":\"b1\", \"type\":\"string\"}, {\"name\":\"b2\", \"type\":\"float\"}, {\"name\":\"b3\", \"type\":\"double\"}]}}\n" +
+                                         "]}");
+            GenericDatumReader<object> reader = new GenericDatumReader<object>(schema, schema);
+            Decoder decoder = new JsonDecoder(schema, value);
+            object o = reader.Read(null, decoder);
+
+            Assert.AreEqual("{\"a\":{\"a1\":null,\"a2\":true},\"b\":{\"b1\":\"h\",\"b2\":3.14,\"b3\":1.4}}",
+                fromDatumToJson(o, schema, false));
+        }
+
+        [TestCase]
+        public void TestJsonRecordOrderingWithProjection()
+        {
+            String value = "{\"b\": { \"b3\": 1.4, \"b2\": 3.14, \"b1\": \"h\"}, \"a\": {\"a2\":true, \"a1\": null}}";
+            Schema writerSchema = Schema.Parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n"
+                                               + "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n"
+                                               + "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}},\n"
+                                               + "{\"name\": \"b\", \"type\": {\"type\":\"record\",\"name\":\"B\",\"fields\":\n"
+                                               + "[{\"name\":\"b1\", \"type\":\"string\"}, {\"name\":\"b2\", \"type\":\"float\"}, {\"name\":\"b3\", \"type\":\"double\"}]}}\n"
+                                               + "]}");
+            Schema readerSchema = Schema.Parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n"
+                                               + "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n"
+                                               + "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}}\n" +
+                                               "]}");
+            GenericDatumReader<object> reader = new GenericDatumReader<object>(writerSchema, readerSchema);
+            Decoder decoder = new JsonDecoder(writerSchema, value);
+            Object o = reader.Read(null, decoder);
+
+            Assert.AreEqual("{\"a\":{\"a1\":null,\"a2\":true}}",
+                fromDatumToJson(o, readerSchema, false));
+        }
+
+
+        [TestCase]
+        public void testJsonRecordOrderingWithProjection2()
+        {
+            String value =
+                "{\"b\": { \"b1\": \"h\", \"b2\": [3.14, 3.56], \"b3\": 1.4}, \"a\": {\"a2\":true, \"a1\": null}}";
+            Schema writerSchema = Schema.Parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n"
+                                               + "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n"
+                                               + "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}},\n"
+                                               + "{\"name\": \"b\", \"type\": {\"type\":\"record\",\"name\":\"B\",\"fields\":\n"
+                                               + "[{\"name\":\"b1\", \"type\":\"string\"}, {\"name\":\"b2\", \"type\":{\"type\":\"array\", \"items\":\"float\"}}, {\"name\":\"b3\", \"type\":\"double\"}]}}\n"
+                                               + "]}");
+
+            Schema readerSchema = Schema.Parse("{\"type\": \"record\", \"name\": \"ab\", \"fields\": [\n"
+                                               + "{\"name\": \"a\", \"type\": {\"type\":\"record\",\"name\":\"A\",\"fields\":\n"
+                                               + "[{\"name\":\"a1\", \"type\":\"null\"}, {\"name\":\"a2\", \"type\":\"boolean\"}]}}\n" +
+                                               "]}");
+
+            GenericDatumReader<object> reader = new GenericDatumReader<object>(writerSchema, readerSchema);
+            Decoder decoder = new JsonDecoder(writerSchema, value);
+            object o = reader.Read(null, decoder);
+
+            Assert.AreEqual("{\"a\":{\"a1\":null,\"a2\":true}}",
+                fromDatumToJson(o, readerSchema, false));
+        }
+
+        [TestCase("int", 1)]
+        [TestCase("long", 1L)]
+        [TestCase("float", 1.0F)]
+        [TestCase("double", 1.0)]
+        public void TestJsonDecoderNumeric(string type, object value)
+        {
+            string def = "{\"type\":\"record\",\"name\":\"X\",\"fields\":" + "[{\"type\":\"" + type +
+                         "\",\"name\":\"n\"}]}";
+            Schema schema = Schema.Parse(def);
+            DatumReader<GenericRecord> reader = new GenericDatumReader<GenericRecord>(schema, schema);
+
+            string[] records = { "{\"n\":1}", "{\"n\":1.0}" };
+
+            foreach (JsonDecoder decoder in records.Select(r => new JsonDecoder(schema, r)))
+            {
+                GenericRecord r = reader.Read(null, decoder);
+                Assert.AreEqual(value, r["n"]);
+            }

Review Comment:
   ## Missed opportunity to use Select
   
   This foreach loop immediately maps its iteration variable to another variable [here](1) - consider mapping the sequence explicitly using '.Select(...)'.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2907)



##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1049 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private Symbol[] symbols;

Review Comment:
   ## Missed 'readonly' opportunity
   
   Field 'symbols' can be 'readonly'.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2906)



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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r955440136


##########
lang/csharp/src/apache/main/IO/JsonDecoder.cs:
##########
@@ -0,0 +1,765 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using System.Text;
+using Avro.IO.Parsing;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// A <see cref="Decoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonDecoder is not thread-safe.
+    /// </summary>
+    public class JsonDecoder : ParsingDecoder
+    {
+        private JsonReader reader;
+        private readonly Stack<ReorderBuffer> reorderBuffers = new Stack<ReorderBuffer>();
+        private ReorderBuffer currentReorderBuffer;
+
+        private class ReorderBuffer
+        {
+            public readonly IDictionary<string, IList<JsonElement>> SavedFields =
+                new Dictionary<string, IList<JsonElement>>();
+
+            public JsonReader OrigParser;
+        }
+
+        private JsonDecoder(Symbol root, Stream stream) : base(root)
+        {
+            Configure(stream);
+        }
+
+        private JsonDecoder(Symbol root, string str) : base(root)
+        {
+            Configure(str);
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, Stream stream) : this(getSymbol(schema), stream)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonDecoder"/> class.
+        /// </summary>
+        public JsonDecoder(Schema schema, string str) : this(getSymbol(schema), str)
+        {
+        }
+
+        private static Symbol getSymbol(Schema schema)
+        {
+            return (new JsonGrammarGenerator()).Generate(schema);
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the InputStream provided.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="stream"> The InputStream to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(Stream stream)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StreamReader(stream));
+            this.reader.Read();
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonDecoder to use the String provided for input.
+        /// Otherwise, this JsonDecoder will reset its state and then reconfigure its
+        /// input.
+        /// </summary>
+        /// <param name="str"> The String to read from. Cannot be null. </param>
+        /// <returns> this JsonDecoder </returns>
+        public JsonDecoder Configure(string str)
+        {
+            Parser.Reset();
+            reorderBuffers.Clear();
+            currentReorderBuffer = null;
+            this.reader = new JsonTextReader(new StringReader(str));
+            this.reader.Read();
+            return this;
+        }
+
+        private void advance(Symbol symbol)
+        {
+            this.Parser.ProcessTrailingImplicitActions();
+            Parser.Advance(symbol);
+        }
+
+        /// <inheritdoc />
+        public override void ReadNull()
+        {
+            advance(Symbol.Null);
+            if (reader.TokenType == JsonToken.Null)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("null");
+            }
+        }
+
+        /// <inheritdoc />
+        public override bool ReadBoolean()
+        {
+            advance(Symbol.Boolean);
+            if (reader.TokenType == JsonToken.Boolean)
+            {
+                bool result = Convert.ToBoolean(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("boolean");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadInt()
+        {
+            advance(Symbol.Int);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                int result = Convert.ToInt32(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("int");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadLong()
+        {
+            advance(Symbol.Long);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                long result = Convert.ToInt64(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("long");
+            }
+        }
+
+        /// <inheritdoc />
+        public override float ReadFloat()
+        {
+            advance(Symbol.Float);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                float result = (float)Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("float");
+            }
+        }
+
+        /// <inheritdoc />
+        public override double ReadDouble()
+        {
+            advance(Symbol.Double);
+            if (reader.TokenType == JsonToken.Integer || reader.TokenType == JsonToken.Float)
+            {
+                double result = Convert.ToDouble(reader.Value);
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("double");
+            }
+        }
+
+        /// <inheritdoc />
+        public override string ReadString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            string result = Convert.ToString(reader.Value);
+            reader.Read();
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipString()
+        {
+            advance(Symbol.String);
+            if (Parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                Parser.Advance(Symbol.MapKeyMarker);
+                if (reader.TokenType != JsonToken.PropertyName)
+                {
+                    throw error("map-key");
+                }
+            }
+            else
+            {
+                if (reader.TokenType != JsonToken.String)
+                {
+                    throw error("string");
+                }
+            }
+
+            reader.Read();
+        }
+
+        /// <inheritdoc />
+        public override byte[] ReadBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                return result;
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private byte[] readByteArray()
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            byte[] result = iso.GetBytes(Convert.ToString(reader.Value));
+            return result;
+        }
+
+        /// <inheritdoc />
+        public override void SkipBytes()
+        {
+            advance(Symbol.Bytes);
+            if (reader.TokenType == JsonToken.String)
+            {
+                reader.Read();
+            }
+            else
+            {
+                throw error("bytes");
+            }
+        }
+
+        private void checkFixed(int size)
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            if (size != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + size + " bytes.");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes)
+        {
+            ReadFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void ReadFixed(byte[] bytes, int start, int len)
+        {
+            checkFixed(len);
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != len)
+                {
+                    throw new AvroTypeException("Expected fixed length " + len + ", but got" + result.Length);
+                }
+
+                Array.Copy(result, 0, bytes, start, len);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipFixed(int length)
+        {
+            checkFixed(length);
+            doSkipFixed(length);
+        }
+
+        private void doSkipFixed(int length)
+        {
+            if (reader.TokenType == JsonToken.String)
+            {
+                byte[] result = readByteArray();
+                reader.Read();
+                if (result.Length != length)
+                {
+                    throw new AvroTypeException("Expected fixed length " + length + ", but got" + result.Length);
+                }
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        protected override void SkipFixed()
+        {
+            advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)Parser.PopSymbol();
+            doSkipFixed(top.Size);
+        }
+
+        /// <inheritdoc />
+        public override int ReadEnum()
+        {
+            advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)Parser.PopSymbol();
+            if (reader.TokenType == JsonToken.String)
+            {
+                string label = Convert.ToString(reader.Value);
+                int n = top.FindLabel(label);
+                if (n >= 0)
+                {
+                    reader.Read();
+                    return n;
+                }
+
+                throw new AvroTypeException("Unknown symbol in enum " + label);
+            }
+            else
+            {
+                throw error("fixed");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayStart()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Read();
+                return doArrayNext();
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadArrayNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doArrayNext();
+        }
+
+        private long doArrayNext()
+        {
+            if (reader.TokenType == JsonToken.EndArray)
+            {
+                Parser.Advance(Symbol.ArrayEnd);
+                reader.Read();
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipArray()
+        {
+            advance(Symbol.ArrayStart);
+            if (reader.TokenType == JsonToken.StartArray)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.ArrayEnd);
+            }
+            else
+            {
+                throw error("array-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapStart()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                return doMapNext();
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override long ReadMapNext()
+        {
+            advance(Symbol.ItemEnd);
+            return doMapNext();
+        }
+
+        private long doMapNext()
+        {
+            if (reader.TokenType == JsonToken.EndObject)
+            {
+                reader.Read();
+                advance(Symbol.MapEnd);
+                return 0;
+            }
+            else
+            {
+                return 1;
+            }
+        }
+
+        /// <inheritdoc />
+        public override void SkipMap()
+        {
+            advance(Symbol.MapStart);
+            if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Skip();
+                reader.Read();
+                advance(Symbol.MapEnd);
+            }
+            else
+            {
+                throw error("map-start");
+            }
+        }
+
+        /// <inheritdoc />
+        public override int ReadUnionIndex()
+        {
+            advance(Symbol.Union);
+            Symbol.Alternative a = (Symbol.Alternative)Parser.PopSymbol();
+
+            string label;
+            if (reader.TokenType == JsonToken.Null)
+            {
+                label = "null";
+            }
+            else if (reader.TokenType == JsonToken.StartObject)
+            {
+                reader.Read();
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    label = Convert.ToString(reader.Value);
+                    reader.Read();
+                    Parser.PushSymbol(Symbol.UnionEnd);
+                }
+                else
+                {
+                    throw error("start-union");
+                }
+            }
+            else
+            {
+                throw error("start-union");
+            }
+
+            int n = a.FindLabel(label);
+            if (n < 0)
+            {
+                throw new AvroTypeException("Unknown union branch " + label);
+            }
+
+            Parser.PushSymbol(a.GetSymbol(n));
+            return n;
+        }
+
+        /// <inheritdoc />
+        public override void SkipNull()
+        {
+            ReadNull();
+        }
+
+        /// <inheritdoc />
+        public override void SkipBoolean()
+        {
+            ReadBoolean();
+        }
+
+        /// <inheritdoc />
+        public override void SkipInt()
+        {
+            ReadInt();
+        }
+
+        /// <inheritdoc />
+        public override void SkipLong()
+        {
+            ReadLong();
+        }
+
+        /// <inheritdoc />
+        public override void SkipFloat()
+        {
+            ReadFloat();
+        }
+
+        /// <inheritdoc />
+        public override void SkipDouble()
+        {
+            ReadDouble();
+        }
+
+        /// <inheritdoc />
+        public override void SkipEnum()
+        {
+            ReadEnum();
+        }
+
+        /// <inheritdoc />
+        public override void SkipUnionIndex()
+        {
+            ReadUnionIndex();
+        }
+
+        /// <inheritdoc />
+        public override Symbol DoAction(Symbol input, Symbol top)
+        {
+            if (top is Symbol.FieldAdjustAction)
+            {
+                Symbol.FieldAdjustAction fa = (Symbol.FieldAdjustAction)top;
+                string name = fa.FName;
+                if (currentReorderBuffer != null)
+                {
+                    IList<JsonElement> node = currentReorderBuffer.SavedFields[name];
+                    if (node != null)
+                    {
+                        currentReorderBuffer.SavedFields.Remove(name);
+                        currentReorderBuffer.OrigParser = reader;
+                        reader = makeParser(node);
+                        return null;
+                    }
+                }
+
+                if (reader.TokenType == JsonToken.PropertyName)
+                {
+                    do
+                    {
+                        string fn = Convert.ToString(reader.Value);
+                        reader.Read();
+                        if (name.Equals(fn) || (fa.Aliases != null && fa.Aliases.Contains(fn)))
+                        {
+                            return null;
+                        }
+                        else
+                        {
+                            if (currentReorderBuffer == null)
+                            {
+                                currentReorderBuffer = new ReorderBuffer();
+                            }
+
+                            currentReorderBuffer.SavedFields[fn] = getValueAsTree(reader);
+                        }
+                    } while (reader.TokenType == JsonToken.PropertyName);
+
+                    throw new AvroTypeException("Expected field name not found: " + fa.FName);
+                }
+            }
+            else if (top == Symbol.FieldEnd)
+            {
+                if (currentReorderBuffer != null && currentReorderBuffer.OrigParser != null)
+                {
+                    reader = currentReorderBuffer.OrigParser;
+                    currentReorderBuffer.OrigParser = null;
+                }
+            }
+            else if (top == Symbol.RecordStart)
+            {
+                if (reader.TokenType == JsonToken.StartObject)
+                {
+                    reader.Read();
+                    reorderBuffers.Push(currentReorderBuffer);
+                    currentReorderBuffer = null;
+                }
+                else
+                {
+                    throw error("record-start");
+                }
+            }
+            else if (top == Symbol.RecordEnd || top == Symbol.UnionEnd)
+            {
+                // AVRO-2034 advance to the end of our object
+                while (reader.TokenType != JsonToken.EndObject)
+                {
+                    reader.Read();
+                }
+
+                if (top == Symbol.RecordEnd)
+                {
+                    if (currentReorderBuffer != null && currentReorderBuffer.SavedFields.Count > 0)
+                    {
+                        throw error("Unknown fields: " + currentReorderBuffer.SavedFields.Keys);

Review Comment:
   Thanks, 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.

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

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


[GitHub] [avro] KhrystynaPopadyuk commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KhrystynaPopadyuk commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1229134158

   Hi @rayokota ,
   
   All comments below, are suggestion only.
   My biggest concern is that user are not able inject own custom logic without changing libraries. If you add more interfaces and use interface instead of concert classes as dependency, it allows users who want benefit from dependency injection use 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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1223162854

   @KalleOlaviNiemitalo , it turns out that the schema above was not parseable in the C# implementation, due to https://issues.apache.org/jira/browse/AVRO-3613.  I've fixed that issue in this PR by overriding the `Fullname` property in `LogicalSchema`.  Fixing that also fixed some behavior in the `JsonGrammarGenerator` and the `ValidatingGrammarGenerator` that relied on `Fullname`.
   
   There was another issue with your schema in that there was an ambiguity when trying to JSON encode a `Date`, since both logicalTypes could be used to represent a `Date`.  I changed your example to have one of the fields be a logical type of `Uuid` 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.

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

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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r951109967


##########
lang/csharp/src/apache/main/IO/JsonEncoder.cs:
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.
+ */
+
+using Avro.IO.Parsing;
+using System.Collections;
+using System.IO;
+using System.Text;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// An <seealso cref="Encoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonEncoder buffers output, and data may not appear on the output until
+    /// <seealso cref="Encoder.Flush()"/> is called.
+    ///
+    /// JsonEncoder is not thread-safe.
+    /// </summary>
+    public class JsonEncoder : ParsingEncoder, Parser.ActionHandler
+    {
+        private readonly Parser parser;
+        private JsonWriter writer;
+        private bool includeNamespace = true;
+
+        // Has anything been written into the collections?
+        private readonly BitArray isEmpty = new BitArray(64);
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream) : this(sc, getJsonWriter(stream, false))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream, bool pretty) : this(sc, getJsonWriter(stream, pretty))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, JsonWriter writer)
+        {
+            Configure(writer);
+            this.parser = new Parser((new JsonGrammarGenerator()).Generate(sc), this);
+        }
+
+        /// <inheritdoc />
+        public override void Flush()
+        {
+            parser.ProcessImplicitActions();
+            if (writer != null)
+            {
+                writer.Flush();
+            }
+        }
+
+        // by default, one object per line.
+        // with pretty option use default pretty printer with root line separator.
+        private static JsonWriter getJsonWriter(Stream stream, bool pretty)
+        {
+            JsonWriter writer = new JsonTextWriter(new StreamWriter(stream));
+            if (pretty)
+            {
+                writer.Formatting = Formatting.Indented;
+            }
+
+            return writer;
+        }
+
+        /// <summary>
+        /// Whether to include the namespace.
+        /// </summary>
+        public virtual bool IncludeNamespace
+        {
+            get { return includeNamespace; }
+            set { this.includeNamespace = value; }
+        }
+
+
+        /// <summary>
+        /// Reconfigures this JsonEncoder to use the output stream provided.
+        /// <p/>
+        /// Otherwise, this JsonEncoder will flush its current output and then
+        /// reconfigure its output to use a default UTF8 JsonWriter that writes to the
+        /// provided OutputStream.
+        /// </summary>
+        /// <param name="stream"> The OutputStream to direct output to. Cannot be null. </param>
+        /// <returns> this JsonEncoder </returns>
+        public JsonEncoder Configure(Stream stream)
+        {
+            this.Configure(getJsonWriter(stream, false));
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonEncoder to output to the JsonWriter provided.
+        /// <p/>
+        /// Otherwise, this JsonEncoder will flush its current output and then
+        /// reconfigure its output to use the provided JsonWriter.
+        /// </summary>
+        /// <param name="jsonWriter"> The JsonWriter to direct output to. Cannot be null. </param>
+        /// <returns> this JsonEncoder </returns>
+        public JsonEncoder Configure(JsonWriter jsonWriter)
+        {
+            if (null != parser)
+            {
+                Flush();
+            }
+
+            this.writer = jsonWriter;
+            return this;
+        }
+
+        /// <inheritdoc />
+        public override void WriteNull()
+        {
+            parser.Advance(Symbol.Null);
+            writer.WriteNull();
+        }
+
+        /// <inheritdoc />
+        public override void WriteBoolean(bool b)
+        {
+            parser.Advance(Symbol.Boolean);
+            writer.WriteValue(b);
+        }
+
+        /// <inheritdoc />
+        public override void WriteInt(int n)
+        {
+            parser.Advance(Symbol.Int);
+            writer.WriteValue(n);
+        }
+
+        /// <inheritdoc />
+        public override void WriteLong(long n)
+        {
+            parser.Advance(Symbol.Long);
+            writer.WriteValue(n);
+        }
+
+        /// <inheritdoc />
+        public override void WriteFloat(float f)
+        {
+            parser.Advance(Symbol.Float);
+            writer.WriteValue(f);
+        }
+
+        /// <inheritdoc />
+        public override void WriteDouble(double d)
+        {
+            parser.Advance(Symbol.Double);
+            writer.WriteValue(d);
+        }
+
+        /// <inheritdoc />
+        public override void WriteString(string str)
+        {
+            parser.Advance(Symbol.String);
+            if (parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                parser.Advance(Symbol.MapKeyMarker);
+                writer.WritePropertyName(str);
+            }
+            else
+            {
+                writer.WriteValue(str);
+            }
+        }
+
+        /// <inheritdoc />
+        public override void WriteBytes(byte[] bytes)
+        {
+            WriteBytes(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void WriteBytes(byte[] bytes, int start, int len)
+        {
+            parser.Advance(Symbol.Bytes);
+            writeByteArray(bytes, start, len);
+        }
+
+        private void writeByteArray(byte[] bytes, int start, int len)
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            writer.WriteValue(iso.GetString(bytes, start, len));
+        }
+
+        /// <inheritdoc />
+        public override void WriteFixed(byte[] bytes)
+        {
+            WriteFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void WriteFixed(byte[] bytes, int start, int len)
+        {
+            parser.Advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)parser.PopSymbol();
+            if (len != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + len + " bytes.");
+            }
+
+            writeByteArray(bytes, start, len);
+        }
+
+        /// <inheritdoc />
+        public override void WriteEnum(int e)
+        {
+            parser.Advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)parser.PopSymbol();
+            if (e < 0 || e >= top.Size)
+            {
+                throw new AvroTypeException("Enumeration out of range: max is " + top.Size + " but received " + e);
+            }
+
+            writer.WriteValue(top.GetLabel(e));
+        }
+
+        /// <inheritdoc />
+        public override void WriteArrayStart()
+        {
+            parser.Advance(Symbol.ArrayStart);
+            writer.WriteStartArray();
+            Push();
+            if (Depth() >= isEmpty.Length)
+            {
+                isEmpty.Length += isEmpty.Length;
+            }
+
+            isEmpty.Set(Depth(), true);
+        }
+
+        /// <inheritdoc />
+        public override void WriteArrayEnd()
+        {
+            if (!isEmpty.Get(Pos))
+            {
+                parser.Advance(Symbol.ItemEnd);
+            }
+
+            Pop();
+            parser.Advance(Symbol.ArrayEnd);
+            writer.WriteEndArray();
+        }
+
+        /// <inheritdoc />
+        public override void WriteMapStart()
+        {
+            Push();
+            if (Depth() >= isEmpty.Length)
+            {
+                isEmpty.Length += isEmpty.Length;
+            }
+
+            isEmpty.Set(Depth(), true);
+
+            parser.Advance(Symbol.MapStart);
+            writer.WriteStartObject();
+        }
+
+        /// <inheritdoc />
+        public override void WriteMapEnd()
+        {
+            if (!isEmpty.Get(Pos))
+            {
+                parser.Advance(Symbol.ItemEnd);
+            }
+
+            Pop();
+
+            parser.Advance(Symbol.MapEnd);
+            writer.WriteEndObject();
+        }
+
+        /// <summary>
+        /// Start an array item.
+        /// </summary>
+        public new void StartItem()

Review Comment:
   The Encoder.StartItem() implementation comes from ParsingEncoder.StartItem(), not from JsonEncoder.StartItem(). The following outputs "BaseImpl.Run":
   
   ```C#
   using System;
   
   namespace InterfaceImpl
   {
       interface IFace
       {
           void Run();
       }
   
       class BaseImpl : IFace
       {
           public void Run()
           {
               Console.WriteLine("BaseImpl.Run");
           }
       }
   
       class DerivedImpl : BaseImpl
       {
           public new void Run()
           {
               Console.WriteLine("DerivedImpl.Run");
           }
       }
   
       class Program
       {
           static void Main()
           {
               IFace i = new DerivedImpl();
               i.Run();
           }
       }
   }
   ```
   
   If you change that to `class DerivedImpl : BaseImpl, IFace`, then it outputs "DerivedImpl.Run". Which would correspond to `public class JsonEncoder : ParsingEncoder, Encoder, Parser.IActionHandler` in the Avro library. However, I think it would be simpler to make ParsingEncoder.StartItem() virtual, because methods that implement interface methods must be virtual in the CLI metadata anyway (ECMA-335 6th ed. §II.12.2 and §II.22.27; if such a method is not virtual in C# source code, then the C# compiler makes it final virtual in the metadata).



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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1221744800

   @martin-g @KyleSchoonover @zcsizmadia could I kindly get a review?  Thanks in advance!


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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r951537746


##########
lang/csharp/src/apache/main/IO/JsonEncoder.cs:
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.
+ */
+
+using Avro.IO.Parsing;
+using System.Collections;
+using System.IO;
+using System.Text;
+using Newtonsoft.Json;
+
+namespace Avro.IO
+{
+    /// <summary>
+    /// An <seealso cref="Encoder"/> for Avro's JSON data encoding.
+    ///
+    /// JsonEncoder buffers output, and data may not appear on the output until
+    /// <seealso cref="Encoder.Flush()"/> is called.
+    ///
+    /// JsonEncoder is not thread-safe.
+    /// </summary>
+    public class JsonEncoder : ParsingEncoder, Parser.ActionHandler
+    {
+        private readonly Parser parser;
+        private JsonWriter writer;
+        private bool includeNamespace = true;
+
+        // Has anything been written into the collections?
+        private readonly BitArray isEmpty = new BitArray(64);
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream) : this(sc, getJsonWriter(stream, false))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, Stream stream, bool pretty) : this(sc, getJsonWriter(stream, pretty))
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="JsonEncoder"/> class.
+        /// </summary>
+        public JsonEncoder(Schema sc, JsonWriter writer)
+        {
+            Configure(writer);
+            this.parser = new Parser((new JsonGrammarGenerator()).Generate(sc), this);
+        }
+
+        /// <inheritdoc />
+        public override void Flush()
+        {
+            parser.ProcessImplicitActions();
+            if (writer != null)
+            {
+                writer.Flush();
+            }
+        }
+
+        // by default, one object per line.
+        // with pretty option use default pretty printer with root line separator.
+        private static JsonWriter getJsonWriter(Stream stream, bool pretty)
+        {
+            JsonWriter writer = new JsonTextWriter(new StreamWriter(stream));
+            if (pretty)
+            {
+                writer.Formatting = Formatting.Indented;
+            }
+
+            return writer;
+        }
+
+        /// <summary>
+        /// Whether to include the namespace.
+        /// </summary>
+        public virtual bool IncludeNamespace
+        {
+            get { return includeNamespace; }
+            set { this.includeNamespace = value; }
+        }
+
+
+        /// <summary>
+        /// Reconfigures this JsonEncoder to use the output stream provided.
+        /// <p/>
+        /// Otherwise, this JsonEncoder will flush its current output and then
+        /// reconfigure its output to use a default UTF8 JsonWriter that writes to the
+        /// provided OutputStream.
+        /// </summary>
+        /// <param name="stream"> The OutputStream to direct output to. Cannot be null. </param>
+        /// <returns> this JsonEncoder </returns>
+        public JsonEncoder Configure(Stream stream)
+        {
+            this.Configure(getJsonWriter(stream, false));
+            return this;
+        }
+
+        /// <summary>
+        /// Reconfigures this JsonEncoder to output to the JsonWriter provided.
+        /// <p/>
+        /// Otherwise, this JsonEncoder will flush its current output and then
+        /// reconfigure its output to use the provided JsonWriter.
+        /// </summary>
+        /// <param name="jsonWriter"> The JsonWriter to direct output to. Cannot be null. </param>
+        /// <returns> this JsonEncoder </returns>
+        public JsonEncoder Configure(JsonWriter jsonWriter)
+        {
+            if (null != parser)
+            {
+                Flush();
+            }
+
+            this.writer = jsonWriter;
+            return this;
+        }
+
+        /// <inheritdoc />
+        public override void WriteNull()
+        {
+            parser.Advance(Symbol.Null);
+            writer.WriteNull();
+        }
+
+        /// <inheritdoc />
+        public override void WriteBoolean(bool b)
+        {
+            parser.Advance(Symbol.Boolean);
+            writer.WriteValue(b);
+        }
+
+        /// <inheritdoc />
+        public override void WriteInt(int n)
+        {
+            parser.Advance(Symbol.Int);
+            writer.WriteValue(n);
+        }
+
+        /// <inheritdoc />
+        public override void WriteLong(long n)
+        {
+            parser.Advance(Symbol.Long);
+            writer.WriteValue(n);
+        }
+
+        /// <inheritdoc />
+        public override void WriteFloat(float f)
+        {
+            parser.Advance(Symbol.Float);
+            writer.WriteValue(f);
+        }
+
+        /// <inheritdoc />
+        public override void WriteDouble(double d)
+        {
+            parser.Advance(Symbol.Double);
+            writer.WriteValue(d);
+        }
+
+        /// <inheritdoc />
+        public override void WriteString(string str)
+        {
+            parser.Advance(Symbol.String);
+            if (parser.TopSymbol() == Symbol.MapKeyMarker)
+            {
+                parser.Advance(Symbol.MapKeyMarker);
+                writer.WritePropertyName(str);
+            }
+            else
+            {
+                writer.WriteValue(str);
+            }
+        }
+
+        /// <inheritdoc />
+        public override void WriteBytes(byte[] bytes)
+        {
+            WriteBytes(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void WriteBytes(byte[] bytes, int start, int len)
+        {
+            parser.Advance(Symbol.Bytes);
+            writeByteArray(bytes, start, len);
+        }
+
+        private void writeByteArray(byte[] bytes, int start, int len)
+        {
+            Encoding iso = Encoding.GetEncoding("ISO-8859-1");
+            writer.WriteValue(iso.GetString(bytes, start, len));
+        }
+
+        /// <inheritdoc />
+        public override void WriteFixed(byte[] bytes)
+        {
+            WriteFixed(bytes, 0, bytes.Length);
+        }
+
+        /// <inheritdoc />
+        public override void WriteFixed(byte[] bytes, int start, int len)
+        {
+            parser.Advance(Symbol.Fixed);
+            Symbol.IntCheckAction top = (Symbol.IntCheckAction)parser.PopSymbol();
+            if (len != top.Size)
+            {
+                throw new AvroTypeException("Incorrect length for fixed binary: expected " + top.Size +
+                                            " but received " + len + " bytes.");
+            }
+
+            writeByteArray(bytes, start, len);
+        }
+
+        /// <inheritdoc />
+        public override void WriteEnum(int e)
+        {
+            parser.Advance(Symbol.Enum);
+            Symbol.EnumLabelsAction top = (Symbol.EnumLabelsAction)parser.PopSymbol();
+            if (e < 0 || e >= top.Size)
+            {
+                throw new AvroTypeException("Enumeration out of range: max is " + top.Size + " but received " + e);
+            }
+
+            writer.WriteValue(top.GetLabel(e));
+        }
+
+        /// <inheritdoc />
+        public override void WriteArrayStart()
+        {
+            parser.Advance(Symbol.ArrayStart);
+            writer.WriteStartArray();
+            Push();
+            if (Depth() >= isEmpty.Length)
+            {
+                isEmpty.Length += isEmpty.Length;
+            }
+
+            isEmpty.Set(Depth(), true);
+        }
+
+        /// <inheritdoc />
+        public override void WriteArrayEnd()
+        {
+            if (!isEmpty.Get(Pos))
+            {
+                parser.Advance(Symbol.ItemEnd);
+            }
+
+            Pop();
+            parser.Advance(Symbol.ArrayEnd);
+            writer.WriteEndArray();
+        }
+
+        /// <inheritdoc />
+        public override void WriteMapStart()
+        {
+            Push();
+            if (Depth() >= isEmpty.Length)
+            {
+                isEmpty.Length += isEmpty.Length;
+            }
+
+            isEmpty.Set(Depth(), true);
+
+            parser.Advance(Symbol.MapStart);
+            writer.WriteStartObject();
+        }
+
+        /// <inheritdoc />
+        public override void WriteMapEnd()
+        {
+            if (!isEmpty.Get(Pos))
+            {
+                parser.Advance(Symbol.ItemEnd);
+            }
+
+            Pop();
+
+            parser.Advance(Symbol.MapEnd);
+            writer.WriteEndObject();
+        }
+
+        /// <summary>
+        /// Start an array item.
+        /// </summary>
+        public new void StartItem()

Review Comment:
   Ah, thanks for the detailed explained.  I fixed 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.

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

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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1223575367

   > > I've fixed that issue in this PR by overriding the `Fullname` property in `LogicalSchema`.
   > 
   > That might fix [AVRO-3568](https://issues.apache.org/jira/browse/AVRO-3568) as well.
   
   Thanks @KalleOlaviNiemitalo , I was able to fix [AVRO-3568](https://issues.apache.org/jira/browse/AVRO-3568) by overriding the `Name` property in `LogicalType` as well.  
   
   I'll try to add some more tests as you requested when I get a chance.


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


[GitHub] [avro] martin-g commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
martin-g commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r958196081


##########
lang/csharp/src/apache/test/IO/JsonCodecTests.cs:
##########
@@ -0,0 +1,329 @@
+/**
+ * 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.
+ */
+
+using System;
+using NUnit.Framework;
+using System.IO;
+using System.Linq;
+using System.Text;
+using Avro.Generic;
+using Avro.IO;
+using Newtonsoft.Json;
+using Newtonsoft.Json.Linq;
+
+namespace Avro.Test
+{
+    using Decoder = Avro.IO.Decoder;
+    using Encoder = Avro.IO.Encoder;
+
+    /// <summary>
+    /// Tests the JsonEncoder and JsonDecoder.
+    /// </summary>
+    [TestFixture]
+    public class JsonCodecTests
+    {
+        [TestCase("{ \"type\": \"record\", \"name\": \"r\", \"fields\": [ " +
+                  " { \"name\" : \"f1\", \"type\": \"int\" }, " +
+                  " { \"name\" : \"f2\", \"type\": \"float\" } " +
+                  "] }",
+            "{ \"f2\": 10.4, \"f1\": 10 } ")]
+        [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": " + "[ \"s1\", \"s2\"] }", " \"s1\" ")]
+        [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": " + "[ \"s1\", \"s2\"] }", " \"s2\" ")]
+        [TestCase("{ \"type\": \"fixed\", \"name\": \"f\", \"size\": 5 }", "\"hello\"")]
+        [TestCase("{ \"type\": \"array\", \"items\": \"int\" }", "[ 10, 20, 30 ]")]
+        [TestCase("{ \"type\": \"map\", \"values\": \"int\" }", "{ \"k1\": 10, \"k2\": 20, \"k3\": 30 }")]
+        [TestCase("[ \"int\", \"long\" ]", "{ \"int\": 10 }")]
+        [TestCase("\"string\"", "\"hello\"")]
+        [TestCase("\"bytes\"", "\"hello\"")]
+        [TestCase("\"int\"", "10")]
+        [TestCase("\"long\"", "10")]
+        [TestCase("\"float\"", "10.0")]
+        [TestCase("\"double\"", "10.0")]
+        [TestCase("\"boolean\"", "true")]
+        [TestCase("\"boolean\"", "false")]
+        [TestCase("\"null\"", "null")]
+        public void TestJsonAllTypesValidValues(String schemaStr, String value)
+        {
+            Schema schema = Schema.Parse(schemaStr);
+            byte[] avroBytes = fromJsonToAvro(value, schema);
+
+            Assert.IsTrue(JToken.DeepEquals(JToken.Parse(value),
+                JToken.Parse(fromAvroToJson(avroBytes, schema, true))));
+        }
+
+        [TestCase("{ \"type\": \"record\", \"name\": \"r\", \"fields\": [ " +
+                  " { \"name\" : \"f1\", \"type\": \"int\" }, " +
+                  " { \"name\" : \"f2\", \"type\": \"float\" } " +
+                  "] }",
+            "{ \"f4\": 10.4, \"f3\": 10 } ")]
+        [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": " + "[ \"s1\", \"s2\"] }", " \"s3\" ")]

Review Comment:
   ```suggestion
           [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": [ \"s1\", \"s2\"] }", " \"s3\" ")]
   ```



##########
lang/csharp/src/apache/test/IO/JsonCodecTests.cs:
##########
@@ -0,0 +1,329 @@
+/**
+ * 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.
+ */
+
+using System;
+using NUnit.Framework;
+using System.IO;
+using System.Linq;
+using System.Text;
+using Avro.Generic;
+using Avro.IO;
+using Newtonsoft.Json;
+using Newtonsoft.Json.Linq;
+
+namespace Avro.Test
+{
+    using Decoder = Avro.IO.Decoder;
+    using Encoder = Avro.IO.Encoder;
+
+    /// <summary>
+    /// Tests the JsonEncoder and JsonDecoder.
+    /// </summary>
+    [TestFixture]
+    public class JsonCodecTests
+    {
+        [TestCase("{ \"type\": \"record\", \"name\": \"r\", \"fields\": [ " +
+                  " { \"name\" : \"f1\", \"type\": \"int\" }, " +
+                  " { \"name\" : \"f2\", \"type\": \"float\" } " +
+                  "] }",
+            "{ \"f2\": 10.4, \"f1\": 10 } ")]
+        [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": " + "[ \"s1\", \"s2\"] }", " \"s1\" ")]

Review Comment:
   ```suggestion
           [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": [ \"s1\", \"s2\"] }", " \"s1\" ")]
   ```



##########
lang/csharp/src/apache/test/IO/JsonCodecTests.cs:
##########
@@ -0,0 +1,329 @@
+/**
+ * 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.
+ */
+
+using System;
+using NUnit.Framework;
+using System.IO;
+using System.Linq;
+using System.Text;
+using Avro.Generic;
+using Avro.IO;
+using Newtonsoft.Json;
+using Newtonsoft.Json.Linq;
+
+namespace Avro.Test
+{
+    using Decoder = Avro.IO.Decoder;
+    using Encoder = Avro.IO.Encoder;
+
+    /// <summary>
+    /// Tests the JsonEncoder and JsonDecoder.
+    /// </summary>
+    [TestFixture]
+    public class JsonCodecTests
+    {
+        [TestCase("{ \"type\": \"record\", \"name\": \"r\", \"fields\": [ " +
+                  " { \"name\" : \"f1\", \"type\": \"int\" }, " +
+                  " { \"name\" : \"f2\", \"type\": \"float\" } " +
+                  "] }",
+            "{ \"f2\": 10.4, \"f1\": 10 } ")]
+        [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": " + "[ \"s1\", \"s2\"] }", " \"s1\" ")]
+        [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": " + "[ \"s1\", \"s2\"] }", " \"s2\" ")]
+        [TestCase("{ \"type\": \"fixed\", \"name\": \"f\", \"size\": 5 }", "\"hello\"")]
+        [TestCase("{ \"type\": \"array\", \"items\": \"int\" }", "[ 10, 20, 30 ]")]
+        [TestCase("{ \"type\": \"map\", \"values\": \"int\" }", "{ \"k1\": 10, \"k2\": 20, \"k3\": 30 }")]
+        [TestCase("[ \"int\", \"long\" ]", "{ \"int\": 10 }")]
+        [TestCase("\"string\"", "\"hello\"")]
+        [TestCase("\"bytes\"", "\"hello\"")]
+        [TestCase("\"int\"", "10")]
+        [TestCase("\"long\"", "10")]
+        [TestCase("\"float\"", "10.0")]
+        [TestCase("\"double\"", "10.0")]
+        [TestCase("\"boolean\"", "true")]
+        [TestCase("\"boolean\"", "false")]
+        [TestCase("\"null\"", "null")]
+        public void TestJsonAllTypesValidValues(String schemaStr, String value)
+        {
+            Schema schema = Schema.Parse(schemaStr);
+            byte[] avroBytes = fromJsonToAvro(value, schema);
+
+            Assert.IsTrue(JToken.DeepEquals(JToken.Parse(value),
+                JToken.Parse(fromAvroToJson(avroBytes, schema, true))));
+        }
+
+        [TestCase("{ \"type\": \"record\", \"name\": \"r\", \"fields\": [ " +
+                  " { \"name\" : \"f1\", \"type\": \"int\" }, " +
+                  " { \"name\" : \"f2\", \"type\": \"float\" } " +
+                  "] }",
+            "{ \"f4\": 10.4, \"f3\": 10 } ")]
+        [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": " + "[ \"s1\", \"s2\"] }", " \"s3\" ")]
+        [TestCase("{ \"type\": \"fixed\", \"name\": \"f\", \"size\": 10 }", "\"hello\"")]
+        [TestCase("{ \"type\": \"array\", \"items\": \"int\" }", "[ \"10\", \"20\", \"30\" ]")]
+        [TestCase("{ \"type\": \"map\", \"values\": \"int\" }", "{ \"k1\": \"10\", \"k2\": \"20\"}")]
+        [TestCase("[ \"int\", \"long\" ]", "10")]
+        [TestCase("\"string\"", "10")]
+        [TestCase("\"bytes\"", "10")]
+        [TestCase("\"int\"", "\"hi\"")]
+        [TestCase("\"long\"", "\"hi\"")]
+        [TestCase("\"float\"", "\"hi\"")]
+        [TestCase("\"double\"", "\"hi\"")]
+        [TestCase("\"boolean\"", "\"hi\"")]
+        [TestCase("\"boolean\"", "\"hi\"")]
+        [TestCase("\"null\"", "\"hi\"")]
+        public void TestJsonAllTypesInvalidValues(String schemaStr, String value)
+        {
+            Schema schema = Schema.Parse(schemaStr);
+            Assert.Throws<AvroTypeException>(() => fromJsonToAvro(value, schema));
+        }
+
+        [TestCase("{ \"type\": \"record\", \"name\": \"r\", \"fields\": [ " +
+                  " { \"name\" : \"f1\", \"type\": \"int\" }, " +
+                  " { \"name\" : \"f2\", \"type\": \"float\" } " +
+                  "] }",
+            "{ \"f2\": 10.4, \"f1")]
+        [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": " + "[ \"s1\", \"s2\"] }", "s1")]

Review Comment:
   ```suggestion
           [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": [ \"s1\", \"s2\"] }", "s1")]
   ```



##########
lang/csharp/src/apache/test/IO/JsonCodecTests.cs:
##########
@@ -0,0 +1,329 @@
+/**
+ * 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.
+ */
+
+using System;
+using NUnit.Framework;
+using System.IO;
+using System.Linq;
+using System.Text;
+using Avro.Generic;
+using Avro.IO;
+using Newtonsoft.Json;
+using Newtonsoft.Json.Linq;
+
+namespace Avro.Test
+{
+    using Decoder = Avro.IO.Decoder;
+    using Encoder = Avro.IO.Encoder;
+
+    /// <summary>
+    /// Tests the JsonEncoder and JsonDecoder.
+    /// </summary>
+    [TestFixture]
+    public class JsonCodecTests
+    {
+        [TestCase("{ \"type\": \"record\", \"name\": \"r\", \"fields\": [ " +
+                  " { \"name\" : \"f1\", \"type\": \"int\" }, " +
+                  " { \"name\" : \"f2\", \"type\": \"float\" } " +
+                  "] }",
+            "{ \"f2\": 10.4, \"f1\": 10 } ")]
+        [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": " + "[ \"s1\", \"s2\"] }", " \"s1\" ")]
+        [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": " + "[ \"s1\", \"s2\"] }", " \"s2\" ")]

Review Comment:
   ```suggestion
           [TestCase("{ \"type\": \"enum\", \"name\": \"e\", \"symbols\": [ \"s1\", \"s2\"] }", " \"s2\" ")]
   ```



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


[GitHub] [avro] KalleOlaviNiemitalo commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1223556806

   I see JsonCodecTests.cs has tests with true and false as JsonEncoder.IncludeNamespace; but I don't see any namespaces in the schemas that the tests 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.

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

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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r954557727


##########
lang/csharp/src/apache/main/IO/Encoder.cs:
##########
@@ -187,5 +187,10 @@ public interface Encoder
         /// <param name="start">Position within data where the contents start.</param>
         /// <param name="len">Number of bytes to write.</param>
         void WriteFixed(byte[] data, int start, int len);
+
+        /// <summary>
+        /// Flushes the encoder.
+        /// </summary>
+        void Flush();

Review Comment:
   Adding a method to a public interface is a breaking change. It is OK for this pull request to master (upcoming 1.12.0) but I think these changes should not be backported to branch-1.11.



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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r953339960


##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,1049 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public readonly Kind SymKind;
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public readonly Symbol[] Production;
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            this.Production = production;
+            this.SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols)
+        {
+            return new Root(symbols);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production)
+        {
+            return new Sequence(production);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat)
+        {
+            return new Repeater(endSymbol, symsToRepeat);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels)
+        {
+            return new Alternative(symbols, labels);
+        }
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e)
+        {
+            return new ErrorAction(e);
+        }
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r)
+        {
+            return new ResolvingAction(w, r);
+        }
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public readonly int Pos;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                this.Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            return this;
+        }
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize()
+        {
+            return 1;
+        }
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    IList<Fixup> l;
+                    if (!map2.TryGetValue((Sequence)s, out l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            copyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void copyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public readonly string PrintName;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                this.PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString()
+            {
+                return PrintName;
+            }
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public readonly bool IsTrailing;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                this.IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, makeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index)
+            {
+                return Production[index];
+            }
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Production.Length;
+            }
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator()
+            {
+                return Enumerable.Reverse(Production).GetEnumerator();
+            }
+
+            IEnumerator IEnumerable.GetEnumerator()
+            {
+                return this.GetEnumerator();
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Sequence result;
+                if (!map.TryGetValue(this, out result))
+                {
+                    result = new Sequence(new Symbol[FlattenedSize()]);
+                    map[this] = result;
+                    IList<Fixup> l = new List<Fixup>();
+                    map2[result] = l;
+
+                    Flatten(Production, 0, result.Production, 0, map, map2);
+                    foreach (Fixup f in l)
+                    {
+                        Array.Copy(result.Production, 0, f.Symbols, f.Pos, result.Production.Length);
+                    }
+
+                    map2.Remove(result);
+                }
+
+                return result;
+            }
+
+            /// <inheritdoc />
+            public override int FlattenedSize()
+            {
+                return FlattenedSize(Production, 0);
+            }
+        }
+
+        /// <summary>
+        /// Repeater symbol.
+        /// </summary>
+        public class Repeater : Symbol
+        {
+            /// <summary>
+            /// The end symbol.
+            /// </summary>
+            public readonly Symbol End;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Repeater"/> class.
+            /// </summary>
+            public Repeater(Symbol end, params Symbol[] sequenceToRepeat) : base(Kind.Repeater,
+                makeProduction(sequenceToRepeat))
+            {
+                this.End = end;
+                Production[0] = this;
+            }
+
+            private static Symbol[] makeProduction(Symbol[] p)
+            {
+                Symbol[] result = new Symbol[p.Length + 1];
+                Array.Copy(p, 0, result, 1, p.Length);
+                return result;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Repeater result = new Repeater(End, new Symbol[FlattenedSize(Production, 1)]);
+                Flatten(Production, 1, result.Production, 1, map, map2);
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Returns true if the Parser contains any Error symbol, indicating that it may
+        /// fail for some inputs.
+        /// </summary>
+        private static bool hasErrors(Symbol symbol)
+        {
+            return hasErrors(symbol, new HashSet<Symbol>());
+        }
+
+        private static bool hasErrors(Symbol symbol, ISet<Symbol> visited)
+        {
+            // avoid infinite recursion
+            if (visited.Contains(symbol))
+            {
+                return false;
+            }
+
+            visited.Add(symbol);
+
+            switch (symbol.SymKind)
+            {
+                case Kind.Alternative:
+                    return hasErrors(symbol, ((Alternative)symbol).Symbols, visited);
+                case Kind.ExplicitAction:
+                    return false;
+                case Kind.ImplicitAction:
+                    if (symbol is ErrorAction)
+                    {
+                        return true;
+                    }
+
+                    if (symbol is UnionAdjustAction)
+                    {
+                        return hasErrors(((UnionAdjustAction)symbol).SymToParse, visited);
+                    }
+
+                    return false;
+                case Kind.Repeater:
+                    Repeater r = (Repeater)symbol;
+                    return hasErrors(r.End, visited) || hasErrors(symbol, r.Production, visited);
+                case Kind.Root:
+                case Kind.Sequence:
+                    return hasErrors(symbol, symbol.Production, visited);
+                case Kind.Terminal:
+                    return false;
+                default:
+                    throw new Exception("unknown symbol kind: " + symbol.SymKind);
+            }
+        }
+
+        private static bool hasErrors(Symbol root, Symbol[] symbols, ISet<Symbol> visited)
+        {
+            if (null != symbols)
+            {
+                foreach (Symbol s in symbols)
+                {
+                    if (s == root)
+                    {
+                        continue;
+                    }
+
+                    if (hasErrors(s, visited))
+                    {
+                        return true;
+                    }
+                }
+            }
+
+            return false;
+        }
+
+        /// <summary>
+        /// Alternative symbol.
+        /// </summary>
+        public class Alternative : Symbol
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public readonly Symbol[] Symbols;
+            /// <summary>
+            /// The labels.
+            /// </summary>
+            public readonly string[] Labels;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Alternative"/> class.
+            /// </summary>
+            public Alternative(Symbol[] symbols, string[] labels) : base(Kind.Alternative)
+            {
+                this.Symbols = symbols;
+                this.Labels = labels;
+            }
+
+            /// <summary>
+            /// Returns the symbol at the given index.
+            /// </summary>
+            public virtual Symbol GetSymbol(int index)
+            {
+                return Symbols[index];
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int index)
+            {
+                return Labels[index];
+            }
+
+            /// <summary>
+            /// Returns the size.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Symbols.Length;
+            }
+
+            /// <summary>
+            /// Returns the index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string label)
+            {
+                if (!ReferenceEquals(label, null))
+                {
+                    for (int i = 0; i < Labels.Length; i++)
+                    {
+                        if (label.Equals(Labels[i]))
+                        {
+                            return i;
+                        }
+                    }
+                }
+
+                return -1;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Symbol[] ss = new Symbol[Symbols.Length];
+                for (int i = 0; i < ss.Length; i++)
+                {
+                    ss[i] = Symbols[i].Flatten(map, map2);
+                }
+
+                return new Alternative(ss, Labels);
+            }
+        }
+
+        /// <summary>
+        /// The error action.
+        /// </summary>
+        public class ErrorAction : ImplicitAction
+        {
+            /// <summary>
+            /// The error message.
+            /// </summary>
+            public readonly string Msg;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ErrorAction"/> class.
+            /// </summary>
+            public ErrorAction(string msg)
+            {
+                this.Msg = msg;
+            }
+        }
+
+        /// <summary>
+        /// Int check action.
+        /// </summary>
+        public class IntCheckAction : Symbol
+        {
+            /// <summary>
+            /// The size.
+            /// </summary>
+            public readonly int Size;
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.IntCheckAction"/> class.
+            /// </summary>
+            public IntCheckAction(int size) : base(Kind.ExplicitAction)
+            {
+                this.Size = size;
+            }
+        }
+
+        /// <summary>
+        /// The enum adjust action.
+        /// </summary>
+        public class EnumAdjustAction : IntCheckAction

Review Comment:
   In the Java code, this class is used by `ResolvingGrammarGenerator`, which was the only class in the `parsing` package that was not ported, since it is not used by the JSON encoder/decoder.  Since `EnumAdjustAction` is not used, I will remove 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


[GitHub] [avro] rayokota commented on pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on PR #1833:
URL: https://github.com/apache/avro/pull/1833#issuecomment-1225811180

   Thanks for the approval @KalleOlaviNiemitalo !
   
   @martin-g @KyleSchoonover @zcsizmadia  Could one of you kindly merge this please?  I would greatly appreciate 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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r955011643


##########
lang/csharp/src/apache/main/IO/Encoder.cs:
##########
@@ -187,5 +187,10 @@ public interface Encoder
         /// <param name="start">Position within data where the contents start.</param>
         /// <param name="len">Number of bytes to write.</param>
         void WriteFixed(byte[] data, int start, int len);
+
+        /// <summary>
+        /// Flushes the encoder.
+        /// </summary>
+        void Flush();

Review Comment:
   Thanks @KalleOlaviNiemitalo 



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


[GitHub] [avro] rayokota commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
rayokota commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r956273784


##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,983 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public Kind SymKind { get; private set; }
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public Symbol[] Production { get; private set; }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            Production = production;
+            SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols) => new Root(symbols);
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production) => new Sequence(production);
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat) =>
+            new Repeater(endSymbol, symsToRepeat);
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels) => new Alternative(symbols, labels);
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e) => new ErrorAction(e);
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r) => new ResolvingAction(w, r);
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public int Pos { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2) => this;
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize() => 1;
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    if (!map2.TryGetValue((Sequence)s, out IList<Fixup> l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            CopyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void CopyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public string PrintName { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString() => PrintName;
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public bool IsTrailing { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, MakeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] MakeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol this[int index] => Production[index];
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index) => Production[index];
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size() => Production.Length;
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator() => Enumerable.Reverse(Production).GetEnumerator();
+
+            IEnumerator IEnumerable.GetEnumerator() => GetEnumerator();
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                if (!map.TryGetValue(this, out Sequence result))
+                {
+                    result = new Sequence(new Symbol[FlattenedSize()]);
+                    map[this] = result;
+                    IList<Fixup> l = new List<Fixup>();
+                    map2[result] = l;
+
+                    Flatten(Production, 0, result.Production, 0, map, map2);
+                    foreach (Fixup f in l)
+                    {
+                        Array.Copy(result.Production, 0, f.Symbols, f.Pos, result.Production.Length);
+                    }
+
+                    map2.Remove(result);
+                }
+
+                return result;
+            }
+
+            /// <inheritdoc />
+            public override int FlattenedSize() => FlattenedSize(Production, 0);
+        }
+
+        /// <summary>
+        /// Repeater symbol.
+        /// </summary>
+        public class Repeater : Symbol
+        {
+            /// <summary>
+            /// The end symbol.
+            /// </summary>
+            public Symbol End { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Repeater"/> class.
+            /// </summary>
+            public Repeater(Symbol end, params Symbol[] sequenceToRepeat) : base(Kind.Repeater,
+                MakeProduction(sequenceToRepeat))
+            {
+                End = end;
+                Production[0] = this;
+            }
+
+            private static Symbol[] MakeProduction(Symbol[] p)
+            {
+                Symbol[] result = new Symbol[p.Length + 1];
+                Array.Copy(p, 0, result, 1, p.Length);
+                return result;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Repeater result = new Repeater(End, new Symbol[FlattenedSize(Production, 1)]);
+                Flatten(Production, 1, result.Production, 1, map, map2);
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Returns true if the Parser contains any Error symbol, indicating that it may
+        /// fail for some inputs.
+        /// </summary>
+        private static bool HasErrors(Symbol symbol)
+        {
+            return HasErrors(symbol, new HashSet<Symbol>());
+        }
+
+        private static bool HasErrors(Symbol symbol, ISet<Symbol> visited)
+        {
+            // avoid infinite recursion
+            if (visited.Contains(symbol))
+            {
+                return false;
+            }
+
+            visited.Add(symbol);
+
+            switch (symbol.SymKind)
+            {
+                case Kind.Alternative:
+                    return HasErrors(symbol, ((Alternative)symbol).Symbols, visited);
+                case Kind.ExplicitAction:
+                    return false;
+                case Kind.ImplicitAction:
+                    if (symbol is ErrorAction)
+                    {
+                        return true;
+                    }
+
+                    if (symbol is UnionAdjustAction)
+                    {
+                        return HasErrors(((UnionAdjustAction)symbol).SymToParse, visited);
+                    }
+
+                    return false;
+                case Kind.Repeater:
+                    Repeater r = (Repeater)symbol;
+                    return HasErrors(r.End, visited) || HasErrors(symbol, r.Production, visited);
+                case Kind.Root:
+                case Kind.Sequence:
+                    return HasErrors(symbol, symbol.Production, visited);
+                case Kind.Terminal:
+                    return false;
+                default:
+                    throw new Exception("unknown symbol kind: " + symbol.SymKind);
+            }
+        }
+
+        private static bool HasErrors(Symbol root, Symbol[] symbols, ISet<Symbol> visited)
+        {
+            if (null != symbols)
+            {
+                foreach (Symbol s in symbols)
+                {
+                    if (s == root)
+                    {
+                        continue;
+                    }
+
+                    if (HasErrors(s, visited))
+                    {
+                        return true;
+                    }
+                }
+            }
+
+            return false;
+        }
+
+        /// <summary>
+        /// Alternative symbol.
+        /// </summary>
+        public class Alternative : Symbol
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols { get; private set; }
+
+            /// <summary>
+            /// The labels.
+            /// </summary>
+            public string[] Labels { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Alternative"/> class.
+            /// </summary>
+            public Alternative(Symbol[] symbols, string[] labels) : base(Kind.Alternative)
+            {
+                Symbols = symbols;
+                Labels = labels;
+            }
+
+            /// <summary>
+            /// Returns the symbol at the given index.
+            /// </summary>
+            public virtual Symbol GetSymbol(int index)
+            {
+                return Symbols[index];
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int index)
+            {
+                return Labels[index];
+            }
+
+            /// <summary>
+            /// Returns the size.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Symbols.Length;
+            }
+
+            /// <summary>
+            /// Returns the index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string label)
+            {
+                if (!ReferenceEquals(label, null))
+                {
+                    for (int i = 0; i < Labels.Length; i++)
+                    {
+                        if (label.Equals(Labels[i]))
+                        {
+                            return i;
+                        }
+                    }
+                }
+
+                return -1;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Symbol[] ss = new Symbol[Symbols.Length];
+                for (int i = 0; i < ss.Length; i++)
+                {
+                    ss[i] = Symbols[i].Flatten(map, map2);
+                }
+
+                return new Alternative(ss, Labels);
+            }
+        }
+
+        /// <summary>
+        /// The error action.
+        /// </summary>
+        public class ErrorAction : ImplicitAction
+        {
+            /// <summary>
+            /// The error message.
+            /// </summary>
+            public string Msg { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ErrorAction"/> class.
+            /// </summary>
+            public ErrorAction(string msg)
+            {
+                Msg = msg;
+            }
+        }
+
+        /// <summary>
+        /// Int check action.
+        /// </summary>
+        public class IntCheckAction : Symbol
+        {
+            /// <summary>
+            /// The size.
+            /// </summary>
+            public int Size { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.IntCheckAction"/> class.
+            /// </summary>
+            public IntCheckAction(int size) : base(Kind.ExplicitAction)
+            {
+                Size = size;
+            }
+        }
+
+        /// <summary>
+        /// The writer union action.
+        /// </summary>
+        public class WriterUnionAction : ImplicitAction
+        {
+        }
+
+        /// <summary>
+        /// The resolving action.
+        /// </summary>
+        public class ResolvingAction : ImplicitAction
+        {
+            /// <summary>
+            /// The writer.
+            /// </summary>
+            public Symbol Writer { get; private set; }
+
+            /// <summary>
+            /// The reader.
+            /// </summary>
+            public Symbol Reader { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ResolvingAction"/> class.
+            /// </summary>
+            public ResolvingAction(Symbol writer, Symbol reader)
+            {
+                Writer = writer;
+                Reader = reader;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new ResolvingAction(Writer.Flatten(map, map2), Reader.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The skip action.
+        /// </summary>
+        public class SkipAction : ImplicitAction
+        {
+            /// <summary>
+            /// The symbol to skip.
+            /// </summary>
+            public Symbol SymToSkip { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.SkipAction"/> class.
+            /// </summary>
+            public SkipAction(Symbol symToSkip) : base(true)
+            {
+                SymToSkip = symToSkip;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new SkipAction(SymToSkip.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The field adjust action.
+        /// </summary>
+        public class FieldAdjustAction : ImplicitAction
+        {
+            /// <summary>
+            /// The index.
+            /// </summary>
+            public int RIndex { get; private set; }
+
+            /// <summary>
+            /// The field name.
+            /// </summary>
+            public string FName { get; private set; }
+
+            /// <summary>
+            /// The field aliases.
+            /// </summary>
+            public IList<string> Aliases { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.FieldAdjustAction"/> class.
+            /// </summary>
+            public FieldAdjustAction(int rindex, string fname, IList<string> aliases)
+            {
+                RIndex = rindex;
+                FName = fname;
+                Aliases = aliases;
+            }
+        }
+
+        /// <summary>
+        /// THe field order action.
+        /// </summary>
+        public sealed class FieldOrderAction : ImplicitAction
+        {
+            /// <summary>
+            /// Whether no reorder is needed.
+            /// </summary>
+            public bool NoReorder { get; private set; }
+
+            /// <summary>
+            /// The fields.
+            /// </summary>
+            public Field[] Fields { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.FieldOrderAction"/> class.
+            /// </summary>
+            public FieldOrderAction(Field[] fields)
+            {
+                Fields = fields;
+                bool noReorder = true;
+                for (int i = 0; noReorder && i < fields.Length; i++)
+                {
+                    noReorder &= (i == fields[i].Pos);
+                }
+
+                NoReorder = noReorder;
+            }
+        }
+
+        /// <summary>
+        /// The default start action.
+        /// </summary>
+        public class DefaultStartAction : ImplicitAction
+        {
+            /// <summary>
+            /// The contents.
+            /// </summary>
+            public byte[] Contents { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.DefaultStartAction"/> class.
+            /// </summary>
+            public DefaultStartAction(byte[] contents)
+            {
+                Contents = contents;
+            }
+        }
+
+        /// <summary>
+        /// The union adjust action.
+        /// </summary>
+        public class UnionAdjustAction : ImplicitAction
+        {
+            /// <summary>
+            /// The index.
+            /// </summary>
+            public int RIndex { get; private set; }
+
+            /// <summary>
+            /// The symbol to parser.
+            /// </summary>
+            public Symbol SymToParse { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.UnionAdjustAction"/> class.
+            /// </summary>
+            public UnionAdjustAction(int rindex, Symbol symToParse)
+            {
+                RIndex = rindex;
+                SymToParse = symToParse;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new UnionAdjustAction(RIndex, SymToParse.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The enum labels action.
+        /// </summary>
+        public class EnumLabelsAction : IntCheckAction
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public IList<string> Symbols { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.EnumLabelsAction"/> class.
+            /// </summary>
+            public EnumLabelsAction(IList<string> symbols) : base(symbols.Count)
+            {
+                Symbols = symbols;
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int n)
+            {
+                return Symbols[n];
+            }
+
+            /// <summary>
+            /// Returns index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string l)
+            {
+                if (!ReferenceEquals(l, null))
+                {
+                    for (int i = 0; i < Symbols.Count; i++)
+                    {
+                        if (l.Equals(Symbols[i]))
+                        {
+                            return i;
+                        }
+                    }
+                }
+
+                return -1;
+            }
+        }
+
+        /// <summary>
+        /// The terminal symbols for the grammar.
+        /// </summary>
+        public static readonly Symbol Null = new Terminal("null");

Review Comment:
   Thanks, I went with `{ get; } =` as the code does `==` checks on the symbols.



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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r956252918


##########
lang/csharp/src/apache/main/IO/Parsing/Symbol.cs:
##########
@@ -0,0 +1,983 @@
+/*
+ * 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.
+ */
+
+using System;
+using System.Collections;
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Avro.IO.Parsing
+{
+    /// <summary>
+    /// Symbol is the base of all symbols (terminals and non-terminals) of the
+    /// grammar.
+    /// </summary>
+    public abstract class Symbol
+    {
+        /// <summary>
+        /// The type of symbol.
+        /// </summary>
+        public enum Kind
+        {
+            /// <summary>
+            /// terminal symbols which have no productions </summary>
+            Terminal,
+
+            /// <summary>
+            /// Start symbol for some grammar </summary>
+            Root,
+
+            /// <summary>
+            /// non-terminal symbol which is a sequence of one or more other symbols </summary>
+            Sequence,
+
+            /// <summary>
+            /// non-terminal to represent the contents of an array or map </summary>
+            Repeater,
+
+            /// <summary>
+            /// non-terminal to represent the union </summary>
+            Alternative,
+
+            /// <summary>
+            /// non-terminal action symbol which are automatically consumed </summary>
+            ImplicitAction,
+
+            /// <summary>
+            /// non-terminal action symbol which is explicitly consumed </summary>
+            ExplicitAction
+        }
+
+        /// The kind of this symbol.
+        public Kind SymKind { get; private set; }
+
+        /// <summary>
+        /// The production for this symbol. If this symbol is a terminal this is
+        /// <tt>null</tt>. Otherwise this holds the the sequence of the symbols that
+        /// forms the production for this symbol. The sequence is in the reverse order of
+        /// production. This is useful for easy copying onto parsing stack.
+        ///
+        /// Please note that this is a final. So the production for a symbol should be
+        /// known before that symbol is constructed. This requirement cannot be met for
+        /// those symbols which are recursive (e.g. a record that holds union a branch of
+        /// which is the record itself). To resolve this problem, we initialize the
+        /// symbol with an array of nulls. Later we fill the symbols. Not clean, but
+        /// works. The other option is to not have this field a final. But keeping it
+        /// final and thus keeping symbol immutable gives some comfort. See various
+        /// generators how we generate records.
+        /// </summary>
+        public Symbol[] Production { get; private set; }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind.
+        /// </summary>
+        protected Symbol(Kind kind) : this(kind, null)
+        {
+        }
+
+        /// <summary>
+        /// Constructs a new symbol of the given kind and production.
+        /// </summary>
+        protected Symbol(Kind kind, Symbol[] production)
+        {
+            Production = production;
+            SymKind = kind;
+        }
+
+        /// <summary>
+        /// A convenience method to construct a root symbol.
+        /// </summary>
+        public static Symbol NewRoot(params Symbol[] symbols) => new Root(symbols);
+
+        /// <summary>
+        /// A convenience method to construct a sequence.
+        /// </summary>
+        /// <param name="production"> The constituent symbols of the sequence. </param>
+        public static Symbol NewSeq(params Symbol[] production) => new Sequence(production);
+
+        /// <summary>
+        /// A convenience method to construct a repeater.
+        /// </summary>
+        /// <param name="endSymbol"> The end symbol. </param>
+        /// <param name="symsToRepeat"> The symbols to repeat in the repeater. </param>
+        public static Symbol NewRepeat(Symbol endSymbol, params Symbol[] symsToRepeat) =>
+            new Repeater(endSymbol, symsToRepeat);
+
+        /// <summary>
+        /// A convenience method to construct a union.
+        /// </summary>
+        public static Symbol NewAlt(Symbol[] symbols, string[] labels) => new Alternative(symbols, labels);
+
+        /// <summary>
+        /// A convenience method to construct an ErrorAction.
+        /// </summary>
+        /// <param name="e"> </param>
+        protected static Symbol Error(string e) => new ErrorAction(e);
+
+        /// <summary>
+        /// A convenience method to construct a ResolvingAction.
+        /// </summary>
+        /// <param name="w"> The writer symbol </param>
+        /// <param name="r"> The reader symbol </param>
+        protected static Symbol Resolve(Symbol w, Symbol r) => new ResolvingAction(w, r);
+
+        /// <summary>
+        /// Fixup symbol.
+        /// </summary>
+        protected class Fixup
+        {
+            private readonly Symbol[] symbols;
+
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols
+            {
+                get { return (Symbol[])symbols.Clone(); }
+            }
+
+            /// <summary>
+            /// The position.
+            /// </summary>
+            public int Pos { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Fixup"/> class.
+            /// </summary>
+            public Fixup(Symbol[] symbols, int pos)
+            {
+                this.symbols = (Symbol[])symbols.Clone();
+                Pos = pos;
+            }
+        }
+
+        /// <summary>
+        /// Flatten the given sub-array of symbols into a sub-array of symbols.
+        /// </summary>
+        protected virtual Symbol Flatten(IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2) => this;
+
+        /// <summary>
+        /// Returns the flattened size.
+        /// </summary>
+        public virtual int FlattenedSize() => 1;
+
+        /// <summary>
+        /// Flattens the given sub-array of symbols into an sub-array of symbols. Every
+        /// <tt>Sequence</tt> in the input are replaced by its production recursively.
+        /// Non-<tt>Sequence</tt> symbols, they internally have other symbols those
+        /// internal symbols also get flattened. When flattening is done, the only place
+        /// there might be Sequence symbols is in the productions of a Repeater,
+        /// Alternative, or the symToParse and symToSkip in a UnionAdjustAction or
+        /// SkipAction.
+        ///
+        /// Why is this done? We want our parsers to be fast. If we left the grammars
+        /// unflattened, then the parser would be constantly copying the contents of
+        /// nested Sequence productions onto the parsing stack. Instead, because of
+        /// flattening, we have a long top-level production with no Sequences unless the
+        /// Sequence is absolutely needed, e.g., in the case of a Repeater or an
+        /// Alternative.
+        ///
+        /// Well, this is not exactly true when recursion is involved. Where there is a
+        /// recursive record, that record will be "inlined" once, but any internal (ie,
+        /// recursive) references to that record will be a Sequence for the record. That
+        /// Sequence will not further inline itself -- it will refer to itself as a
+        /// Sequence. The same is true for any records nested in this outer recursive
+        /// record. Recursion is rare, and we want things to be fast in the typical case,
+        /// which is why we do the flattening optimization.
+        ///
+        ///
+        /// The algorithm does a few tricks to handle recursive symbol definitions. In
+        /// order to avoid infinite recursion with recursive symbols, we have a map of
+        /// Symbol->Symbol. Before fully constructing a flattened symbol for a
+        /// <tt>Sequence</tt> we insert an empty output symbol into the map and then
+        /// start filling the production for the <tt>Sequence</tt>. If the same
+        /// <tt>Sequence</tt> is encountered due to recursion, we simply return the
+        /// (empty) output <tt>Sequence</tt> from the map. Then we actually fill out
+        /// the production for the <tt>Sequence</tt>. As part of the flattening process
+        /// we copy the production of <tt>Sequence</tt>s into larger arrays. If the
+        /// original <tt>Sequence</tt> has not not be fully constructed yet, we copy a
+        /// bunch of <tt>null</tt>s. Fix-up remembers all those <tt>null</tt> patches.
+        /// The fix-ups gets finally filled when we know the symbols to occupy those
+        /// patches.
+        /// </summary>
+        /// <param name="input">    The array of input symbols to flatten </param>
+        /// <param name="start"> The position where the input sub-array starts. </param>
+        /// <param name="output">   The output that receives the flattened list of symbols. The
+        ///              output array should have sufficient space to receive the
+        ///              expanded sub-array of symbols. </param>
+        /// <param name="skip">  The position where the output input sub-array starts. </param>
+        /// <param name="map">   A map of symbols which have already been expanded. Useful for
+        ///              handling recursive definitions and for caching. </param>
+        /// <param name="map2">  A map to to store the list of fix-ups. </param>
+        protected static void Flatten(Symbol[] input, int start, Symbol[] output, int skip,
+            IDictionary<Sequence, Sequence> map, IDictionary<Sequence, IList<Fixup>> map2)
+        {
+            for (int i = start, j = skip; i < input.Length; i++)
+            {
+                Symbol s = input[i].Flatten(map, map2);
+                if (s is Sequence)
+                {
+                    Symbol[] p = s.Production;
+                    if (!map2.TryGetValue((Sequence)s, out IList<Fixup> l))
+                    {
+                        Array.Copy(p, 0, output, j, p.Length);
+                        // Copy any fixups that will be applied to p to add missing symbols
+                        foreach (IList<Fixup> fixups in map2.Values)
+                        {
+                            CopyFixups(fixups, output, j, p);
+                        }
+                    }
+                    else
+                    {
+                        l.Add(new Fixup(output, j));
+                    }
+
+                    j += p.Length;
+                }
+                else
+                {
+                    output[j++] = s;
+                }
+            }
+        }
+
+        private static void CopyFixups(IList<Fixup> fixups, Symbol[] output, int outPos, Symbol[] toCopy)
+        {
+            for (int i = 0, n = fixups.Count; i < n; i += 1)
+            {
+                Fixup fixup = fixups[i];
+                if (fixup.Symbols == toCopy)
+                {
+                    fixups.Add(new Fixup(output, fixup.Pos + outPos));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Returns the amount of space required to flatten the given sub-array of
+        /// symbols.
+        /// </summary>
+        /// <param name="symbols"> The array of input symbols. </param>
+        /// <param name="start">   The index where the subarray starts. </param>
+        /// <returns> The number of symbols that will be produced if one expands the given
+        ///         input. </returns>
+        protected static int FlattenedSize(Symbol[] symbols, int start)
+        {
+            int result = 0;
+            for (int i = start; i < symbols.Length; i++)
+            {
+                if (symbols[i] is Sequence)
+                {
+                    Sequence s = (Sequence)symbols[i];
+                    result += s.FlattenedSize();
+                }
+                else
+                {
+                    result += 1;
+                }
+            }
+
+            return result;
+        }
+
+        /// <summary>
+        /// Terminal symbol.
+        /// </summary>
+        protected class Terminal : Symbol
+        {
+            /// <summary>
+            /// Printable name.
+            /// </summary>
+            public string PrintName { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Terminal"/> class.
+            /// </summary>
+            public Terminal(string printName) : base(Kind.Terminal)
+            {
+                PrintName = printName;
+            }
+
+            /// <inheritdoc />
+            public override string ToString() => PrintName;
+        }
+
+        /// <summary>
+        /// Implicit action.
+        /// </summary>
+        public class ImplicitAction : Symbol
+        {
+            /// <summary>
+            /// Set to <tt>true</tt> if and only if this implicit action is a trailing
+            /// action. That is, it is an action that follows real symbol. E.g
+            /// <see cref="Symbol.DefaultEndAction"/>.
+            /// </summary>
+            public bool IsTrailing { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction() : this(false)
+            {
+            }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ImplicitAction"/> class.
+            /// </summary>
+            public ImplicitAction(bool isTrailing) : base(Kind.ImplicitAction)
+            {
+                IsTrailing = isTrailing;
+            }
+        }
+
+        /// <summary>
+        /// Root symbol.
+        /// </summary>
+        protected class Root : Symbol
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Root"/> class.
+            /// </summary>
+            public Root(params Symbol[] symbols) : base(Kind.Root, MakeProduction(symbols))
+            {
+                Production[0] = this;
+            }
+
+            private static Symbol[] MakeProduction(Symbol[] symbols)
+            {
+                Symbol[] result = new Symbol[FlattenedSize(symbols, 0) + 1];
+                Flatten(symbols, 0, result, 1, new Dictionary<Sequence, Sequence>(),
+                    new Dictionary<Sequence, IList<Fixup>>());
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Sequence symbol.
+        /// </summary>
+        protected class Sequence : Symbol, IEnumerable<Symbol>
+        {
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Sequence"/> class.
+            /// </summary>
+            public Sequence(Symbol[] productions) : base(Kind.Sequence, productions)
+            {
+            }
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol this[int index] => Production[index];
+
+            /// <summary>
+            /// Get the symbol at the given index.
+            /// </summary>
+            public virtual Symbol Get(int index) => Production[index];
+
+            /// <summary>
+            /// Returns the number of symbols.
+            /// </summary>
+            public virtual int Size() => Production.Length;
+
+            /// <inheritdoc />
+            public IEnumerator<Symbol> GetEnumerator() => Enumerable.Reverse(Production).GetEnumerator();
+
+            IEnumerator IEnumerable.GetEnumerator() => GetEnumerator();
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                if (!map.TryGetValue(this, out Sequence result))
+                {
+                    result = new Sequence(new Symbol[FlattenedSize()]);
+                    map[this] = result;
+                    IList<Fixup> l = new List<Fixup>();
+                    map2[result] = l;
+
+                    Flatten(Production, 0, result.Production, 0, map, map2);
+                    foreach (Fixup f in l)
+                    {
+                        Array.Copy(result.Production, 0, f.Symbols, f.Pos, result.Production.Length);
+                    }
+
+                    map2.Remove(result);
+                }
+
+                return result;
+            }
+
+            /// <inheritdoc />
+            public override int FlattenedSize() => FlattenedSize(Production, 0);
+        }
+
+        /// <summary>
+        /// Repeater symbol.
+        /// </summary>
+        public class Repeater : Symbol
+        {
+            /// <summary>
+            /// The end symbol.
+            /// </summary>
+            public Symbol End { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Repeater"/> class.
+            /// </summary>
+            public Repeater(Symbol end, params Symbol[] sequenceToRepeat) : base(Kind.Repeater,
+                MakeProduction(sequenceToRepeat))
+            {
+                End = end;
+                Production[0] = this;
+            }
+
+            private static Symbol[] MakeProduction(Symbol[] p)
+            {
+                Symbol[] result = new Symbol[p.Length + 1];
+                Array.Copy(p, 0, result, 1, p.Length);
+                return result;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Repeater result = new Repeater(End, new Symbol[FlattenedSize(Production, 1)]);
+                Flatten(Production, 1, result.Production, 1, map, map2);
+                return result;
+            }
+        }
+
+        /// <summary>
+        /// Returns true if the Parser contains any Error symbol, indicating that it may
+        /// fail for some inputs.
+        /// </summary>
+        private static bool HasErrors(Symbol symbol)
+        {
+            return HasErrors(symbol, new HashSet<Symbol>());
+        }
+
+        private static bool HasErrors(Symbol symbol, ISet<Symbol> visited)
+        {
+            // avoid infinite recursion
+            if (visited.Contains(symbol))
+            {
+                return false;
+            }
+
+            visited.Add(symbol);
+
+            switch (symbol.SymKind)
+            {
+                case Kind.Alternative:
+                    return HasErrors(symbol, ((Alternative)symbol).Symbols, visited);
+                case Kind.ExplicitAction:
+                    return false;
+                case Kind.ImplicitAction:
+                    if (symbol is ErrorAction)
+                    {
+                        return true;
+                    }
+
+                    if (symbol is UnionAdjustAction)
+                    {
+                        return HasErrors(((UnionAdjustAction)symbol).SymToParse, visited);
+                    }
+
+                    return false;
+                case Kind.Repeater:
+                    Repeater r = (Repeater)symbol;
+                    return HasErrors(r.End, visited) || HasErrors(symbol, r.Production, visited);
+                case Kind.Root:
+                case Kind.Sequence:
+                    return HasErrors(symbol, symbol.Production, visited);
+                case Kind.Terminal:
+                    return false;
+                default:
+                    throw new Exception("unknown symbol kind: " + symbol.SymKind);
+            }
+        }
+
+        private static bool HasErrors(Symbol root, Symbol[] symbols, ISet<Symbol> visited)
+        {
+            if (null != symbols)
+            {
+                foreach (Symbol s in symbols)
+                {
+                    if (s == root)
+                    {
+                        continue;
+                    }
+
+                    if (HasErrors(s, visited))
+                    {
+                        return true;
+                    }
+                }
+            }
+
+            return false;
+        }
+
+        /// <summary>
+        /// Alternative symbol.
+        /// </summary>
+        public class Alternative : Symbol
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public Symbol[] Symbols { get; private set; }
+
+            /// <summary>
+            /// The labels.
+            /// </summary>
+            public string[] Labels { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.Alternative"/> class.
+            /// </summary>
+            public Alternative(Symbol[] symbols, string[] labels) : base(Kind.Alternative)
+            {
+                Symbols = symbols;
+                Labels = labels;
+            }
+
+            /// <summary>
+            /// Returns the symbol at the given index.
+            /// </summary>
+            public virtual Symbol GetSymbol(int index)
+            {
+                return Symbols[index];
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int index)
+            {
+                return Labels[index];
+            }
+
+            /// <summary>
+            /// Returns the size.
+            /// </summary>
+            public virtual int Size()
+            {
+                return Symbols.Length;
+            }
+
+            /// <summary>
+            /// Returns the index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string label)
+            {
+                if (!ReferenceEquals(label, null))
+                {
+                    for (int i = 0; i < Labels.Length; i++)
+                    {
+                        if (label.Equals(Labels[i]))
+                        {
+                            return i;
+                        }
+                    }
+                }
+
+                return -1;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                Symbol[] ss = new Symbol[Symbols.Length];
+                for (int i = 0; i < ss.Length; i++)
+                {
+                    ss[i] = Symbols[i].Flatten(map, map2);
+                }
+
+                return new Alternative(ss, Labels);
+            }
+        }
+
+        /// <summary>
+        /// The error action.
+        /// </summary>
+        public class ErrorAction : ImplicitAction
+        {
+            /// <summary>
+            /// The error message.
+            /// </summary>
+            public string Msg { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ErrorAction"/> class.
+            /// </summary>
+            public ErrorAction(string msg)
+            {
+                Msg = msg;
+            }
+        }
+
+        /// <summary>
+        /// Int check action.
+        /// </summary>
+        public class IntCheckAction : Symbol
+        {
+            /// <summary>
+            /// The size.
+            /// </summary>
+            public int Size { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.IntCheckAction"/> class.
+            /// </summary>
+            public IntCheckAction(int size) : base(Kind.ExplicitAction)
+            {
+                Size = size;
+            }
+        }
+
+        /// <summary>
+        /// The writer union action.
+        /// </summary>
+        public class WriterUnionAction : ImplicitAction
+        {
+        }
+
+        /// <summary>
+        /// The resolving action.
+        /// </summary>
+        public class ResolvingAction : ImplicitAction
+        {
+            /// <summary>
+            /// The writer.
+            /// </summary>
+            public Symbol Writer { get; private set; }
+
+            /// <summary>
+            /// The reader.
+            /// </summary>
+            public Symbol Reader { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.ResolvingAction"/> class.
+            /// </summary>
+            public ResolvingAction(Symbol writer, Symbol reader)
+            {
+                Writer = writer;
+                Reader = reader;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new ResolvingAction(Writer.Flatten(map, map2), Reader.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The skip action.
+        /// </summary>
+        public class SkipAction : ImplicitAction
+        {
+            /// <summary>
+            /// The symbol to skip.
+            /// </summary>
+            public Symbol SymToSkip { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.SkipAction"/> class.
+            /// </summary>
+            public SkipAction(Symbol symToSkip) : base(true)
+            {
+                SymToSkip = symToSkip;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new SkipAction(SymToSkip.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The field adjust action.
+        /// </summary>
+        public class FieldAdjustAction : ImplicitAction
+        {
+            /// <summary>
+            /// The index.
+            /// </summary>
+            public int RIndex { get; private set; }
+
+            /// <summary>
+            /// The field name.
+            /// </summary>
+            public string FName { get; private set; }
+
+            /// <summary>
+            /// The field aliases.
+            /// </summary>
+            public IList<string> Aliases { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.FieldAdjustAction"/> class.
+            /// </summary>
+            public FieldAdjustAction(int rindex, string fname, IList<string> aliases)
+            {
+                RIndex = rindex;
+                FName = fname;
+                Aliases = aliases;
+            }
+        }
+
+        /// <summary>
+        /// THe field order action.
+        /// </summary>
+        public sealed class FieldOrderAction : ImplicitAction
+        {
+            /// <summary>
+            /// Whether no reorder is needed.
+            /// </summary>
+            public bool NoReorder { get; private set; }
+
+            /// <summary>
+            /// The fields.
+            /// </summary>
+            public Field[] Fields { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.FieldOrderAction"/> class.
+            /// </summary>
+            public FieldOrderAction(Field[] fields)
+            {
+                Fields = fields;
+                bool noReorder = true;
+                for (int i = 0; noReorder && i < fields.Length; i++)
+                {
+                    noReorder &= (i == fields[i].Pos);
+                }
+
+                NoReorder = noReorder;
+            }
+        }
+
+        /// <summary>
+        /// The default start action.
+        /// </summary>
+        public class DefaultStartAction : ImplicitAction
+        {
+            /// <summary>
+            /// The contents.
+            /// </summary>
+            public byte[] Contents { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.DefaultStartAction"/> class.
+            /// </summary>
+            public DefaultStartAction(byte[] contents)
+            {
+                Contents = contents;
+            }
+        }
+
+        /// <summary>
+        /// The union adjust action.
+        /// </summary>
+        public class UnionAdjustAction : ImplicitAction
+        {
+            /// <summary>
+            /// The index.
+            /// </summary>
+            public int RIndex { get; private set; }
+
+            /// <summary>
+            /// The symbol to parser.
+            /// </summary>
+            public Symbol SymToParse { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.UnionAdjustAction"/> class.
+            /// </summary>
+            public UnionAdjustAction(int rindex, Symbol symToParse)
+            {
+                RIndex = rindex;
+                SymToParse = symToParse;
+            }
+
+            /// <inheritdoc />
+            protected override Symbol Flatten(IDictionary<Sequence, Sequence> map,
+                IDictionary<Sequence, IList<Fixup>> map2)
+            {
+                return new UnionAdjustAction(RIndex, SymToParse.Flatten(map, map2));
+            }
+        }
+
+        /// <summary>
+        /// The enum labels action.
+        /// </summary>
+        public class EnumLabelsAction : IntCheckAction
+        {
+            /// <summary>
+            /// The symbols.
+            /// </summary>
+            public IList<string> Symbols { get; private set; }
+
+            /// <summary>
+            /// Initializes a new instance of the <see cref="Symbol.EnumLabelsAction"/> class.
+            /// </summary>
+            public EnumLabelsAction(IList<string> symbols) : base(symbols.Count)
+            {
+                Symbols = symbols;
+            }
+
+            /// <summary>
+            /// Returns the label at the given index.
+            /// </summary>
+            public virtual string GetLabel(int n)
+            {
+                return Symbols[n];
+            }
+
+            /// <summary>
+            /// Returns index of the given label.
+            /// </summary>
+            public virtual int FindLabel(string l)
+            {
+                if (!ReferenceEquals(l, null))
+                {
+                    for (int i = 0; i < Symbols.Count; i++)
+                    {
+                        if (l.Equals(Symbols[i]))
+                        {
+                            return i;
+                        }
+                    }
+                }
+
+                return -1;
+            }
+        }
+
+        /// <summary>
+        /// The terminal symbols for the grammar.
+        /// </summary>
+        public static readonly Symbol Null = new Terminal("null");

Review Comment:
   How about `public static Symbol Null { get; } = new Terminal("null");` so it constructs only once



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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #1833: AVRO-3001 AVRO-3274 AVRO-3568 AVRO-3613: Add JSON encoder/decoder for C#

Posted by GitBox <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #1833:
URL: https://github.com/apache/avro/pull/1833#discussion_r958289034


##########
lang/csharp/src/apache/main/IO/Encoder.cs:
##########
@@ -187,5 +187,10 @@ public interface Encoder
         /// <param name="start">Position within data where the contents start.</param>
         /// <param name="len">Number of bytes to write.</param>
         void WriteFixed(byte[] data, int start, int len);
+
+        /// <summary>
+        /// Flushes the encoder.
+        /// </summary>
+        void Flush();

Review Comment:
   That seems right



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