avro-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From cutt...@apache.org
Subject svn commit: r1498613 [1/2] - in /avro/trunk: ./ lang/csharp/src/apache/codegen/ lang/csharp/src/apache/main/ lang/csharp/src/apache/main/CodeGen/ lang/csharp/src/apache/main/File/ lang/csharp/src/apache/main/Specific/ lang/csharp/src/apache/test/ lang/...
Date Mon, 01 Jul 2013 18:04:38 GMT
Author: cutting
Date: Mon Jul  1 18:04:37 2013
New Revision: 1498613

URL: http://svn.apache.org/r1498613
Log:
AVRO-823: C#: Add data file support.  Contributed by David McIntosh.

Added:
    avro/trunk/lang/csharp/src/apache/main/File/
    avro/trunk/lang/csharp/src/apache/main/File/Codec.cs
    avro/trunk/lang/csharp/src/apache/main/File/DataBlock.cs
    avro/trunk/lang/csharp/src/apache/main/File/DataFileConstants.cs
    avro/trunk/lang/csharp/src/apache/main/File/DataFileReader.cs
    avro/trunk/lang/csharp/src/apache/main/File/DataFileWriter.cs
    avro/trunk/lang/csharp/src/apache/main/File/DeflateCodec.cs
    avro/trunk/lang/csharp/src/apache/main/File/Header.cs
    avro/trunk/lang/csharp/src/apache/main/File/IFileReader.cs
    avro/trunk/lang/csharp/src/apache/main/File/IFileWriter.cs
    avro/trunk/lang/csharp/src/apache/main/File/NullCodec.cs
    avro/trunk/lang/csharp/src/apache/test/File/
    avro/trunk/lang/csharp/src/apache/test/File/FileTests.cs
Modified:
    avro/trunk/CHANGES.txt
    avro/trunk/lang/csharp/src/apache/codegen/Avro.codegen.csproj
    avro/trunk/lang/csharp/src/apache/main/Avro.main.csproj
    avro/trunk/lang/csharp/src/apache/main/CodeGen/CodeGen.cs
    avro/trunk/lang/csharp/src/apache/main/Specific/SpecificReader.cs
    avro/trunk/lang/csharp/src/apache/test/Avro.test.csproj
    avro/trunk/lang/csharp/src/apache/test/CodGen/CodeGenTest.cs
    avro/trunk/lang/csharp/src/apache/test/Specific/SpecificTests.cs

Modified: avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/avro/trunk/CHANGES.txt?rev=1498613&r1=1498612&r2=1498613&view=diff
==============================================================================
--- avro/trunk/CHANGES.txt (original)
+++ avro/trunk/CHANGES.txt Mon Jul  1 18:04:37 2013
@@ -12,6 +12,8 @@ Trunk (not yet released)
     AVRO-1319. Java: Add command line tools to generate random data
     files and to convert Avro to Trevni.  (cutting)
 
+    AVRO-823: C#: Add data file support. (David McIntosh via cutting)
+
   IMPROVEMENTS
 
     AVRO-1260. Ruby: Improve read performance. (Martin Kleppmann via cutting)

Modified: avro/trunk/lang/csharp/src/apache/codegen/Avro.codegen.csproj
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/codegen/Avro.codegen.csproj?rev=1498613&r1=1498612&r2=1498613&view=diff
==============================================================================
--- avro/trunk/lang/csharp/src/apache/codegen/Avro.codegen.csproj (original)
+++ avro/trunk/lang/csharp/src/apache/codegen/Avro.codegen.csproj Mon Jul  1 18:04:37 2013
@@ -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
@@ -117,7 +117,6 @@
     <Reference Include="System.Core" />
     <Reference Include="System.Xml.Linq" />
     <Reference Include="System.Data.DataSetExtensions" />
-    <Reference Include="Microsoft.CSharp" />
     <Reference Include="System.Data" />
     <Reference Include="System.Xml" />
   </ItemGroup>

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=1498613&r1=1498612&r2=1498613&view=diff
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/Avro.main.csproj (original)
+++ avro/trunk/lang/csharp/src/apache/main/Avro.main.csproj Mon Jul  1 18:04:37 2013
@@ -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,6 +88,16 @@
     <Compile Include="CodeGen\CodeGen.cs" />
     <Compile Include="CodeGen\CodeGenException.cs" />
     <Compile Include="CodeGen\CodeGenUtil.cs" />
+    <Compile Include="File\Codec.cs" />
+    <Compile Include="File\DataBlock.cs" />
+    <Compile Include="File\DataFileConstants.cs" />
+    <Compile Include="File\DataFileReader.cs" />
+    <Compile Include="File\DataFileWriter.cs" />
+    <Compile Include="File\DeflateCodec.cs" />
+    <Compile Include="File\Header.cs" />
+    <Compile Include="File\IFileReader.cs" />
+    <Compile Include="File\IFileWriter.cs" />
+    <Compile Include="File\NullCodec.cs" />
     <Compile Include="Generic\DatumReader.cs" />
     <Compile Include="Generic\DatumWriter.cs" />
     <Compile Include="Generic\GenericEnum.cs" />
@@ -153,4 +163,4 @@
   <Target Name="AfterBuild">
   </Target>
   -->
-</Project>
\ No newline at end of file
+</Project>

Modified: avro/trunk/lang/csharp/src/apache/main/CodeGen/CodeGen.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/CodeGen/CodeGen.cs?rev=1498613&r1=1498612&r2=1498613&view=diff
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/CodeGen/CodeGen.cs (original)
+++ avro/trunk/lang/csharp/src/apache/main/CodeGen/CodeGen.cs Mon Jul  1 18:04:37 2013
@@ -565,7 +565,7 @@ namespace Avro
             var ctrfield = new CodeTypeReference("Schema");
             string schemaFname = "_SCHEMA";
             var codeField = new CodeMemberField(ctrfield, schemaFname);
-            codeField.Attributes = MemberAttributes.Private | MemberAttributes.Static;
+            codeField.Attributes = MemberAttributes.Public | MemberAttributes.Static;
             // create function call Schema.Parse(json)
             var cpe = new CodePrimitiveExpression(schema.ToString());
             var cmie = new CodeMethodInvokeExpression(

Added: avro/trunk/lang/csharp/src/apache/main/File/Codec.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/File/Codec.cs?rev=1498613&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/File/Codec.cs (added)
+++ avro/trunk/lang/csharp/src/apache/main/File/Codec.cs Mon Jul  1 18:04:37 2013
@@ -0,0 +1,115 @@
+/**
+ * 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 System.IO;
+
+namespace Avro.File
+{
+    abstract public class Codec
+    {
+        /// <summary>
+        /// Compress data using implemented codec
+        /// </summary>
+        /// <param name="uncompressedData"></param>
+        /// <returns></returns>
+        abstract public byte[] Compress(byte[] uncompressedData);
+
+        /// <summary>
+        /// Decompress data using implemented codec
+        /// </summary>
+        /// <param name="compressedData"></param>
+        /// <returns></returns>
+        abstract public byte[] Decompress(byte[] compressedData);
+
+        /// <summary>
+        /// Name of this codec type
+        /// </summary>
+        /// <returns></returns>
+        abstract public string GetName();
+
+        /// <summary>
+        ///  Codecs must implement an equals() method
+        /// </summary>
+        /// <param name="other"></param>
+        /// <returns></returns>
+        abstract public override bool Equals(object other);
+
+        /// <summary>
+        /// Codecs must implement a HashCode() method that is
+        /// consistent with Equals
+        /// </summary>
+        /// <returns></returns>
+        abstract public override int GetHashCode();
+
+        /// <summary>
+        /// Codec types
+        /// </summary>
+        public enum Type
+        {
+            Deflate,
+            //Snappy 
+            Null
+        };
+
+        /// <summary>
+        /// Factory method to return child
+        /// codec instance based on Codec.Type  
+        /// </summary>
+        /// <param name="codecType"></param>
+        /// <returns></returns>
+        public static Codec CreateCodec(Type codecType)
+        {
+            switch (codecType)
+            {
+                case Type.Deflate:
+                    return new DeflateCodec();
+                default:
+                    return new NullCodec();
+            }
+        }
+
+        /// <summary>
+        /// Factory method to return child
+        /// codec instance based on string type  
+        /// </summary>
+        /// <param name="codecType"></param>
+        /// <returns></returns>
+        public static Codec CreateCodecFromString(string codecType)
+        {
+            switch (codecType)
+            {
+                case DataFileConstants.DeflateCodec:
+                    return new DeflateCodec();
+                default:
+                    return new NullCodec();
+            }
+        }
+
+        /// <summary>
+        /// Returns name of codec
+        /// </summary>
+        /// <returns></returns>
+        public override string ToString()
+        {
+            return GetName();
+        }
+    }
+}
\ No newline at end of file

Added: avro/trunk/lang/csharp/src/apache/main/File/DataBlock.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/File/DataBlock.cs?rev=1498613&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/File/DataBlock.cs (added)
+++ avro/trunk/lang/csharp/src/apache/main/File/DataBlock.cs Mon Jul  1 18:04:37 2013
@@ -0,0 +1,43 @@
+/**
+ * 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.IO;
+using Avro.IO;
+using Avro.Generic;
+using System.Collections.Generic;
+
+namespace Avro.File
+{
+    public class DataBlock
+    {
+        public byte[] Data { get;  set; }
+        public long NumberOfEntries { get; set; }
+        public long BlockSize { get; set; }
+       
+        public DataBlock(long numberOfEntries, long blockSize)
+        {
+            this.NumberOfEntries = numberOfEntries;
+            this.BlockSize = blockSize;
+            this.Data = new byte[blockSize];
+        }
+
+        internal Stream GetDataAsStream()
+        {
+            return new MemoryStream(Data);
+        }
+    }
+}

Added: avro/trunk/lang/csharp/src/apache/main/File/DataFileConstants.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/File/DataFileConstants.cs?rev=1498613&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/File/DataFileConstants.cs (added)
+++ avro/trunk/lang/csharp/src/apache/main/File/DataFileConstants.cs Mon Jul  1 18:04:37 2013
@@ -0,0 +1,46 @@
+/**
+ * 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;
+
+namespace Avro.File
+{
+    public class DataFileConstants
+    {
+        public const string MetaDataSync = "avro.sync";
+        public const string MetaDataCodec = "avro.codec";
+        public const string MetaDataSchema = "avro.schema";
+        public const string NullCodec = "null";
+        public const string DeflateCodec = "deflate";
+        public const string MetaDataReserved = "avro";
+
+        public const int Version = 1;
+        public static byte[] Magic = { (byte)'O', 
+                                       (byte)'b', 
+                                       (byte)'j', 
+                                       (byte)Version };
+
+        public const int NullCodecHash = 2;
+        public const int DeflateCodecHash = 0;
+
+        public const int SyncSize = 16;
+        public const int DefaultSyncInterval = 1000 * SyncSize;
+    }
+}

Added: avro/trunk/lang/csharp/src/apache/main/File/DataFileReader.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/File/DataFileReader.cs?rev=1498613&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/File/DataFileReader.cs (added)
+++ avro/trunk/lang/csharp/src/apache/main/File/DataFileReader.cs Mon Jul  1 18:04:37 2013
@@ -0,0 +1,410 @@
+/**
+ * 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.IO;
+using Avro.Generic;
+using Avro.IO;
+using Avro.Specific;
+
+namespace Avro.File
+{
+    public class DataFileReader<T> : IFileReader<T>
+    {
+        private DatumReader<T> _reader;
+        private Decoder _decoder, _datumDecoder;
+        private Header _header;
+        private Codec _codec;
+        private DataBlock _currentBlock;
+        private long _blockRemaining;
+        private long _blockSize;
+        private bool _availableBlock;
+        private byte[] _syncBuffer;
+        private long _blockStart;
+        private Stream _stream;
+        private Schema _readerSchema;
+
+        /// <summary>
+        ///  Open a reader for a file using path
+        /// </summary>
+        /// <param name="path"></param>
+        /// <returns></returns>
+        public static IFileReader<T> OpenReader(string path)
+        {
+            return OpenReader(new FileStream(path, FileMode.Open), null);
+        }
+
+        /// <summary>
+        ///  Open a reader for a file using path and the reader's schema
+        /// </summary>
+        /// <param name="path"></param>
+        /// <returns></returns>
+        public static IFileReader<T> OpenReader(string path, Schema readerSchema)
+        {
+            return OpenReader(new FileStream(path, FileMode.Open), readerSchema);
+        }
+
+        /// <summary>
+        ///  Open a reader for a stream
+        /// </summary>
+        /// <param name="inStream"></param>
+        /// <returns></returns>
+        public static IFileReader<T> OpenReader(Stream inStream)
+        {
+            return OpenReader(inStream, null);
+        }
+
+        /// <summary>
+        ///  Open a reader for a stream and using the reader's schema
+        /// </summary>
+        /// <param name="inStream"></param>
+        /// <returns></returns>
+        public static IFileReader<T> OpenReader(Stream inStream, Schema readerSchema)
+        {
+            if (!inStream.CanSeek)
+                throw new AvroRuntimeException("Not a valid input stream - must be seekable!");
+
+            if (inStream.Length < DataFileConstants.Magic.Length)
+                throw new AvroRuntimeException("Not an Avro data file");
+
+            // verify magic header
+            byte[] magic = new byte[DataFileConstants.Magic.Length];
+            inStream.Seek(0, SeekOrigin.Begin);
+            for (int c = 0; c < magic.Length; c = inStream.Read(magic, c, magic.Length - c)) { }
+            inStream.Seek(0, SeekOrigin.Begin);
+
+            if (magic.SequenceEqual(DataFileConstants.Magic))   // current format
+                return new DataFileReader<T>(inStream, readerSchema);         // (not supporting 1.2 or below, format) 
+
+            throw new AvroRuntimeException("Not an Avro data file");
+        }
+
+        DataFileReader(Stream stream, Schema readerSchema)
+        {
+            _readerSchema = readerSchema;
+            Init(stream);
+            BlockFinished();
+        }
+
+        public Header GetHeader()
+        {
+            return _header;
+        }
+
+        public Schema GetSchema()
+        {
+            return _header.Schema;
+        }
+
+        public ICollection<string> GetMetaKeys()
+        {
+            return _header.MetaData.Keys;
+        }
+
+        public byte[] GetMeta(string key)
+        {
+            try
+            {
+                return _header.MetaData[key];
+            }
+            catch (KeyNotFoundException knfe)
+            {
+                return null; 
+            }
+        }
+
+        public long GetMetaLong(string key)
+        {
+            return long.Parse(GetMetaString(key));
+        }
+
+        public string GetMetaString(string key)
+        {
+            byte[] value = GetMeta(key);
+            if (value == null)
+            {
+                return null;
+            }
+            try
+            {
+                return System.Text.Encoding.UTF8.GetString(value);          
+            }
+            catch (Exception e)
+            {
+                throw new AvroRuntimeException(string.Format("Error fetching meta data for key: {0}", key), e);
+            }
+        }
+
+        public void Seek(long position)
+        {
+            _stream.Position = position;
+            _decoder = new BinaryDecoder(_stream);
+            _datumDecoder = null;
+            _blockRemaining = 0;
+            _blockStart = position;
+        }
+
+        public void Sync(long position)
+        {
+            Seek(position);
+            // work around an issue where 1.5.4 C stored sync in metadata
+            if ((position == 0) && (GetMeta(DataFileConstants.MetaDataSync) != null)) 
+            {
+                Init(_stream); // re-init to skip header
+                return;
+            }
+
+            try
+            {
+                bool done = false;
+
+                do // read until sync mark matched
+                {
+                    _decoder.ReadFixed(_syncBuffer);
+                    if (Enumerable.SequenceEqual(_syncBuffer, _header.SyncData))
+                        done = true;
+                    else
+                        _stream.Position = _stream.Position - (DataFileConstants.SyncSize - 1);
+                } while (!done);
+            }
+            catch (Exception e) { } // could not find .. default to EOF
+
+            _blockStart = _stream.Position;
+        }
+
+        public bool PastSync(long position)
+        {
+            return ((_blockStart >= position + DataFileConstants.SyncSize) || (_blockStart >= _stream.Length));
+        }
+
+        public long PreviousSync()
+        {
+            return _blockStart;
+        }
+
+        public long Tell()
+        {
+            return _stream.Position;
+        }
+
+        public IEnumerable<T> NextEntries
+        {
+            get
+            {
+                while (HasNext())
+                {
+                    yield return Next();
+                }
+            }
+        }
+
+        public bool HasNext()
+        {
+            try
+            {
+                if (_blockRemaining == 0)
+                {
+                    // TODO: Check that the (block) stream is not partially read
+                    /*if (_datumDecoder != null) 
+                    { }*/
+                    if (HasNextBlock())
+                    {
+                        _currentBlock = NextRawBlock(_currentBlock);
+                        _currentBlock.Data = _codec.Decompress(_currentBlock.Data);
+                        _datumDecoder = new BinaryDecoder(_currentBlock.GetDataAsStream());
+                    }
+                }
+                return _blockRemaining != 0;
+            }
+            catch (Exception e)
+            {
+                throw new AvroRuntimeException(string.Format("Error fetching next object from block: {0}", e));
+            }
+        }
+
+        public void Reset()
+        {
+            Init(_stream);
+        }
+
+        public void Dispose()
+        {
+            _stream.Close();
+        }
+
+        private void Init(Stream stream)
+        {
+            _stream = stream;
+            _header = new Header();
+            _decoder = new BinaryDecoder(stream);
+            _syncBuffer = new byte[DataFileConstants.SyncSize];
+
+            // read magic 
+            byte[] firstBytes = new byte[DataFileConstants.Magic.Length];
+            try
+            {
+                _decoder.ReadFixed(firstBytes);
+            }
+            catch (Exception e)
+            {
+                throw new AvroRuntimeException("Not a valid data file!", e);
+            }
+            if (!firstBytes.SequenceEqual(DataFileConstants.Magic))
+                throw new AvroRuntimeException("Not a valid data file!");
+
+            // read meta data 
+            long len = _decoder.ReadMapStart();
+            if (len > 0)
+            {
+                do
+                {
+                    for (long i = 0; i < len; i++)
+                    {
+                        string key = _decoder.ReadString();
+                        byte[] val = _decoder.ReadBytes();
+                        _header.MetaData.Add(key, val);
+                    }
+                } while ((len = _decoder.ReadMapNext()) != 0);
+            }
+
+            // read in sync data 
+            _decoder.ReadFixed(_header.SyncData);
+
+            // parse schema and set codec 
+            _header.Schema = Schema.Parse(GetMetaString(DataFileConstants.MetaDataSchema));
+            _reader = GetReaderFromSchema();
+            _codec = ResolveCodec();
+        }
+
+        private DatumReader<T> GetReaderFromSchema()
+        {
+            DatumReader<T> reader = null;
+            Type type = typeof(T);
+
+            if (typeof(ISpecificRecord).IsAssignableFrom(type))
+            {
+                reader = new SpecificReader<T>(_header.Schema, _readerSchema ?? _header.Schema);
+            }
+            else // generic
+            {
+                reader = new GenericReader<T>(_header.Schema, _readerSchema ?? _header.Schema);
+            }
+            return reader;
+        }
+
+        private Codec ResolveCodec()
+        {
+            return Codec.CreateCodecFromString(GetMetaString(DataFileConstants.MetaDataCodec));
+        }
+
+        public T Next()
+        {
+            return Next(default(T));
+        }
+
+        private T Next(T reuse)
+        {
+            try
+            {
+                if (!HasNext())
+                    throw new AvroRuntimeException("No more datum objects remaining in block!");
+
+                T result = _reader.Read(reuse, _datumDecoder);
+                if (--_blockRemaining == 0)
+                {
+                    BlockFinished();
+                }
+                return result;
+            }
+            catch (Exception e)
+            {
+                throw new AvroRuntimeException(string.Format("Error fetching next object from block: {0}", e));
+            }
+        }
+
+        private void BlockFinished()
+        {
+            _blockStart = _stream.Position;
+        }
+
+        private DataBlock NextRawBlock(DataBlock reuse)
+        {
+            if (!HasNextBlock())
+                throw new AvroRuntimeException("No data remaining in block!");
+
+            if (reuse == null || reuse.Data.Length < _blockSize)
+            {
+                reuse = new DataBlock(_blockRemaining, _blockSize);
+            }
+            else
+            {
+                reuse.NumberOfEntries = _blockRemaining;
+                reuse.BlockSize = _blockSize;
+            }
+
+            _decoder.ReadFixed(reuse.Data, 0, (int)reuse.BlockSize);
+            _decoder.ReadFixed(_syncBuffer);
+
+            if (!Enumerable.SequenceEqual(_syncBuffer, _header.SyncData))
+                throw new AvroRuntimeException("Invalid sync!");
+
+            _availableBlock = false;
+            return reuse;
+        }
+
+        private bool DataLeft()
+        {
+            long currentPosition = _stream.Position;
+            if (_stream.ReadByte() != -1)
+                _stream.Position = currentPosition;
+            else
+                return false;
+
+            return true;
+        }
+
+        private bool HasNextBlock()
+        {
+            try
+            {
+                // block currently being read 
+                if (_availableBlock)
+                    return true;
+
+                // check to ensure still data to read 
+                if (!DataLeft())
+                    return false;
+
+                _blockRemaining = _decoder.ReadLong();      // read block count
+                _blockSize = _decoder.ReadLong();           // read block size
+                if (_blockSize > System.Int32.MaxValue || _blockSize < 0)
+                {
+                    throw new AvroRuntimeException("Block size invalid or too large for this " +
+                                                   "implementation: " + _blockSize);
+                }
+                _availableBlock = true;
+                return true;
+            }
+            catch (Exception e)
+            {
+                throw new AvroRuntimeException(string.Format("Error ascertaining if data has next block: {0}", e));
+            }
+        }
+    }
+}
\ No newline at end of file

Added: avro/trunk/lang/csharp/src/apache/main/File/DataFileWriter.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/File/DataFileWriter.cs?rev=1498613&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/File/DataFileWriter.cs (added)
+++ avro/trunk/lang/csharp/src/apache/main/File/DataFileWriter.cs Mon Jul  1 18:04:37 2013
@@ -0,0 +1,315 @@
+/**
+ * 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.Globalization;
+using System.IO;
+using Avro.IO;
+using Avro.Generic;
+
+namespace Avro.File
+{
+    public class DataFileWriter<T> : IFileWriter<T>
+    {
+        private Schema _schema;
+        private Codec _codec;
+        private Stream _stream;
+        private MemoryStream _blockStream;
+        private Encoder _encoder, _blockEncoder;
+        private DatumWriter<T> _writer;
+
+        private byte[] _syncData;
+        private bool _isOpen;
+        private bool _headerWritten;
+        private int _blockCount;
+        private int _syncInterval;
+        private IDictionary<string, byte[]> _metaData;
+
+        /// <summary>
+        /// Open a new writer instance to write  
+        /// to a file path, using a Null codec
+        /// </summary>
+        /// <param name="writer"></param>
+        /// <param name="path"></param>
+        /// <returns></returns>
+        public static IFileWriter<T> OpenWriter(DatumWriter<T> writer, string path)
+        {
+            return OpenWriter(writer, new FileStream(path, FileMode.Create), Codec.CreateCodec(Codec.Type.Null));
+        }
+
+        /// <summary>
+        /// Open a new writer instance to write  
+        /// to an output stream, using a Null codec
+        /// </summary>
+        /// <param name="writer"></param>
+        /// <param name="outStream"></param>
+        /// <returns></returns>
+        public static IFileWriter<T> OpenWriter(DatumWriter<T> writer, Stream outStream)
+        {
+            return OpenWriter(writer, outStream, Codec.CreateCodec(Codec.Type.Null));
+        }
+
+        /// <summary>
+        /// Open a new writer instance to write  
+        /// to a file path with a specified codec
+        /// </summary>
+        /// <param name="writer"></param>
+        /// <param name="path"></param>
+        /// <param name="codec"></param>
+        /// <returns></returns>
+        public static IFileWriter<T> OpenWriter(DatumWriter<T> writer, string path, Codec codec)
+        {
+            return OpenWriter(writer, new FileStream(path, FileMode.Create), codec);
+        }
+
+        /// <summary>
+        /// Open a new writer instance to write
+        /// to an output stream with a specified codec
+        /// </summary>
+        /// <param name="writer"></param>
+        /// <param name="outStream"></param>
+        /// <param name="codec"></param>
+        /// <returns></returns>
+        public static IFileWriter<T> OpenWriter(DatumWriter<T> writer, Stream outStream, Codec codec)
+        {
+            return new DataFileWriter<T>(writer).Create(writer.Schema, outStream, codec);
+        }
+
+        DataFileWriter(DatumWriter<T> writer)
+        {
+            _writer = writer;
+            _syncInterval = DataFileConstants.DefaultSyncInterval;
+        }
+
+        public bool IsReservedMeta(string key)
+        {
+            return key.StartsWith(DataFileConstants.MetaDataReserved);
+        }
+
+        public void SetMeta(String key, byte[] value)
+        {
+            if (IsReservedMeta(key))
+            {
+                throw new AvroRuntimeException("Cannot set reserved meta key: " + key);
+            }
+            _metaData.Add(key, value);
+        }
+
+        public void SetMeta(String key, long value)
+        {
+            try
+            {
+                SetMeta(key, GetByteValue(value.ToString(CultureInfo.InvariantCulture)));
+            }
+            catch (Exception e)
+            {
+                throw new AvroRuntimeException(e.Message, e);
+            }
+        }
+
+        public void SetMeta(String key, string value)
+        {
+            try
+            {
+                SetMeta(key, GetByteValue(value));
+            }
+            catch (Exception e)
+            {
+                throw new AvroRuntimeException(e.Message, e);
+            }
+        }
+
+        public void SetSyncInterval(int syncInterval)
+        {
+            if (syncInterval < 32 || syncInterval > (1 << 30))
+            {
+                throw new AvroRuntimeException("Invalid sync interval value: " + syncInterval);
+            }
+            _syncInterval = syncInterval;
+        }
+
+        public void Append(T datum) 
+        {
+            AssertOpen();
+            EnsureHeader();
+
+            long usedBuffer = _blockStream.Position;
+
+            try
+            {
+                _writer.Write(datum, _blockEncoder);
+            }
+            catch (Exception e)
+            {
+                _blockStream.Position = usedBuffer;
+                throw new AvroRuntimeException("Error appending datum to writer", e);
+            }
+            _blockCount++;
+            WriteIfBlockFull();
+        }
+
+        private void EnsureHeader()
+        {
+            if (!_headerWritten)
+            {
+                WriteHeader();
+                _headerWritten = true;
+            }
+        }
+
+        public void Flush()
+        {
+            EnsureHeader();
+            Sync();
+        }
+
+        public long Sync()
+        {
+            AssertOpen();
+            WriteBlock();
+            return _stream.Position;
+        }
+
+        public void Close()
+        {
+            EnsureHeader();
+            Flush();
+            _stream.Flush();
+            _stream.Close();
+            _isOpen = false;
+        }
+
+        private void WriteHeader()
+        {
+            _encoder.WriteFixed(DataFileConstants.Magic);
+            WriteMetaData();
+            WriteSyncData();
+        }
+
+        private void Init()
+        {
+            _blockCount = 0;
+            _encoder = new BinaryEncoder(_stream);
+            _blockStream = new MemoryStream();
+            _blockEncoder = new BinaryEncoder(_blockStream);
+
+            if (_codec == null)
+                _codec = Codec.CreateCodec(Codec.Type.Null);
+
+            _isOpen = true;
+        }
+
+        private void AssertOpen()
+        {
+            if (!_isOpen) throw new AvroRuntimeException("Cannot complete operation: avro file/stream not open");
+        }
+
+        private IFileWriter<T> Create(Schema schema, Stream outStream, Codec codec)
+        {
+            _codec = codec;
+            _stream = outStream;
+            _metaData = new Dictionary<string, byte[]>();
+            _schema = schema;
+
+            Init();
+
+            return this;
+        }
+
+        private void WriteMetaData()
+        {
+            // Add sync, code & schema to metadata
+            GenerateSyncData();
+            //SetMetaInternal(DataFileConstants.MetaDataSync, _syncData); - Avro 1.5.4 C
+            SetMetaInternal(DataFileConstants.MetaDataCodec, GetByteValue(_codec.GetName()));
+            SetMetaInternal(DataFileConstants.MetaDataSchema, GetByteValue(_schema.ToString()));
+            
+            // write metadata 
+            int size = _metaData.Count;
+            _encoder.WriteInt(size);
+
+            foreach (KeyValuePair<String, byte[]> metaPair in _metaData)
+            {
+                _encoder.WriteString(metaPair.Key);
+                _encoder.WriteBytes(metaPair.Value);
+            }
+            _encoder.WriteMapEnd();
+        }
+
+        private void WriteIfBlockFull()
+        {
+            if (BufferInUse() >= _syncInterval)
+                WriteBlock();
+        }
+
+        private long BufferInUse()
+        {
+            return _blockStream.Position;
+        }
+
+        private void WriteBlock() 
+        { 
+            if (_blockCount > 0) 
+            {
+                byte[] dataToWrite = _blockStream.ToArray();
+
+                // write count 
+                _encoder.WriteLong(_blockCount);
+
+                // write data 
+                _encoder.WriteBytes(_codec.Compress(dataToWrite));
+                    
+                // write sync marker 
+                _encoder.WriteFixed(_syncData);
+            
+                // reset / re-init block
+                _blockCount = 0;
+                _blockStream = new MemoryStream();
+                _blockEncoder = new BinaryEncoder(_blockStream);
+            }
+        }
+
+        private void WriteSyncData()
+        {
+            _encoder.WriteFixed(_syncData);
+        }
+
+        private void GenerateSyncData()
+        {
+            _syncData = new byte[16];
+
+            Random random = new Random();
+            random.NextBytes(_syncData);
+        }
+
+        private void SetMetaInternal(string key, byte[] value)
+        {
+            _metaData.Add(key, value);
+        }
+  
+        private byte[] GetByteValue(string value)
+        {
+            return System.Text.Encoding.UTF8.GetBytes(value);
+        }
+
+        public void Dispose()
+        {
+            Close();
+        }
+    }
+}

Added: avro/trunk/lang/csharp/src/apache/main/File/DeflateCodec.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/File/DeflateCodec.cs?rev=1498613&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/File/DeflateCodec.cs (added)
+++ avro/trunk/lang/csharp/src/apache/main/File/DeflateCodec.cs Mon Jul  1 18:04:37 2013
@@ -0,0 +1,83 @@
+/**
+ * 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 System.IO;
+using System.IO.Compression;
+
+namespace Avro.File
+{
+    public class DeflateCodec : Codec
+    {
+        public override byte[] Compress(byte[] uncompressedData)
+        {
+            MemoryStream outStream = new MemoryStream();
+
+            using (DeflateStream Compress =
+                        new DeflateStream(outStream,
+                        CompressionMode.Compress))
+            {
+                Compress.Write(uncompressedData, 0, uncompressedData.Length);
+            }
+            return outStream.ToArray();
+        }
+
+        public override byte[] Decompress(byte[] compressedData)
+        {
+            MemoryStream inStream = new MemoryStream(compressedData);
+            MemoryStream outStream = new MemoryStream();
+
+            using (DeflateStream Decompress =
+                        new DeflateStream(inStream,
+                        CompressionMode.Decompress))
+            {
+                CopyTo(Decompress, outStream);
+            }
+            return outStream.ToArray();
+        }
+
+        private static void CopyTo(Stream from, Stream to)
+        {
+            byte[] buffer = new byte[4096];
+            int read;
+            while((read = from.Read(buffer, 0, buffer.Length)) != 0)
+            {
+                to.Write(buffer, 0, read);
+            }
+        }
+
+        public override string GetName()
+        {
+            return DataFileConstants.DeflateCodec;
+        }
+
+        public override bool Equals(object other)
+        {
+            if (this == other)
+                return true;
+            return (this.GetType().Name == other.GetType().Name);
+        }
+
+        public override int GetHashCode()
+        {
+            return DataFileConstants.DeflateCodecHash;
+        }
+    }
+}

Added: avro/trunk/lang/csharp/src/apache/main/File/Header.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/File/Header.cs?rev=1498613&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/File/Header.cs (added)
+++ avro/trunk/lang/csharp/src/apache/main/File/Header.cs Mon Jul  1 18:04:37 2013
@@ -0,0 +1,40 @@
+/**
+ * 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 Avro.IO;
+
+namespace Avro.File
+{
+    public class Header
+    {
+        private IDictionary<string, byte[]> _metaData;
+        private byte[] _syncData;
+
+        public IDictionary<string, byte[]> MetaData { get { return _metaData; }}
+        public byte[] SyncData { get { return _syncData; }}
+        public Schema Schema { get; set; }
+
+        public Header()
+        {
+            _metaData = new Dictionary<string, byte[]>();
+            _syncData = new byte[16];
+        }
+    }
+}

Added: avro/trunk/lang/csharp/src/apache/main/File/IFileReader.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/File/IFileReader.cs?rev=1498613&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/File/IFileReader.cs (added)
+++ avro/trunk/lang/csharp/src/apache/main/File/IFileReader.cs Mon Jul  1 18:04:37 2013
@@ -0,0 +1,119 @@
+/**
+ * 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;
+
+namespace Avro.File
+{
+    public interface IFileReader<T> : IDisposable
+    {
+        /// <summary>
+        /// Return the header for the input 
+        /// file / stream
+        /// </summary>
+        /// <returns></returns>
+        Header GetHeader();
+
+        /// <summary>
+        /// Return the schema as read from 
+        /// the input file / stream
+        /// </summary>
+        /// <returns></returns>
+        Schema GetSchema();
+
+        /// <summary>
+        /// Return the list of keys in the metadata
+        /// </summary>
+        /// <returns></returns>
+        ICollection<string> GetMetaKeys();
+
+        /// <summary>
+        /// Return an enumeration of the remaining entries in the file
+        /// </summary>
+        /// <returns></returns>
+        IEnumerable<T> NextEntries { get; }
+
+        /// <summary>
+        /// Read the next datum from the file.
+        /// </summary>
+        T Next();
+
+        /// <summary>
+        /// True if more entries remain in this file.
+        /// </summary>
+        bool HasNext();
+
+        /// <summary>
+        /// Return the byte value of a metadata property
+        /// </summary>
+        /// <param name="key"></param>
+        /// <returns></returns>
+        byte[] GetMeta(string key);
+
+        /// <summary>
+        /// Return the long value of a metadata property
+        /// </summary>
+        /// <param name="key"></param>
+        /// <returns></returns>
+        long GetMetaLong(string key);
+
+        /// <summary>
+        /// Return the string value of a metadata property
+        /// </summary>
+        /// <param name="key"></param>
+        /// <returns></returns>
+        string GetMetaString(string key);
+
+        /// <summary>
+        /// Return true if past the next synchronization
+        /// point after a position
+        /// </summary>
+        /// <param name="position"></param>
+        /// <returns></returns>
+        bool PastSync(long position);
+
+        /// <summary>
+        /// Return the last synchronization point before
+        /// our current position
+        /// </summary>
+        /// <returns></returns>
+        long PreviousSync();
+
+        /// <summary>
+        /// Move to a specific, known synchronization point, 
+        /// one returned from IFileWriter.Sync() while writing
+        /// </summary>
+        /// <param name="position"></param>
+        void Seek(long position);
+
+        /// <summary>
+        /// Move to the next synchronization point
+        /// after a position
+        /// </summary>
+        /// <param name="position"></param>
+        void Sync(long position);
+
+        /// <summary>
+        /// Return the current position in the input
+        /// </summary>
+        /// <returns></returns>
+        long Tell();
+    }
+}

Added: avro/trunk/lang/csharp/src/apache/main/File/IFileWriter.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/File/IFileWriter.cs?rev=1498613&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/File/IFileWriter.cs (added)
+++ avro/trunk/lang/csharp/src/apache/main/File/IFileWriter.cs Mon Jul  1 18:04:37 2013
@@ -0,0 +1,85 @@
+/**
+ * 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;
+
+namespace Avro.File
+{
+    public interface IFileWriter<T> : IDisposable
+    {
+        /// <summary>
+        /// Append datum to a file / stream
+        /// </summary>
+        /// <param name="datum"></param>
+        void Append(T datum);
+
+        /// <summary>
+        /// Closes the file / stream
+        /// </summary>
+        void Close();
+
+        /// <summary>
+        /// Flush out any buffered data
+        /// </summary>
+        void Flush();
+
+        /// Returns true if parameter is a
+        /// reserved Avro meta data value
+        /// </summary>
+        /// <param name="key"></param>
+        /// <returns></returns>
+        bool IsReservedMeta(string key);
+
+        /// <summary>
+        /// Set meta data pair
+        /// </summary>
+        /// <param name="key"></param>
+        /// <param name="value"></param>
+        void SetMeta(String key, byte[] value);
+
+        /// <summary>
+        /// Set meta data pair (long value)
+        /// </summary>
+        /// <param name="key"></param>
+        /// <param name="value"></param>
+        void SetMeta(String key, long value);
+
+        /// <summary>
+        /// Set meta data pair (string value)
+        /// </summary>
+        /// <param name="key"></param>
+        /// <param name="value"></param>
+        void SetMeta(String key, string value);
+
+        /// <summary>
+        /// Set the synchronization interval for this 
+        /// file / stream, in bytes. Valid values range 
+        /// from 32 to 2^30. Suggested values are 
+        /// between 2K and 2M
+        /// </summary>
+        /// <param name="syncInterval"></param>
+        /// <returns></returns>
+        void SetSyncInterval(int syncInterval);
+
+        /// <summary>
+        /// Forces the end of the current block, 
+        /// emitting a synchronization marker
+        /// </summary>
+        /// <returns></returns>
+        long Sync();
+    }
+}

Added: avro/trunk/lang/csharp/src/apache/main/File/NullCodec.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/File/NullCodec.cs?rev=1498613&view=auto
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/File/NullCodec.cs (added)
+++ avro/trunk/lang/csharp/src/apache/main/File/NullCodec.cs Mon Jul  1 18:04:37 2013
@@ -0,0 +1,56 @@
+/**
+ * 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;
+
+namespace Avro.File
+{
+    public class NullCodec : Codec
+    {
+        public NullCodec() { }
+
+        public override byte[] Compress(byte[] uncompressedData)
+        {
+            return uncompressedData;
+        }
+
+        public override byte[] Decompress(byte[] compressedData)
+        {
+            return compressedData;
+        }
+
+        public override string GetName()
+        {
+            return DataFileConstants.NullCodec;
+        }
+
+        public override bool Equals(object other)
+        {
+            if (this == other)
+                return true;
+            return (this.GetType().Name == other.GetType().Name);
+        }
+
+        public override int GetHashCode()
+        {
+            return DataFileConstants.NullCodecHash;
+        }
+    }
+}

Modified: avro/trunk/lang/csharp/src/apache/main/Specific/SpecificReader.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/main/Specific/SpecificReader.cs?rev=1498613&r1=1498612&r2=1498613&view=diff
==============================================================================
--- avro/trunk/lang/csharp/src/apache/main/Specific/SpecificReader.cs (original)
+++ avro/trunk/lang/csharp/src/apache/main/Specific/SpecificReader.cs Mon Jul  1 18:04:37 2013
@@ -181,7 +181,8 @@ namespace Avro.Specific
         /// <returns>enum value</returns>
         protected override object ReadEnum(object reuse, EnumSchema writerSchema, Schema readerSchema, Decoder dec)
         {
-            return dec.ReadEnum();
+            EnumSchema rs = readerSchema as EnumSchema;
+            return rs.Ordinal(writerSchema[dec.ReadEnum()]);
         }
 
         /// <summary>

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=1498613&r1=1498612&r2=1498613&view=diff
==============================================================================
--- avro/trunk/lang/csharp/src/apache/test/Avro.test.csproj (original)
+++ avro/trunk/lang/csharp/src/apache/test/Avro.test.csproj Mon Jul  1 18:04:37 2013
@@ -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
@@ -77,12 +77,12 @@
     <Reference Include="System.Core" />
     <Reference Include="System.Xml.Linq" />
     <Reference Include="System.Data.DataSetExtensions" />
-    <Reference Include="Microsoft.CSharp" />
     <Reference Include="System.Data" />
     <Reference Include="System.Xml" />
   </ItemGroup>
   <ItemGroup>
     <Compile Include="CodGen\CodeGenTest.cs" />
+    <Compile Include="File\FileTests.cs" />
     <Compile Include="Generic\GenericTests.cs" />
     <Compile Include="IO\BinaryCodecTests.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
@@ -126,4 +126,4 @@
   <Target Name="AfterBuild">
   </Target>
   -->
-</Project>
\ No newline at end of file
+</Project>

Modified: avro/trunk/lang/csharp/src/apache/test/CodGen/CodeGenTest.cs
URL: http://svn.apache.org/viewvc/avro/trunk/lang/csharp/src/apache/test/CodGen/CodeGenTest.cs?rev=1498613&r1=1498612&r2=1498613&view=diff
==============================================================================
--- avro/trunk/lang/csharp/src/apache/test/CodGen/CodeGenTest.cs (original)
+++ avro/trunk/lang/csharp/src/apache/test/CodGen/CodeGenTest.cs Mon Jul  1 18:04:37 2013
@@ -98,10 +98,11 @@ namespace Avro.Test
             }
         }
 
+       
         [Test]
         public void CanCodeGenTraceProtocol()
         {
-            var traceProtocol = File.ReadAllText("../../../../../share/schemas/org/apache/avro/ipc/trace/avroTrace.avpr");
+            var traceProtocol = System.IO.File.ReadAllText("../../../../../share/schemas/org/apache/avro/ipc/trace/avroTrace.avpr");
             Protocol protocol = Protocol.Parse(traceProtocol);
             var compilerResults = GenerateProtocol(protocol);
 
@@ -114,6 +115,7 @@ namespace Avro.Test
             Assert.That(types.Contains("org.apache.avro.ipc.trace.TimestampedEvent"), "Should have contained TimestampedEvent type");
         }
 
+
         private static CompilerResults GenerateSchema(Schema schema)
         {
             var codegen = new CodeGen();
@@ -134,8 +136,6 @@ namespace Avro.Test
 
             var comparam = new CompilerParameters(new string[] { "mscorlib.dll" });
             comparam.ReferencedAssemblies.Add("System.dll");
-            comparam.ReferencedAssemblies.Add("System.Core.dll");
-            comparam.ReferencedAssemblies.Add(Type.GetType("Mono.Runtime") != null ? "Mono.CSharp.dll" : "Microsoft.CSharp.dll");
             comparam.ReferencedAssemblies.Add("Avro.dll");
             comparam.GenerateInMemory = true;
             var ccp = new CSharpCodeProvider();



Mime
View raw message