You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by we...@apache.org on 2017/08/09 14:40:25 UTC

reef git commit: [REEF-1859] Support Avro code generation of multiple C# files from single avsc.

Repository: reef
Updated Branches:
  refs/heads/master 18c24b1c7 -> 46911db2a


[REEF-1859] Support Avro code generation of multiple C# files from single avsc.

  * This task modifies the code generation task to support generating
    multiple C# program files from a single avsc file that has multiple
    parallel and nested types.
  * Adds the bridge message wrapper avsc file that code gens to four C# files
    and provides the framework for the bridge Java/CLR Avro protocol.

JIRA:
  [REEF-1859](https://issues.apache.org/jira/browse/REEF-1859)

Pull request:
  This closes #1359


Project: http://git-wip-us.apache.org/repos/asf/reef/repo
Commit: http://git-wip-us.apache.org/repos/asf/reef/commit/46911db2
Tree: http://git-wip-us.apache.org/repos/asf/reef/tree/46911db2
Diff: http://git-wip-us.apache.org/repos/asf/reef/diff/46911db2

Branch: refs/heads/master
Commit: 46911db2a5fb3aec7990db6d07491a71e9cf59e7
Parents: 18c24b1
Author: Doug Service <do...@microsoft.com>
Authored: Tue Aug 8 03:00:17 2017 +0000
Committer: Markus Weimer <we...@apache.org>
Committed: Wed Aug 9 07:38:55 2017 -0700

----------------------------------------------------------------------
 lang/common/bridge/avro/Acknowledgement.avsc    | 36 ---------
 lang/common/bridge/avro/BridgeInterop.avsc      | 74 +++++++++++++++++
 lang/common/bridge/avro/Header.avsc             | 41 ----------
 lang/common/bridge/avro/SystemOnStart.avsc      | 36 ---------
 lang/cs/AvroCodeGeneration.targets              | 80 +++++++++++--------
 .../Message/BridgeInterop.cs                    | 83 ++++++++++++++++++++
 .../Message/MessageType.cs                      | 47 +++++++++++
 .../Message/SetupBridge.cs                      | 66 ++++++++++++++++
 .../Message/SystemOnStart.cs                    |  2 +-
 .../Org.Apache.REEF.Bridge.CLR.csproj           |  3 +
 10 files changed, 321 insertions(+), 147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/46911db2/lang/common/bridge/avro/Acknowledgement.avsc
----------------------------------------------------------------------
diff --git a/lang/common/bridge/avro/Acknowledgement.avsc b/lang/common/bridge/avro/Acknowledgement.avsc
deleted file mode 100644
index 31c9e53..0000000
--- a/lang/common/bridge/avro/Acknowledgement.avsc
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 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.
- */
-[
-  /*
-   * Acknowledgement that a specific message was received and processed.
-   */
-  {
-    "namespace":"org.apache.reef.bridge.message",
-    "type":"record",
-    "name":"Acknowledgement",
-    "doc":"The Acknowledgement message is sent to the Java bridge to acknowledge receipt and processing of a specific message.",
-    "fields":[
-      {
-        "name":"messageIdentifier",
-        "type":"long",
-        "doc":"The message identifier of the message that was successfully processed."
-      }
-    ]
-  }
-]

http://git-wip-us.apache.org/repos/asf/reef/blob/46911db2/lang/common/bridge/avro/BridgeInterop.avsc
----------------------------------------------------------------------
diff --git a/lang/common/bridge/avro/BridgeInterop.avsc b/lang/common/bridge/avro/BridgeInterop.avsc
new file mode 100644
index 0000000..357191a
--- /dev/null
+++ b/lang/common/bridge/avro/BridgeInterop.avsc
@@ -0,0 +1,74 @@
+/*
+ * 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.
+ */
+[
+  /*
+   * Message protocol for the Java/C# bridge communications.
+   */
+  { "type": "record", "name": "SetupBridge", "namespace": "org.apache.reef.bridge.message",
+    "doc": "Notify the C# bridge of the http port of the Java bridge webserver.",
+    "fields":
+    [
+      { "name": "httpServerPortNumber", "type": "int", "doc": "The Java bridge http server port number." }
+    ]
+  },
+  { "type": "record", "name": "SystemOnStart", "namespace": "org.apache.reef.bridge.message",
+    "doc": "Notify the C# bridge the system is now running.",
+    "fields":
+    [
+      { "name": "dateTime", "type": "long", "doc": "Date time in seconds as a long since January 1, 1970" }
+    ]
+  },
+  { "type":"record", "name":"Acknowledgement", "namespace":"org.apache.reef.bridge.message",
+    "doc":"The Acknowledgement message is sent to the Java bridge to acknowledge receipt and processing of a specific message.",
+    "fields":[
+      {
+        "name":"messageIdentifier",
+        "type":"long",
+        "doc":"The message identifier of the message that was successfully processed."
+      }
+    ]
+  },
+  { "type": "record", "name":"BridgeInterop", "namespace": "org.apache.reef.bridge.message",
+    "doc": "Container message for all Java/CLR bridge messages in the protocol.",
+    "fields": [
+      { "name": "sequence", "type": "long", "doc": "The unique squence identifier of the message in the protocol stream." },
+      { "name": "messageType", "type":
+        { "type": "enum", "name": "MessageType",
+          "doc": "An enumeration of all possible Java/C# bridge protocol messages.",
+          "symbols":
+          [
+            "SetupBridge",
+            "SystemOnStart",
+            "Acknowledgement"
+          ]
+        },
+         "doc": "The type of the bridge Java/CLR interop message." },
+      { "name": "message",
+        "doc": "A union which contains the actual message.",
+        "type":
+        [
+          "SetupBridge",
+          "SystemOnStart",
+          "Acknowledgement",
+          { "type": "array", "items": "bytes"}
+        ]
+      }
+    ]
+  }
+]

http://git-wip-us.apache.org/repos/asf/reef/blob/46911db2/lang/common/bridge/avro/Header.avsc
----------------------------------------------------------------------
diff --git a/lang/common/bridge/avro/Header.avsc b/lang/common/bridge/avro/Header.avsc
deleted file mode 100644
index c543f0b..0000000
--- a/lang/common/bridge/avro/Header.avsc
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.
- */
-[
-  /*
-   * Identify the next message in the Java/C# bridge protocol.
-   */
-  {
-    "namespace":"org.apache.reef.bridge.message",
-    "type":"record",
-    "name":"Header",
-    "doc":"Identifies the following message in the Java/C# bridge protocol.",
-    "fields":[
-      {
-        "name":"identifier",
-        "doc":"Identifier of the next message to be read.",
-        "type":"long"
-      },
-      {
-        "name":"className",
-        "doc":"The fully qualified name of the message class.",
-        "type":"string"
-      }
-    ]
-  }
-]

http://git-wip-us.apache.org/repos/asf/reef/blob/46911db2/lang/common/bridge/avro/SystemOnStart.avsc
----------------------------------------------------------------------
diff --git a/lang/common/bridge/avro/SystemOnStart.avsc b/lang/common/bridge/avro/SystemOnStart.avsc
deleted file mode 100644
index 8f25fa1..0000000
--- a/lang/common/bridge/avro/SystemOnStart.avsc
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 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.
- */
-[
-  /*
-   * First message from the Java bridge to the CLR bridge.
-   */
-  {
-    "namespace":"org.apache.reef.bridge.message",
-    "type":"record",
-    "name":"SystemOnStart",
-    "doc":"The SystemOnStart message is sent to the CLR bridge from the Java bridge when the round trip communication path has been established.",
-    "fields":[
-      {
-        "name":"dateTime",
-        "type":"long",
-        "doc":"Date time in seconds as a long since January 1, 1970"
-      }
-    ]
-  }
-]

http://git-wip-us.apache.org/repos/asf/reef/blob/46911db2/lang/cs/AvroCodeGeneration.targets
----------------------------------------------------------------------
diff --git a/lang/cs/AvroCodeGeneration.targets b/lang/cs/AvroCodeGeneration.targets
index c6599b2..c753ac3 100644
--- a/lang/cs/AvroCodeGeneration.targets
+++ b/lang/cs/AvroCodeGeneration.targets
@@ -14,8 +14,8 @@ software distributed under the License is distributed on an
 KIND, either express or implied.  See the License for the
 specific language governing permissions and limitations
 under the License.
---> 
-<!-- AVRO CODE GERNATION TARGETS FOR MSBUILD -->   
+-->
+<!-- AVRO CODE GERNATION TARGETS FOR MSBUILD -->
 <Project ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
   <UsingTask TaskName="AvscToCSharp" TaskFactory="CodeTaskFactory" AssemblyFile="$(MSBuildToolsPath)\Microsoft.Build.Tasks.v4.0.dll">
     <ParameterGroup>
@@ -39,19 +39,19 @@ under the License.
         // every message defined in the Messages directory.
 
         // Create a temporary working directory.
-        String tempDir = Path.GetFullPath(Path.Combine(Path.GetTempPath(), Guid.NewGuid().ToString()));
+        string tempDir = Path.GetFullPath(Path.Combine(Path.GetTempPath(), Guid.NewGuid().ToString()));
         Directory.CreateDirectory(tempDir);
-        String[] sourceFiles = SourceFileList.Split(new char[]{';'});
+        string[] sourceFiles = SourceFileList.Split(new char[]{';'});
 
         // Get the full path where the avro avsc files are located.
-        String messageDirectory = Path.GetFullPath(AvroSchemaDirectory);
+        string messageDirectory = Path.GetFullPath(AvroSchemaDirectory);
 
         // Add the avro tools directory to the path.
-        String path = Environment.GetEnvironmentVariable("path");
+        string path = Environment.GetEnvironmentVariable("path");
         path = Path.GetFullPath(AvroBinaryDirectory) + ";" + path;
         Environment.SetEnvironmentVariable("path", path);
 
-        String copyright = String.Join(Environment.NewLine,
+        string copyright = string.Join(Environment.NewLine,
           "// 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",
@@ -75,6 +75,7 @@ under the License.
             RedirectStandardOutput = true,
             RedirectStandardError = true,
             UseShellExecute = false,
+            CreateNoWindow = true,
             FileName = "Microsoft.Avro.Tools.exe"
         };
 
@@ -82,41 +83,56 @@ under the License.
         bool conversionError = false;
         using (StreamWriter logWriter = new StreamWriter(Path.Combine(tempDir, "avroout.log")))
         {
-          logWriter.WriteLine(Environment.GetEnvironmentVariable("path"));
-          foreach (String file in sourceFiles)
+          foreach (string file in sourceFiles)
           {
             if (file.Contains(@"Message\"))
             {
               // Build the name of the source avsc file from the destiname CS file and avro message path.
-              String avroSourceName = Path.Combine(messageDirectory, (Path.GetFileName(file)).Replace(".cs", ".avsc"));
+              string avroSourceName = Path.Combine(messageDirectory, (Path.GetFileName(file)).Replace(".cs", ".avsc"));
               rProcInfo.Arguments = String.Format(" CodeGen /I:{0} /O:{1}", avroSourceName, tempDir);
               logWriter.Write(rProcInfo.FileName);
               logWriter.WriteLine(rProcInfo.Arguments);
 
-              StringBuilder stdOutBuilder = new StringBuilder();
-              StringBuilder stdErrBuilder = new StringBuilder();
-
-              // Start the conversion process
-              using (Process rProc = Process.Start(rProcInfo))
+              // Check for for a matching file in the avsc directory.
+              // This may not be the case when a single avsc file
+              // defines multiple types.
+              if (File.Exists(avroSourceName))
               {
-                  stdOutBuilder.Append(rProc.StandardOutput.ReadToEnd());
-                  stdErrBuilder.Append(rProc.StandardError.ReadToEnd());
-                  rProc.WaitForExit();
-                  if (rProc.ExitCode != 0)
-                  {
-                    conversionError = true;
-                  }
-              }
-              logWriter.WriteLine(stdOutBuilder.ToString());
-              logWriter.WriteLine(stdErrBuilder.ToString());
+                StringBuilder stdOutBuilder = new StringBuilder();
+                StringBuilder stdErrBuilder = new StringBuilder();
 
-              using (StreamReader tmpReader = new StreamReader(Path.Combine(tempDir, Path.GetFileName(file))))
-              {
-                String source = tmpReader.ReadToEnd();
-                using (StreamWriter destWriter = new StreamWriter(Path.GetFullPath(file)))
+                // Start the conversion process
+                using (Process rProc = Process.Start(rProcInfo))
+                {
+                    stdOutBuilder.Append(rProc.StandardOutput.ReadToEnd());
+                    stdErrBuilder.Append(rProc.StandardError.ReadToEnd());
+                    rProc.WaitForExit();
+                    if (rProc.ExitCode != 0)
+                    {
+                      conversionError = true;
+                    }
+                }
+                logWriter.WriteLine(stdOutBuilder.ToString());
+                logWriter.WriteLine(stdErrBuilder.ToString());
+
+                if (!conversionError)
                 {
-                  destWriter.WriteLine(copyright);
-                  destWriter.Write(source);
+                  // Get all of the generated C# files and copy to destination with copyright.
+                  var genFiles = Directory.EnumerateFiles(tempDir, "*.cs");
+                  foreach (string genFile in genFiles)
+                  {
+                    logWriter.WriteLine("Processing file {0}", genFile);
+                    using (StreamReader tmpReader = new StreamReader(genFile))
+                    {
+                      using (StreamWriter destWriter =
+                        new StreamWriter(Path.Combine(Path.GetDirectoryName(file), Path.GetFileName(genFile))))
+                      {
+                        destWriter.WriteLine(copyright);
+                        destWriter.Write(tmpReader.ReadToEnd());
+                      }
+                    }
+                    File.Delete(genFile);
+                  }
                 }
               }
             }
@@ -140,5 +156,3 @@ under the License.
     <AvscToCSharp ProjectDirectory="$(ProjectDir)" SourceFileList="@(Compile)" AvroSchemaDirectory="$(AvroSchemaDirectory)" AvroBinaryDirectory="$(AvroBinaryDirectory)" />
   </Target>
 </Project>
-
-

http://git-wip-us.apache.org/repos/asf/reef/blob/46911db2/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/BridgeInterop.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/BridgeInterop.cs b/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/BridgeInterop.cs
new file mode 100644
index 0000000..2e862a7
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/BridgeInterop.cs
@@ -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.
+//<auto-generated />
+namespace org.apache.reef.bridge.message
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Runtime.Serialization;
+    using Microsoft.Hadoop.Avro;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro record org.apache.reef.bridge.message.BridgeInterop.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.bridge.message")]
+    public partial class BridgeInterop
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.bridge.message.BridgeInterop"",""doc"":""Container message for all Java/CLR bridge messages in the protocol."",""fields"":[{""name"":""sequence"",""doc"":""The unique squence identifier of the message in the protocol stream."",""type"":""long""},{""name"":""messageType"",""doc"":""The type of the bridge Java/CLR interop message."",""type"":{""type"":""enum"",""name"":""org.apache.reef.bridge.message.MessageType"",""doc"":""An enumeration of all possible Java/C# bridge protocol messages."",""symbols"":[""SetupBridge"",""SystemOnStart"",""Acknowledgement""]}},{""name"":""message"",""doc"":""A union which contains the actual message."",""type"":[{""type"":""record"",""name"":""org.apache.reef.bridge.message.SetupBridge"",""doc"":""Notify the C# bridge of the http port of the Java bridge webserver."",""fields"":[{""name"":""httpServerPortNumber"",""doc"":""The Java bridge http server port number.
 "",""type"":""int""}]},{""type"":""record"",""name"":""org.apache.reef.bridge.message.SystemOnStart"",""doc"":""Notify the C# bridge the system is now running."",""fields"":[{""name"":""dateTime"",""doc"":""Date time in seconds as a long since January 1, 1970"",""type"":""long""}]},{""type"":""record"",""name"":""org.apache.reef.bridge.message.Acknowledgement"",""doc"":""The Acknowledgement message is sent to the Java bridge to acknowledge receipt and processing of a specific message."",""fields"":[{""name"":""messageIdentifier"",""doc"":""The message identifier of the message that was successfully processed."",""type"":""long""}]},{""type"":""array"",""items"":""bytes""}]}]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+      
+        /// <summary>
+        /// Gets or sets the sequence field.
+        /// </summary>
+        [DataMember]
+        public long sequence { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the messageType field.
+        /// </summary>
+        [DataMember]
+        public org.apache.reef.bridge.message.MessageType messageType { get; set; }
+              
+        /// <summary>
+        /// Gets or sets the message field.
+        /// </summary>
+        [DataMember]
+        [AvroUnion(typeof(org.apache.reef.bridge.message.SetupBridge), typeof(org.apache.reef.bridge.message.SystemOnStart), typeof(org.apache.reef.bridge.message.Acknowledgement), typeof(List<byte[]>))]
+        public object message { get; set; }
+                
+        /// <summary>
+        /// Initializes a new instance of the <see cref="BridgeInterop"/> class.
+        /// </summary>
+        public BridgeInterop()
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="BridgeInterop"/> class.
+        /// </summary>
+        /// <param name="sequence">The sequence.</param>
+        /// <param name="messageType">The messageType.</param>
+        /// <param name="message">The message.</param>
+        public BridgeInterop(long sequence, org.apache.reef.bridge.message.MessageType messageType, object message)
+        {
+            this.sequence = sequence;
+            this.messageType = messageType;
+            this.message = message;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/46911db2/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/MessageType.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/MessageType.cs b/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/MessageType.cs
new file mode 100644
index 0000000..be59777
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/MessageType.cs
@@ -0,0 +1,47 @@
+// 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.
+//<auto-generated />
+namespace org.apache.reef.bridge.message
+{
+    using System.Runtime.Serialization;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro enum org.apache.reef.bridge.message.MessageType.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.bridge.message")]
+    public enum MessageType
+    {
+        /// <summary>
+        /// The SetupBridge symbol.
+        /// </summary>
+        [EnumMember]
+        SetupBridge,
+
+        /// <summary>
+        /// The SystemOnStart symbol.
+        /// </summary>
+        [EnumMember]
+        SystemOnStart,
+
+        /// <summary>
+        /// The Acknowledgement symbol.
+        /// </summary>
+        [EnumMember]
+        Acknowledgement,
+
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/46911db2/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/SetupBridge.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/SetupBridge.cs b/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/SetupBridge.cs
new file mode 100644
index 0000000..0694d49
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/SetupBridge.cs
@@ -0,0 +1,66 @@
+// 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.
+//<auto-generated />
+namespace org.apache.reef.bridge.message
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Runtime.Serialization;
+    using Microsoft.Hadoop.Avro;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro record org.apache.reef.bridge.message.SetupBridge.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.bridge.message")]
+    public partial class SetupBridge
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.bridge.message.SetupBridge"",""doc"":""Notify the C# bridge of the http port of the Java bridge webserver."",""fields"":[{""name"":""httpServerPortNumber"",""doc"":""The Java bridge http server port number."",""type"":""int""}]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+      
+        /// <summary>
+        /// Gets or sets the httpServerPortNumber field.
+        /// </summary>
+        [DataMember]
+        public int httpServerPortNumber { get; set; }
+                
+        /// <summary>
+        /// Initializes a new instance of the <see cref="SetupBridge"/> class.
+        /// </summary>
+        public SetupBridge()
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="SetupBridge"/> class.
+        /// </summary>
+        /// <param name="httpServerPortNumber">The httpServerPortNumber.</param>
+        public SetupBridge(int httpServerPortNumber)
+        {
+            this.httpServerPortNumber = httpServerPortNumber;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/46911db2/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/SystemOnStart.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/SystemOnStart.cs b/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/SystemOnStart.cs
index b066679..a719017 100644
--- a/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/SystemOnStart.cs
+++ b/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/SystemOnStart.cs
@@ -28,7 +28,7 @@ namespace org.apache.reef.bridge.message
     [DataContract(Namespace = "org.apache.reef.bridge.message")]
     public partial class SystemOnStart
     {
-        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.bridge.message.SystemOnStart"",""doc"":""The SystemOnStart message is sent to the CLR bridge from the Java bridge when the round trip communication path has been established."",""fields"":[{""name"":""dateTime"",""doc"":""Date time in seconds as a long since January 1, 1970"",""type"":""long""}]}";
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.bridge.message.SystemOnStart"",""doc"":""Notify the C# bridge the system is now running."",""fields"":[{""name"":""dateTime"",""doc"":""Date time in seconds as a long since January 1, 1970"",""type"":""long""}]}";
 
         /// <summary>
         /// Gets the schema.

http://git-wip-us.apache.org/repos/asf/reef/blob/46911db2/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj b/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj
index fca5894..dbbe571 100644
--- a/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj
+++ b/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj
@@ -54,7 +54,10 @@ under the License.
       <Link>Properties\SharedAssemblyInfo.cs</Link>
     </Compile>
     <Compile Include="Message\Acknowledgement.cs" />
+    <Compile Include="Message\BridgeInterop.cs" />
+    <Compile Include="Message\MessageType.cs" />
     <Compile Include="Message\Protocol.cs" />
+    <Compile Include="Message\SetupBridge.cs" />
     <Compile Include="Message\SystemOnStart.cs" />
   </ItemGroup>
   <ItemGroup>