You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by th...@apache.org on 2012/05/16 09:58:46 UTC

svn commit: r1339045 - in /avro/trunk: ./ lang/csharp/src/apache/main/ lang/csharp/src/apache/main/Schema/ lang/csharp/src/apache/test/ lang/csharp/src/apache/test/Schema/ lang/csharp/src/apache/test/Utils/

Author: thiru
Date: Wed May 16 07:58:45 2012
New Revision: 1339045

URL: http://svn.apache.org/viewvc?rev=1339045&view=rev
Log:
AVRO-1085. Fingerprinting for C#

Added:
    avro/trunk/lang/csharp/src/apache/main/Schema/SchemaNormalization.cs
    avro/trunk/lang/csharp/src/apache/test/Schema/SchemaNormalizationTests.cs
    avro/trunk/lang/csharp/src/apache/test/Utils/
    avro/trunk/lang/csharp/src/apache/test/Utils/CaseFinder.cs
    avro/trunk/lang/csharp/src/apache/test/Utils/CaseFinderTests.cs
Modified:
    avro/trunk/CHANGES.txt
    avro/trunk/lang/csharp/src/apache/main/Avro.main.csproj
    avro/trunk/lang/csharp/src/apache/main/Schema/Schema.cs
    avro/trunk/lang/csharp/src/apache/main/Schema/UnionSchema.cs
    avro/trunk/lang/csharp/src/apache/test/Avro.test.csproj

Modified: avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/avro/trunk/CHANGES.txt?rev=1339045&r1=1339044&r2=1339045&view=diff
==============================================================================
--- avro/trunk/CHANGES.txt (original)
+++ avro/trunk/CHANGES.txt Wed May 16 07:58:45 2012
@@ -27,6 +27,8 @@ Avro 1.7.0 (unreleased)
 
     AVRO-1075. Add some Eclipse stuff to .gitignore.  (Karthik K via cutting)
 
+    AVRO-1085. Fingerprinting for C#. (Eric Hauser via thiru)
+
   BUG FIXES
 
     AVRO-1045. Java: Fix a bug in GenericData#deepCopy() of ByteBuffer values.

Modified: avro/trunk/lang/csharp/src/apache/main/Avro.main.csproj
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/Avro.main.csproj?rev=1339045&r1=1339044&r2=1339045&view=diff
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/Avro.main.csproj (original)
+++ avro/trunk/lang/csharp/src/apache/main/Avro.main.csproj Wed May 16 07:58:45 2012
@@ -118,6 +118,7 @@
     <Compile Include="Schema\RecordSchema.cs" />
     <Compile Include="Schema\Schema.cs" />
     <Compile Include="Schema\SchemaName.cs" />
+    <Compile Include="Schema\SchemaNormalization.cs" />
     <Compile Include="Schema\SchemaParseException.cs" />
     <Compile Include="Schema\UnionSchema.cs" />
     <Compile Include="Schema\UnnamedSchema.cs" />
@@ -152,4 +153,4 @@
   <Target Name="AfterBuild">
   </Target>
   -->
-</Project>
+</Project>
\ No newline at end of file

Modified: avro/trunk/lang/csharp/src/apache/main/Schema/Schema.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/Schema/Schema.cs?rev=1339045&r1=1339044&r2=1339045&view=diff
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/Schema/Schema.cs (original)
+++ avro/trunk/lang/csharp/src/apache/main/Schema/Schema.cs Wed May 16 07:58:45 2012
@@ -215,7 +215,7 @@ namespace Avro
         {
             writer.WriteStartObject();
             writer.WritePropertyName("type");
-            writer.WriteValue(getTypeString(this.Tag));
+            writer.WriteValue(GetTypeString(this.Tag));
         }
 
         /// <summary>
@@ -223,7 +223,7 @@ namespace Avro
         /// </summary>
         /// <param name="type">schema type</param>
         /// <returns>symbol name</returns>
-        protected static string getTypeString(Type type)
+        public static string GetTypeString(Type type)
         {
             if (type != Type.Enumeration) return type.ToString().ToLower();
             return "enum";

Added: avro/trunk/lang/csharp/src/apache/main/Schema/SchemaNormalization.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/Schema/SchemaNormalization.cs?rev=1339045&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/Schema/SchemaNormalization.cs (added)
+++ avro/trunk/lang/csharp/src/apache/main/Schema/SchemaNormalization.cs Wed May 16 07:58:45 2012
@@ -0,0 +1,256 @@
+/**
+ * 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
+ *
+ *     http://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.Collections.Generic;
+using System.Text;
+using System;
+
+namespace Avro
+{
+    /// <summary>
+    /// Collection of static methods for generating the cannonical form of schemas.
+    /// </summary>
+    public static class SchemaNormalization
+    {
+        public static long Empty64 = -4513414715797952619;
+
+        /// <summary>
+        /// Parses a schema into the canonical form as defined by Avro spec.
+        /// </summary>
+        /// <param name="s">Schema</param>
+        /// <returns>Parsing Canonical Form of a schema as defined by Avro spec.</returns>
+        public static string ToParsingForm(Schema s)
+        {
+            IDictionary<string, string> env = new Dictionary<string, string>();
+            return Build(env, s, new StringBuilder()).ToString();
+        }
+
+        /// <summary>
+        /// <para>Returns a fingerprint of a string of bytes. This string is
+        /// presumed to contain a canonical form of a schema. The
+        /// algorithm used to compute the fingerprint is selected by the
+        /// argument <i>fpName</i>.
+        /// </para>
+        /// <para>If <i>fpName</i> equals the string
+        /// <code>"CRC-64-AVRO"</code>, then the result of <see cref="Fingerprint64(byte[])"/> is
+        /// returned in little-endian format.
+        /// </para>
+        /// <para>If <i>fpName</i> equals the string
+        /// <code>"MD5"</code>, then the standard MD5 algorithm is used.
+        /// </para>
+        /// <para>If <i>fpName</i> equals the string
+        /// <code>"SHA-256"</code>, then the standard SHA-256 algorithm is used.
+        /// </para>
+        /// <para>Otherwise, <i>fpName</i> is
+        /// not recognized and an
+        /// <code>ArgumentException</code> is thrown
+        /// </para>
+        /// <para> Recommended Avro practice dictiates that
+        /// <code>"CRC-64-AVRO"</code> is used for 64-bit fingerprints,
+        /// <code>"MD5"</code> is used for 128-bit fingerprints, and
+        /// <code>"SHA-256"</code> is used for 256-bit fingerprints.
+        /// </para>
+        /// </summary>
+        /// <param name="fpName">Name of the hashing algorithm.</param>
+        /// <param name="data">Data to be hashed.</param>
+        /// <returns>Fingerprint</returns>
+        public static byte[] Fingerprint(string fpName, byte[] data)
+        {
+            switch (fpName)
+            {
+                case "CRC-64-AVRO":
+                    long fp = Fingerprint64(data);
+                    byte[] result = new byte[8];
+                    for (int i = 0; i < 8; i++)
+                    {
+                        result[i] = (byte) fp;
+                        fp >>= 8;
+                    }
+                    return result;
+                case "MD5":
+                    var md5 = System.Security.Cryptography.MD5.Create();
+                    return md5.ComputeHash(data);
+                case "SHA-256":
+                    var sha256 = System.Security.Cryptography.SHA256.Create();
+                    return sha256.ComputeHash(data);
+                default:
+                    throw new ArgumentException(string.Format("Unsupported fingerprint computation algorithm ({0})", fpName));
+            }
+        }
+
+        /// <summary>
+        /// Returns <see cref="Fingerprint(string, byte[])"/> applied to the parsing canonical form of the supplied schema.
+        /// </summary>
+        /// <param name="fpName">Name of the hashing algorithm.</param>
+        /// <param name="s">Schema to be hashed.</param>
+        /// <returns>Fingerprint</returns>
+        public static byte[] ParsingFingerprint(string fpName, Schema s)
+        {
+            return Fingerprint(fpName, Encoding.UTF8.GetBytes(ToParsingForm(s)));
+        }
+
+        /// <summary>
+        /// Returns <see cref="Fingerprint64(byte[])"/> applied to the parsing canonical form of the supplied schema.
+        /// </summary>
+        /// <param name="s">Schema to be hashed.</param>
+        /// <returns>Fingerprint</returns>
+        public static long ParsingFingerprint64(Schema s)
+        {
+            return Fingerprint64(Encoding.UTF8.GetBytes(ToParsingForm(s)));
+        }
+
+        /// <summary>
+        /// Computes the 64-bit Rabin Fingerprint (as recommended in the Avro spec) of a byte string.
+        /// </summary>
+        /// <param name="data">Data to be hashed.</param>
+        /// <returns>Fingerprint</returns>
+        private static long Fingerprint64(byte[] data)
+        {
+            long result = Empty64;
+            foreach (var b in data)
+            {
+                result = ((long)(((ulong)result) >> 8)) ^ Fp64.FpTable[(int) (result ^ b) & 0xff];
+            }
+            return result;
+        }
+
+        private static StringBuilder Build(IDictionary<string, string> env, Schema s, StringBuilder o)
+        {
+            bool firstTime = true;
+            Schema.Type st = s.Tag;
+            switch (st)
+            {
+                case Schema.Type.Union:
+                    UnionSchema us = s as UnionSchema;
+                    o.Append('[');
+                    foreach(Schema b in us.Schemas)
+                    {
+                        if (!firstTime)
+                        {
+                            o.Append(",");
+                        }
+                        else
+                        {
+                            firstTime = false;
+                        }
+                        Build(env, b, o);
+                    }
+                    return o.Append(']');
+
+                case Schema.Type.Array:
+                case Schema.Type.Map:
+                    o.Append("{\"type\":\"").Append(Schema.GetTypeString(s.Tag)).Append("\"");
+                    if (st == Schema.Type.Array)
+                    {
+                        ArraySchema arraySchema  = s as ArraySchema;
+                        Build(env, arraySchema.ItemSchema, o.Append(",\"items\":"));
+                    }
+                    else
+                    {
+                        MapSchema mapSchema = s as MapSchema;
+                        Build(env, mapSchema.ValueSchema, o.Append(",\"values\":"));
+                    }
+                    return o.Append("}");
+
+                case Schema.Type.Enumeration:
+                case Schema.Type.Fixed:
+                case Schema.Type.Record:
+                    NamedSchema namedSchema = s as NamedSchema;
+                    var name = namedSchema.Fullname;
+                    if (env.ContainsKey(name))
+                    {
+                        return o.Append(env[name]);
+                    }
+                    var qname = "\"" + name + "\"";
+                    env.Add(name, qname);
+                    o.Append("{\"name\":").Append(qname);
+                    o.Append(",\"type\":\"").Append(Schema.GetTypeString(s.Tag)).Append("\"");
+                    if (st == Schema.Type.Enumeration)
+                    {
+                        EnumSchema enumSchema = s as EnumSchema;
+                        o.Append(",\"symbols\":[");
+                        foreach (var enumSymbol in enumSchema.Symbols)
+                        {
+                            if (!firstTime)
+                            {
+                                o.Append(",");
+                            }
+                            else
+                            {
+                                firstTime = false;
+                            }
+                            o.Append("\"").Append(enumSymbol).Append("\"");
+                        }
+                        o.Append("]");
+                    }
+                    else if (st == Schema.Type.Fixed)
+                    {
+                        FixedSchema fixedSchema = s as FixedSchema;
+                        o.Append(",\"size\":").Append(fixedSchema.Size.ToString());
+                    }
+                    else  // st == Schema.Type.Record
+                    {
+                        RecordSchema recordSchema = s as RecordSchema;
+                        o.Append(",\"fields\":[");
+                        foreach (var field in recordSchema.Fields)
+                        {
+                            if (!firstTime)
+                            {
+                                o.Append(",");
+                            }
+                            else
+                            {
+                                firstTime = false;
+                            }
+                            o.Append("{\"name\":\"").Append(field.Name).Append("\"");
+                            Build(env, field.Schema, o.Append(",\"type\":")).Append("}");
+                        }
+                        o.Append("]");
+                    }
+                    return o.Append("}");
+
+                default:    //boolean, bytes, double, float, int, long, null, string
+                    return o.Append("\"").Append(s.Name).Append("\"");
+            }
+        }
+
+        private static class Fp64
+        {
+            private static readonly long[] fpTable = new long[256];
+
+            public static long[] FpTable
+            {
+                get { return fpTable; }
+            }
+
+            static Fp64()
+            {
+                for (int i = 0; i < 256; i++)
+                {
+                    long fp = i;
+                    for (int j = 0; j < 8; j++)
+                    {
+                        long mask = -(fp & 1L);
+                        fp = ((long) (((ulong) fp) >> 1)) ^ (Empty64 & mask);
+                    }
+                    FpTable[i] = fp;
+                }
+            }
+        }
+    }
+}

Modified: avro/trunk/lang/csharp/src/apache/main/Schema/UnionSchema.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/Schema/UnionSchema.cs?rev=1339045&r1=1339044&r2=1339045&view=diff
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/Schema/UnionSchema.cs (original)
+++ avro/trunk/lang/csharp/src/apache/main/Schema/UnionSchema.cs Wed May 16 07:58:45 2012
@@ -73,7 +73,7 @@ namespace Avro
         /// <param name="schemas"></param>
         private UnionSchema(List<Schema> schemas, PropertyMap props) : base(Type.Union, props)
         {
-            if (schemas.Count == 0)
+            if (schemas == null)
                 throw new ArgumentNullException("schemas");
             this.Schemas = schemas;
         }

Modified: avro/trunk/lang/csharp/src/apache/test/Avro.test.csproj
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/test/Avro.test.csproj?rev=1339045&r1=1339044&r2=1339045&view=diff
==============================================================================
--- avro/trunk/lang/csharp/src/apache/test/Avro.test.csproj (original)
+++ avro/trunk/lang/csharp/src/apache/test/Avro.test.csproj Wed May 16 07:58:45 2012
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?xml version="1.0" encoding="utf-8"?>
 <!--
    Licensed to the Apache Software Foundation (ASF) under one or more
    contributor license agreements.  See the NOTICE file distributed with
@@ -88,8 +88,11 @@
     <Compile Include="Properties\AssemblyInfo.cs" />
     <Compile Include="Protocol\ProtocolTest.cs" />
     <Compile Include="Schema\AliasTest.cs" />
+    <Compile Include="Schema\SchemaNormalizationTests.cs" />
     <Compile Include="Schema\SchemaTests.cs" />
     <Compile Include="Specific\SpecificTests.cs" />
+    <Compile Include="Utils\CaseFinder.cs" />
+    <Compile Include="Utils\CaseFinderTests.cs" />
   </ItemGroup>
   <ItemGroup>
     <ProjectReference Include="..\main\Avro.main.csproj">
@@ -114,6 +117,7 @@
       <Install>true</Install>
     </BootstrapperPackage>
   </ItemGroup>
+  <ItemGroup />
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 
        Other similar extension points exist, see Microsoft.Common.targets.
@@ -122,4 +126,4 @@
   <Target Name="AfterBuild">
   </Target>
   -->
-</Project>
+</Project>
\ No newline at end of file

Added: avro/trunk/lang/csharp/src/apache/test/Schema/SchemaNormalizationTests.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/test/Schema/SchemaNormalizationTests.cs?rev=1339045&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/test/Schema/SchemaNormalizationTests.cs (added)
+++ avro/trunk/lang/csharp/src/apache/test/Schema/SchemaNormalizationTests.cs Wed May 16 07:58:45 2012
@@ -0,0 +1,95 @@
+/**
+ * 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
+ *
+ *     http://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.Linq;
+using System.Text;
+using NUnit.Framework;
+using System.IO;
+using Avro.Test.Utils;
+using Avro;
+
+namespace Avro.Test
+{
+    [TestFixture]
+    public class SchemaNormalizationTests
+    {
+        private static readonly long One = -9223372036854775808;
+        private static readonly byte[] Postfix = { 0, 0, 0, 0, 0, 0, 0, 0 };
+
+        [Test, TestCaseSource("ProvideCanonicalTestCases")]
+        public void CanonicalTest(string input, string expectedOutput)
+        {
+            Assert.AreEqual(expectedOutput, SchemaNormalization.ToParsingForm(Schema.Parse(input)));
+        }
+
+        [Test, TestCaseSource("ProvideFingerprintTestCases")]
+        public void FingerprintTest(string input, string expectedOutput)
+        {
+            Schema s = Schema.Parse(input);
+            long carefulFP = AltFingerprint(SchemaNormalization.ToParsingForm(s));
+            Assert.AreEqual(long.Parse(expectedOutput), carefulFP);
+            Assert.AreEqual(carefulFP, SchemaNormalization.ParsingFingerprint64(s));
+        }
+
+        private static IEnumerable<object> ProvideFingerprintTestCases()
+        {
+            using (StreamReader reader = new StreamReader("../../../../../share/test/data/schema-tests.txt"))
+            {
+                return CaseFinder.Find(reader, "fingerprint", new List<object[]>());
+            }
+        }
+
+        private static IEnumerable<object> ProvideCanonicalTestCases()
+        {
+            using (StreamReader reader = new StreamReader("../../../../../share/test/data/schema-tests.txt"))
+            {
+                return CaseFinder.Find(reader, "canonical", new List<object[]>());
+            }
+        }
+
+        private static long AltFingerprint(string s)
+        {
+            long tmp = AltExtended(SchemaNormalization.Empty64, 64, One, Encoding.UTF8.GetBytes(s));
+            return AltExtended(SchemaNormalization.Empty64, 64, tmp, Postfix);
+        }
+
+        private static long AltExtended(long poly, int degree, long fp, byte[] b)
+        {
+            long overflowBit = 1L << (64 - degree);
+            for (int i = 0; i < b.Length; i++)
+            {
+                for (int j = 1; j < 129; j = j << 1)
+                {
+                    bool overflow = (0 != (fp & overflowBit));
+                    fp = (long) (((ulong) fp) >> 1);
+                    if (0 != (j & b[i]))
+                    {
+                        fp |= One;
+                    }
+                    if (overflow)
+                    {
+                        fp ^= poly;
+                    }
+                }
+            }
+            return fp;
+        }
+    }
+}

Added: avro/trunk/lang/csharp/src/apache/test/Utils/CaseFinder.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/test/Utils/CaseFinder.cs?rev=1339045&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/test/Utils/CaseFinder.cs (added)
+++ avro/trunk/lang/csharp/src/apache/test/Utils/CaseFinder.cs Wed May 16 07:58:45 2012
@@ -0,0 +1,134 @@
+/**
+ * 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
+ *
+ *     http://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 System.Text.RegularExpressions;
+
+namespace Avro.Test.Utils
+{
+    public class CaseFinder
+    {
+        private static string labelRegex = "[a-zA-Z][_a-zA-Z0-9]*";
+        private static string newCaseName = "INPUT";
+        private static string newCaseMarker = "<<" + newCaseName;
+        private static string startLinePattern = "^<<("+labelRegex+")(.*)$";
+
+        /// <summary>
+        /// Scan test-case file <code>streamReader</code> looking for test subcases
+        /// marked with <code>label</code>.  Any such cases are appended
+        /// (in order) to the "cases" parameter.
+        /// </summary>
+        /// <param name="streamReader"></param>
+        /// <param name="label"></param>
+        /// <param name="cases"></param>
+        /// <returns></returns>
+        public static List<object[]> Find(StreamReader streamReader, string label, List<object[]> cases)
+        {
+            if (!Regex.IsMatch(label, "^" + labelRegex + "$"))
+            {
+                throw new ArgumentException("Bad case subcase label: " + label);
+            }
+
+            string subcaseMarker = "<<" + label;
+
+            var line = streamReader.ReadLine();
+            while (true)
+            {
+                while (line != null && !line.StartsWith(newCaseMarker))
+                {
+                    line = streamReader.ReadLine();
+                }
+                if (line == null)
+                {
+                    break;
+                }
+                string input = ProcessHereDoc(streamReader, line);
+
+                if (label == newCaseName)
+                {
+                    cases.Add(new object[] { input, null });
+                    line = streamReader.ReadLine();
+                    continue;
+                }
+
+                do
+                {
+                    line = streamReader.ReadLine();
+                } while (line != null && (!line.StartsWith(newCaseMarker) && !line.StartsWith(subcaseMarker)));
+
+                if (line == null || line.StartsWith(newCaseMarker))
+                {
+                    continue;
+                }
+
+                string expectedOutput = ProcessHereDoc(streamReader, line);
+                cases.Add(new object[] { input, expectedOutput });
+            }
+            return cases;
+        }
+
+        private static string ProcessHereDoc(StreamReader streamReader, string docStart)
+        {
+            var match = Regex.Match(docStart, startLinePattern);
+            if (!match.Success)
+            {
+                throw new ArgumentException(string.Format("Wasn't given the start of a heredoc (\"{0}\")", docStart));
+            }
+
+            string docName = match.Groups[1].Value;
+
+            // Determine if this is a single-line heredoc, and process if it is
+            string singleLineText = match.Groups[2].Value;
+            if (singleLineText.Length != 0)
+            {
+                if (!singleLineText.StartsWith(" "))
+                {
+                    throw new IOException(string.Format("Single-line heredoc missing initial space (\"{0}\")", docStart));
+                }
+                return singleLineText.Substring(1);
+            }
+            
+            // Process multi-line heredocs
+            var sb = new StringBuilder();
+            string line = streamReader.ReadLine();
+            string prevLine = string.Empty;
+            bool firstTime = true;
+            while (line != null && line != docName)
+            {
+                if (!firstTime)
+                {
+                    sb.Append(prevLine).Append("\n");
+                }
+                else
+                {
+                    firstTime = false;
+                }
+                prevLine = line;
+                line = streamReader.ReadLine();
+            }
+            if (line == null)
+            {
+                throw new IOException(string.Format("Here document ({0}) terminated by end-of-file.", docName));
+            }
+            return sb.Append(prevLine).ToString();
+        }
+    }
+}

Added: avro/trunk/lang/csharp/src/apache/test/Utils/CaseFinderTests.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/test/Utils/CaseFinderTests.cs?rev=1339045&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/test/Utils/CaseFinderTests.cs (added)
+++ avro/trunk/lang/csharp/src/apache/test/Utils/CaseFinderTests.cs Wed May 16 07:58:45 2012
@@ -0,0 +1,145 @@
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using System.Text;
+using NUnit.Framework;
+using System.IO;
+
+namespace Avro.Test.Utils
+{
+    [TestFixture]
+    public class CaseFinderTests
+    {
+        [Test]
+        public void TestBadDocLabel1()
+        {
+            List<Object[]> result = new List<Object[]>();
+            Assert.Throws<ArgumentException>(
+                () => CaseFinder.Find(Mk("<<INPUT blah"), "", result)
+                );
+        }
+
+        [Test]
+        public void TestBadDocLabel2()
+        {
+            List<Object[]> result = new List<Object[]>();
+            Assert.Throws<ArgumentException>(
+                () => CaseFinder.Find(Mk("<<INPUT blah"), "kill-er", result)
+                );
+        }
+
+        [Test]
+        public void TestBadSingleLineHeredoc()
+        {
+            List<Object[]> result = new List<Object[]>();
+            Assert.Throws<IOException>(
+                () => CaseFinder.Find(Mk("<<INPUTblah"), "foo", result)
+                );
+        }
+
+        [Test]
+        public void TestUnterminatedHeredoc()
+        {
+            List<Object[]> result = new List<Object[]>();
+            Assert.Throws<IOException>(
+                () => CaseFinder.Find(Mk("<<INPUT"), "foo", result)
+                );
+        }
+
+        [Test, TestCaseSource("OutputTestCases")]
+        public void TestOutput(string input, string label, List<object[]> expectedOutput)
+        {
+            List<Object[]> result = new List<Object[]>();
+            CaseFinder.Find(Mk(input), label, result);
+            Assert.True(Eq(result, expectedOutput), Pr(result));
+        }
+
+        private List<Object[]> OutputTestCases()
+        {
+            List<Object[]> result = new List<Object[]>();
+            result.Add(new Object[] { "", "foo", new List<object[]> { } });
+            result.Add(new Object[] { "<<INPUT a\n<<OUTPUT b", "OUTPUT", new List<object[]> { new object[] {"a","b"} } });
+            result.Add(new Object[] { "<<INPUT a\n<<OUTPUT b\n", "OUTPUT", new List<object[]> { new object[] { "a", "b" } } });
+            result.Add(new Object[] { "<<INPUT a\n<<OUTPUT b\n\n", "OUTPUT", new List<object[]> { new object[] { "a", "b" } } });
+            result.Add(new Object[] { "<<INPUT a\r<<OUTPUT b", "OUTPUT", new List<object[]> { new object[] { "a", "b" } } });
+            result.Add(new Object[] { "// This is a test\n<<INPUT a\n\n\n<<OUTPUT b", "OUTPUT", new List<object[]> { new object[] { "a", "b" } } });
+            result.Add(new Object[] { "<<INPUT a\n<<OUTPUT\nb\nOUTPUT", "OUTPUT", new List<object[]> { new object[] { "a", "b" } } });
+            result.Add(new Object[] { "<<INPUT a\n<<OUTPUT\nb\nOUTPUT", "OUTPUT", new List<object[]> { new object[] { "a", "b" } } });
+            result.Add(new Object[] { "<<INPUT a\n<<OUTPUT\nb\n\nOUTPUT", "OUTPUT", new List<object[]> { new object[] { "a", "b\n" } } });
+            result.Add(new Object[] { "<<INPUT a\n<<OUTPUT\n\n  b  \n\nOUTPUT", "OUTPUT", new List<object[]> { new object[] { "a", "\n  b  \n" } } });
+            result.Add(new Object[] { "<<INPUT a\n<<O b\n<<INPUT c\n<<O d", "O", new List<object[]> { new object[] { "a", "b" }, new object[] { "c", "d" } } });
+            result.Add(new Object[] { "<<INPUT a\n<<O b\n<<F z\n<<INPUT c\n<<O d", "O", new List<object[]> { new object[] { "a", "b" }, new object[] { "c", "d" } } });
+            result.Add(new Object[] { "<<INPUT a\n<<O b\n<<F z\n<<INPUT c\n<<O d", "F", new List<object[]> { new object[] { "a", "z" } } });
+            result.Add(new Object[] { "<<INPUT a\n<<O b\n<<F z\n<<INPUT\nc\nINPUT\n<<O d\n<<INPUT e", "INPUT", new List<object[]> { new object[] { "a", null }, new object[] { "c", null }, new object[] { "e", null } } });
+            return result;
+        }
+
+        private StreamReader Mk(string s)
+        {
+            byte[] byteArray = Encoding.ASCII.GetBytes(s);
+            MemoryStream stream = new MemoryStream(byteArray);
+            return new StreamReader(stream);
+        }
+
+        private string Pr(List<object[]> t)
+        {
+            StringBuilder sb = new StringBuilder();
+            sb.Append("{ ");
+            bool firstTime = true;
+            foreach (var obj in t)
+            {
+                if (!firstTime)
+                {
+                    sb.Append(", ");
+                }
+                else
+                {
+                    firstTime = false;
+                }
+                sb.Append("{ \"").Append(obj[0]).Append("\", \"").Append(obj[1]).Append("\" }");
+            }
+            sb.Append("}");
+            return sb.ToString();
+        }
+
+        private bool Eq(List<object []> l1, List<object []> l2)
+        {
+            if (l1 == null || l2 == null)
+            {
+                return l1 == l2;
+            }
+            if (l1.Count != l2.Count)
+            {
+                return false;
+            }
+            for (int i = 0; i < l1.Count; i++)
+            {
+                if (!ArraysEqual(l1[i], l2[i]))
+                {
+                    return false;
+                }
+            }
+            return true;
+        }
+
+        static bool ArraysEqual<T>(T[] a1, T[] a2)
+        {
+            if (ReferenceEquals(a1, a2))
+                return true;
+
+            if (a1 == null || a2 == null)
+                return false;
+
+            if (a1.Length != a2.Length)
+                return false;
+
+            EqualityComparer<T> comparer = EqualityComparer<T>.Default;
+            for (int i = 0; i < a1.Length; i++)
+            {
+                if (!comparer.Equals(a1[i], a2[i])) return false;
+            }
+            return true;
+        }
+
+    }
+}