Browse Source

HADOOP-9140 Cleanup rpc PB protos (sanjay Radia)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1423189 13f79535-47bb-0310-9956-ffa450edef68
Sanjay Radia 12 years ago
parent
commit
aa4fe26a01

+ 2 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -144,6 +144,8 @@ Trunk (Unreleased)
     HADOOP-9093. Move all the Exception in PathExceptions to o.a.h.fs package.
     (suresh)
 
+    HADOOP-9140 Cleanup rpc PB protos (sanjay Radia)
+
   BUG FIXES
 
     HADOOP-9041. FsUrlStreamHandlerFactory could cause an infinite loop in

+ 2 - 2
hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml

@@ -260,7 +260,7 @@
     </Match>
     <Match>
       <!-- protobuf generated code -->
-      <Class name="~org\.apache\.hadoop\.ipc\.protobuf\.HadoopRpcProtos.*"/>
+      <Class name="~org\.apache\.hadoop\.ipc\.protobuf\.ProtobufRpcEngineProtos.*"/>
     </Match>
     <Match>
       <!-- protobuf generated code -->
@@ -272,7 +272,7 @@
     </Match>
         <Match>
       <!-- protobuf generated code -->
-      <Class name="~org\.apache\.hadoop\.ipc\.protobuf\.RpcPayloadHeaderProtos.*"/>
+      <Class name="~org\.apache\.hadoop\.ipc\.protobuf\.RpcHeaderProtos.*"/>
     </Match>
     <Match>
       <!-- protobuf generated code -->

+ 2 - 2
hadoop-common-project/hadoop-common/pom.xml

@@ -378,9 +378,9 @@
                 <argument>src/main/proto/HAServiceProtocol.proto</argument>
                 <argument>src/main/proto/IpcConnectionContext.proto</argument>
                 <argument>src/main/proto/ProtocolInfo.proto</argument>
-                <argument>src/main/proto/RpcPayloadHeader.proto</argument>
+                <argument>src/main/proto/RpcHeader.proto</argument>
                 <argument>src/main/proto/ZKFCProtocol.proto</argument>
-                <argument>src/main/proto/hadoop_rpc.proto</argument>
+                <argument>src/main/proto/ProtobufRpcEngine.proto</argument>
               </arguments>
             </configuration>
           </execution>

+ 20 - 20
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java

@@ -63,11 +63,10 @@ import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
-import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
-import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.RpcPayloadHeaderProto;
-import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.RpcPayloadOperationProto;
-import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.RpcResponseHeaderProto;
-import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.RpcStatusProto;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto.OperationProto;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.SaslRpcClient;
@@ -191,7 +190,7 @@ public class Client {
    */
   private class Call {
     final int id;               // call id
-    final Writable rpcRequest;  // the serialized rpc request - RpcPayload
+    final Writable rpcRequest;  // the serialized rpc request
     Writable rpcResponse;       // null if rpc has error
     IOException error;          // exception, null if success
     final RPC.RpcKind rpcKind;      // Rpc EngineKind
@@ -266,7 +265,7 @@ public class Client {
     private AtomicBoolean shouldCloseConnection = new AtomicBoolean();  // indicate if the connection is closed
     private IOException closeException; // close reason
     
-    private final Object sendParamsLock = new Object();
+    private final Object sendRpcRequestLock = new Object();
 
     public Connection(ConnectionId remoteId) throws IOException {
       this.remoteId = remoteId;
@@ -768,7 +767,7 @@ public class Client {
           remoteId.getTicket(),
           authMethod).writeTo(buf);
       
-      // Write out the payload length
+      // Write out the packet length
       int bufLen = buf.getLength();
 
       out.writeInt(bufLen);
@@ -832,7 +831,7 @@ public class Client {
 
       try {
         while (waitForWork()) {//wait here for work - read or close connection
-          receiveResponse();
+          receiveRpcResponse();
         }
       } catch (Throwable t) {
         // This truly is unexpected, since we catch IOException in receiveResponse
@@ -849,11 +848,12 @@ public class Client {
             + connections.size());
     }
 
-    /** Initiates a call by sending the parameter to the remote server.
+    /** Initiates a rpc call by sending the rpc request to the remote server.
      * Note: this is not called from the Connection thread, but by other
      * threads.
+     * @param call - the rpc request
      */
-    public void sendParam(final Call call)
+    public void sendRpcRequest(final Call call)
         throws InterruptedException, IOException {
       if (shouldCloseConnection.get()) {
         return;
@@ -866,17 +866,17 @@ public class Client {
       //
       // Format of a call on the wire:
       // 0) Length of rest below (1 + 2)
-      // 1) PayloadHeader  - is serialized Delimited hence contains length
-      // 2) the Payload - the RpcRequest
+      // 1) RpcRequestHeader  - is serialized Delimited hence contains length
+      // 2) RpcRequest
       //
       // Items '1' and '2' are prepared here. 
       final DataOutputBuffer d = new DataOutputBuffer();
-      RpcPayloadHeaderProto header = ProtoUtil.makeRpcPayloadHeader(
-         call.rpcKind, RpcPayloadOperationProto.RPC_FINAL_PAYLOAD, call.id);
+      RpcRequestHeaderProto header = ProtoUtil.makeRpcRequestHeader(
+         call.rpcKind, OperationProto.RPC_FINAL_PACKET, call.id);
       header.writeDelimitedTo(d);
       call.rpcRequest.write(d);
 
-      synchronized (sendParamsLock) {
+      synchronized (sendRpcRequestLock) {
         Future<?> senderFuture = SEND_PARAMS_EXECUTOR.submit(new Runnable() {
           @Override
           public void run() {
@@ -892,7 +892,7 @@ public class Client {
                 byte[] data = d.getData();
                 int totalLength = d.getLength();
                 out.writeInt(totalLength); // Total Length
-                out.write(data, 0, totalLength);//PayloadHeader + RpcRequest
+                out.write(data, 0, totalLength);// RpcRequestHeader + RpcRequest
                 out.flush();
               }
             } catch (IOException e) {
@@ -927,7 +927,7 @@ public class Client {
     /* Receive a response.
      * Because only one receiver, so no synchronization on in.
      */
-    private void receiveResponse() {
+    private void receiveRpcResponse() {
       if (shouldCloseConnection.get()) {
         return;
       }
@@ -1194,12 +1194,12 @@ public class Client {
     Call call = new Call(rpcKind, rpcRequest);
     Connection connection = getConnection(remoteId, call);
     try {
-      connection.sendParam(call);                 // send the parameter
+      connection.sendRpcRequest(call);                 // send the rpc request
     } catch (RejectedExecutionException e) {
       throw new IOException("connection has been closed", e);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      LOG.warn("interrupted waiting to send params to server", e);
+      LOG.warn("interrupted waiting to send rpc request to server", e);
       throw new IOException(e);
     }
 

+ 9 - 9
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java

@@ -39,7 +39,7 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.ipc.Client.ConnectionId;
 import org.apache.hadoop.ipc.RPC.RpcInvoker;
-import org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcRequestProto;
+import org.apache.hadoop.ipc.protobuf.ProtobufRpcEngineProtos.RequestProto;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -128,10 +128,10 @@ public class ProtobufRpcEngine implements RpcEngine {
           .getProtocolVersion(protocol);
     }
 
-    private HadoopRpcRequestProto constructRpcRequest(Method method,
+    private RequestProto constructRpcRequest(Method method,
         Object[] params) throws ServiceException {
-      HadoopRpcRequestProto rpcRequest;
-      HadoopRpcRequestProto.Builder builder = HadoopRpcRequestProto
+      RequestProto rpcRequest;
+      RequestProto.Builder builder = RequestProto
           .newBuilder();
       builder.setMethodName(method.getName());
 
@@ -190,7 +190,7 @@ public class ProtobufRpcEngine implements RpcEngine {
         startTime = Time.now();
       }
 
-      HadoopRpcRequestProto rpcRequest = constructRpcRequest(method, args);
+      RequestProto rpcRequest = constructRpcRequest(method, args);
       RpcResponseWritable val = null;
       
       if (LOG.isTraceEnabled()) {
@@ -271,13 +271,13 @@ public class ProtobufRpcEngine implements RpcEngine {
    * Writable Wrapper for Protocol Buffer Requests
    */
   private static class RpcRequestWritable implements Writable {
-    HadoopRpcRequestProto message;
+    RequestProto message;
 
     @SuppressWarnings("unused")
     public RpcRequestWritable() {
     }
 
-    RpcRequestWritable(HadoopRpcRequestProto message) {
+    RpcRequestWritable(RequestProto message) {
       this.message = message;
     }
 
@@ -292,7 +292,7 @@ public class ProtobufRpcEngine implements RpcEngine {
       int length = ProtoUtil.readRawVarint32(in);
       byte[] bytes = new byte[length];
       in.readFully(bytes);
-      message = HadoopRpcRequestProto.parseFrom(bytes);
+      message = RequestProto.parseFrom(bytes);
     }
     
     @Override
@@ -426,7 +426,7 @@ public class ProtobufRpcEngine implements RpcEngine {
       public Writable call(RPC.Server server, String connectionProtocolName,
           Writable writableRequest, long receiveTime) throws Exception {
         RpcRequestWritable request = (RpcRequestWritable) writableRequest;
-        HadoopRpcRequestProto rpcRequest = request.message;
+        RequestProto rpcRequest = request.message;
         String methodName = rpcRequest.getMethodName();
         
         

+ 9 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -80,7 +80,8 @@ import org.apache.hadoop.ipc.RPC.VersionMismatch;
 import org.apache.hadoop.ipc.metrics.RpcDetailedMetrics;
 import org.apache.hadoop.ipc.metrics.RpcMetrics;
 import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
-import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.*;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.*;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SaslRpcServer;
@@ -160,7 +161,7 @@ public abstract class Server {
   public static final ByteBuffer HEADER = ByteBuffer.wrap("hrpc".getBytes());
   
   /**
-   * Serialization type for ConnectionContext and RpcPayloadHeader
+   * Serialization type for ConnectionContext and RpcRequestHeader
    */
   public enum IpcSerializationType {
     // Add new serialization type to the end without affecting the enum order
@@ -197,7 +198,7 @@ public abstract class Server {
   // 4 : Introduced SASL security layer
   // 5 : Introduced use of {@link ArrayPrimitiveWritable$Internal}
   //     in ObjectWritable to efficiently transmit arrays of primitives
-  // 6 : Made RPC payload header explicit
+  // 6 : Made RPC Request header explicit
   // 7 : Changed Ipc Connection Header to use Protocol buffers
   // 8 : SASL server always sends a final response
   public static final byte CURRENT_VERSION = 8;
@@ -1637,14 +1638,15 @@ public abstract class Server {
     private void processData(byte[] buf) throws  IOException, InterruptedException {
       DataInputStream dis =
         new DataInputStream(new ByteArrayInputStream(buf));
-      RpcPayloadHeaderProto header = RpcPayloadHeaderProto.parseDelimitedFrom(dis);
+      RpcRequestHeaderProto header = RpcRequestHeaderProto.parseDelimitedFrom(dis);
         
       if (LOG.isDebugEnabled())
         LOG.debug(" got #" + header.getCallId());
       if (!header.hasRpcOp()) {
-        throw new IOException(" IPC Server: No rpc op in rpcPayloadHeader");
+        throw new IOException(" IPC Server: No rpc op in rpcRequestHeader");
       }
-      if (header.getRpcOp() != RpcPayloadOperationProto.RPC_FINAL_PAYLOAD) {
+      if (header.getRpcOp() != 
+          RpcRequestHeaderProto.OperationProto.RPC_FINAL_PACKET) {
         throw new IOException("IPC Server does not implement operation" + 
               header.getRpcOp());
       }
@@ -1652,7 +1654,7 @@ public abstract class Server {
       // (Note it would make more sense to have the handler deserialize but 
       // we continue with this original design.
       if (!header.hasRpcKind()) {
-        throw new IOException(" IPC Server: No rpc kind in rpcPayloadHeader");
+        throw new IOException(" IPC Server: No rpc kind in rpcRequestHeader");
       }
       Class<? extends Writable> rpcRequestClass = 
           getRpcRequestWrapper(header.getRpcKind());

+ 4 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java

@@ -24,7 +24,7 @@ import java.io.IOException;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
 import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.UserInformationProto;
-import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.*;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.*;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -157,9 +157,9 @@ public abstract class ProtoUtil {
     return null;
   }
  
-  public static RpcPayloadHeaderProto makeRpcPayloadHeader(RPC.RpcKind rpcKind,
-      RpcPayloadOperationProto operation, int callId) {
-    RpcPayloadHeaderProto.Builder result = RpcPayloadHeaderProto.newBuilder();
+  public static RpcRequestHeaderProto makeRpcRequestHeader(RPC.RpcKind rpcKind,
+      RpcRequestHeaderProto.OperationProto operation, int callId) {
+    RpcRequestHeaderProto.Builder result = RpcRequestHeaderProto.newBuilder();
     result.setRpcKind(convert(rpcKind)).setRpcOp(operation).setCallId(callId);
     return result.build();
   }

+ 9 - 6
hadoop-common-project/hadoop-common/src/main/proto/hadoop_rpc.proto → hadoop-common-project/hadoop-common/src/main/proto/ProtobufRpcEngine.proto

@@ -17,11 +17,13 @@
  */
 
 /**
- * These are the messages used by Hadoop RPC to marshal the
- * request and response in the RPC layer.
+ * These are the messages used by Hadoop RPC for the Rpc Engine Protocol Buffer
+ * to marshal the request and response in the RPC layer.
+ * The messages are sent in addition to the normal RPC header as 
+ * defined in RpcHeader.proto
  */
 option java_package = "org.apache.hadoop.ipc.protobuf";
-option java_outer_classname = "HadoopRpcProtos";
+option java_outer_classname = "ProtobufRpcEngineProtos";
 option java_generate_equals_and_hash = true;
 package hadoop.common;
 
@@ -29,10 +31,11 @@ package hadoop.common;
  * This message is used for Protobuf Rpc Engine.
  * The message is used to marshal a Rpc-request
  * from RPC client to the RPC server.
- * The Response to the Rpc call (including errors) are handled
- * as part of the standard Rpc response. 
+ *
+ * No special header is needed for the Rpc Response for Protobuf Rpc Engine.
+ * The normal RPC response header (see RpcHeader.proto) are sufficient. 
  */
-message HadoopRpcRequestProto {
+message RequestProto {
   /** Name of the RPC method */
   required string methodName = 1;
 

+ 92 - 0
hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto

@@ -0,0 +1,92 @@
+/**
+ * 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.
+ */
+option java_package = "org.apache.hadoop.ipc.protobuf";
+option java_outer_classname = "RpcHeaderProtos";
+option java_generate_equals_and_hash = true;
+package hadoop.common;
+
+/**
+ * This is the rpc request header. It is sent with every rpc call.
+ * 
+ * The format of RPC call is as follows:
+ * +--------------------------------------------------------------+
+ * | Rpc length in bytes (4 bytes int) sum of next two parts      |
+ * +--------------------------------------------------------------+
+ * | RpcRequestHeaderProto - serialized delimited ie has len      |
+ * +--------------------------------------------------------------+
+ * | RpcRequest The actual rpc request                            |
+ * | This request is serialized based on RpcKindProto             |
+ * +--------------------------------------------------------------+
+ *
+ */
+
+/**
+ * RpcKind determine the rpcEngine and the serialization of the rpc request
+ */
+enum RpcKindProto {
+  RPC_BUILTIN          = 0;  // Used for built in calls by tests
+  RPC_WRITABLE         = 1;  // Use WritableRpcEngine 
+  RPC_PROTOCOL_BUFFER  = 2;  // Use ProtobufRpcEngine
+}
+
+
+   
+message RpcRequestHeaderProto { // the header for the RpcRequest
+  enum OperationProto {
+    RPC_FINAL_PACKET        = 0; // The final RPC Packet
+    RPC_CONTINUATION_PACKET = 1; // not implemented yet
+    RPC_CLOSE_CONNECTION     = 2; // close the rpc connection
+  }
+
+  optional RpcKindProto rpcKind = 1;
+  optional OperationProto rpcOp = 2;
+  required uint32 callId = 3; // each rpc has a callId that is also used in response
+}
+
+
+
+/**
+ * Rpc Response Header
+ * ** If request is successfull response is returned as below ********
+ * +------------------------------------------------------------------+
+ * | Rpc reponse length in bytes (4 bytes int)                        |
+ * |  (sum of next two parts)                                         |
+ * +------------------------------------------------------------------+
+ * | RpcResponseHeaderProto - serialized delimited ie has len         |
+ * +------------------------------------------------------------------+
+ * | if request is successful:                                        |
+ * |   - RpcResponse -  The actual rpc response  bytes                 |
+ * |     This response is serialized based on RpcKindProto             |
+ * | if request fails :                                               |
+ * |   - length (4 byte int) + Class name of exception - UTF-8 string |
+ * |   - length (4 byte int) + Stacktrace - UTF-8 string              |
+ * |   if the strings are null then the length is -1                  |
+ * +------------------------------------------------------------------+
+ *
+ */
+message RpcResponseHeaderProto {
+  enum RpcStatusProto {
+   SUCCESS = 0;  // RPC succeeded
+   ERROR = 1;    // RPC Failed
+   FATAL = 2;    // Fatal error - connection is closed
+  }
+
+  required uint32 callId = 1; // callId used in Request
+  required RpcStatusProto status = 2;
+  optional uint32 serverIpcVersionNum = 3; // in case of an fatal IPC error 
+}

+ 0 - 78
hadoop-common-project/hadoop-common/src/main/proto/RpcPayloadHeader.proto

@@ -1,78 +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.
- */
-option java_package = "org.apache.hadoop.ipc.protobuf";
-option java_outer_classname = "RpcPayloadHeaderProtos";
-option java_generate_equals_and_hash = true;
-package hadoop.common;
-
-/**
- * This is the rpc payload header. It is sent with every rpc call.
- * 
- * The format of RPC call is as follows:
- * +-----------------------------------------------------+
- * |  Rpc length in bytes                                |
- * +-----------------------------------------------------+
- * | RpcPayloadHeader - serialized delimited ie has len  |
- * +-----------------------------------------------------+
- * |  RpcRequest Payload                                 |
- * +-----------------------------------------------------+
- *
- */
-
-/**
- * RpcKind determine the rpcEngine and the serialization of the rpc payload
- */
-enum RpcKindProto {
-  RPC_BUILTIN          = 0;  // Used for built in calls by tests
-  RPC_WRITABLE         = 1;  // Use WritableRpcEngine 
-  RPC_PROTOCOL_BUFFER  = 2;  // Use ProtobufRpcEngine
-}
-
-enum RpcPayloadOperationProto {
-  RPC_FINAL_PAYLOAD        = 0; // The final payload
-  RPC_CONTINUATION_PAYLOAD = 1; // not implemented yet
-  RPC_CLOSE_CONNECTION     = 2; // close the rpc connection
-}
-   
-message RpcPayloadHeaderProto { // the header for the RpcRequest
-  optional RpcKindProto rpcKind = 1;
-  optional RpcPayloadOperationProto rpcOp = 2;
-  required uint32 callId = 3; // each rpc has a callId that is also used in response
-}
-
-enum RpcStatusProto {
- SUCCESS = 0;  // RPC succeeded
- ERROR = 1;    // RPC Failed
- FATAL = 2;    // Fatal error - connection is closed
-}
-
-/**
- * Rpc Response Header
- *    - If successfull then the Respose follows after this header
- *        - length (4 byte int), followed by the response
- *    - If error or fatal - the exception info follow
- *        - length (4 byte int) Class name of exception - UTF-8 string
- *        - length (4 byte int) Stacktrace - UTF-8 string
- *        - if the strings are null then the length is -1
- * In case of Fatal error then the respose contains the Serverside's IPC version
- */
-message RpcResponseHeaderProto {
-  required uint32 callId = 1; // callId used in Request
-  required RpcStatusProto status = 2;
-  optional uint32 serverIpcVersionNum = 3; // in case of an fatal IPC error 
-}