You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@qpid.apache.org by kp...@apache.org on 2007/01/30 17:40:42 UTC

svn commit: r501456 - in /incubator/qpid/branches/qpid.0-9: cpp/gen/ gentools/src/org/apache/qpid/gentools/ java/client/src/main/java/org/apache/qpid/client/ java/common/ java/common/src/main/java/org/apache/qpid/framing/ specs/

Author: kpvdr
Date: Tue Jan 30 08:40:41 2007
New Revision: 501456

URL: http://svn.apache.org/viewvc?view=rev&rev=501456
Log:
Separated the errata in the XML spec file and put them into a file amqp-errara.0-9.xml. Added this file to the codegen. Fixed minor codegen bug in which fields in merged classes have repeating ordinal numbers. Added missing mandatory field to Message.Transger in the new errata file.

Added:
    incubator/qpid/branches/qpid.0-9/specs/amqp-errata.0-9.xml
Modified:
    incubator/qpid/branches/qpid.0-9/cpp/gen/Makefile.am
    incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/AmqpClass.java
    incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/AmqpMethod.java
    incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/Main.java
    incubator/qpid/branches/qpid.0-9/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer.java
    incubator/qpid/branches/qpid.0-9/java/common/protocol-version.xml
    incubator/qpid/branches/qpid.0-9/java/common/src/main/java/org/apache/qpid/framing/RequestManager.java
    incubator/qpid/branches/qpid.0-9/specs/amqp.0-9.xml

Modified: incubator/qpid/branches/qpid.0-9/cpp/gen/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/qpid/branches/qpid.0-9/cpp/gen/Makefile.am?view=diff&rev=501456&r1=501455&r2=501456
==============================================================================
--- incubator/qpid/branches/qpid.0-9/cpp/gen/Makefile.am (original)
+++ incubator/qpid/branches/qpid.0-9/cpp/gen/Makefile.am Tue Jan 30 08:40:41 2007
@@ -19,8 +19,8 @@
 
 # FIXME aconway 2007-01-04: Enabling Basic class until
 # new messaging class is ready to replace it.
-# spec = $(spec_dir)/amqp.0-9.xml $(spec_dir)/amqp-nogen.0-9.xml
-spec = $(spec_dir)/amqp.0-9.xml 
+# spec = $(spec_dir)/amqp.0-9.xml $(spec_dir)/amqp-errata.0-9.xml $(spec_dir)/amqp-nogen.0-9.xml
+spec = $(spec_dir)/amqp.0-9.xml $(spec_dir)/amqp-errata.0-9.xml
 
 gentools_srcdir = $(gentools_dir)/src/org/apache/qpid/gentools
 

Modified: incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/AmqpClass.java
URL: http://svn.apache.org/viewvc/incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/AmqpClass.java?view=diff&rev=501456&r1=501455&r2=501456
==============================================================================
--- incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/AmqpClass.java (original)
+++ incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/AmqpClass.java Tue Jan 30 08:40:41 2007
@@ -59,7 +59,7 @@
 			indexMap.put(index, indexVersionSet);
 		}
 		NodeList nList = classNode.getChildNodes();
-		int fieldCntr = 0;
+		int fieldCntr = fieldMap.size();
 		for (int i=0; i<nList.getLength(); i++)
 		{
 			Node child = nList.item(i);
@@ -82,7 +82,6 @@
 					thisField.removeVersion(version);
 					fieldMap.remove(fieldName);
 				}
-				fieldCntr++;
 			}
 			else if (child.getNodeName().compareTo(Utils.ELEMENT_METHOD) == 0)
 			{

Modified: incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/AmqpMethod.java
URL: http://svn.apache.org/viewvc/incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/AmqpMethod.java?view=diff&rev=501456&r1=501455&r2=501456
==============================================================================
--- incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/AmqpMethod.java (original)
+++ incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/AmqpMethod.java Tue Jan 30 08:40:41 2007
@@ -64,7 +64,7 @@
 			indexMap.put(index, indexVersionSet);
 		}
 		NodeList nList = methodNode.getChildNodes();
-		int fieldCntr = 0;
+		int fieldCntr = fieldMap.size();
 		isRequest = false; // Assume not a request  unless we find a response node.
 		for (int i=0; i<nList.getLength(); i++)
 		{

Modified: incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/Main.java
URL: http://svn.apache.org/viewvc/incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/Main.java?view=diff&rev=501456&r1=501455&r2=501456
==============================================================================
--- incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/Main.java (original)
+++ incubator/qpid/branches/qpid.0-9/gentools/src/org/apache/qpid/gentools/Main.java Tue Jan 30 08:40:41 2007
@@ -202,7 +202,7 @@
             new File(tmplDir + Utils.fileSeparator + "AMQP_Constants.h.tmpl"),
             new File(tmplDir + Utils.fileSeparator + "AMQP_MethodVersionMap.h.tmpl"),
             new File(tmplDir + Utils.fileSeparator + "AMQP_MethodVersionMap.cpp.tmpl"),
-           new File(tmplDir + Utils.fileSeparator + "AMQP_HighestVersion.h.tmpl")
+            new File(tmplDir + Utils.fileSeparator + "AMQP_HighestVersion.h.tmpl")
         };
         methodTemplateFiles = new File[]
         {

Modified: incubator/qpid/branches/qpid.0-9/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer.java
URL: http://svn.apache.org/viewvc/incubator/qpid/branches/qpid.0-9/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer.java?view=diff&rev=501456&r1=501455&r2=501456
==============================================================================
--- incubator/qpid/branches/qpid.0-9/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer.java (original)
+++ incubator/qpid/branches/qpid.0-9/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer.java Tue Jan 30 08:40:41 2007
@@ -579,7 +579,7 @@
 
         	Content data = new Content(Content.TypeEnum.INLINE_T, payload);
 
-        	doMessageTransfer(messageHeaders,destination,data,message,deliveryMode,priority,timeToLive,immediate);
+        	doMessageTransfer(messageHeaders,destination,data,message,deliveryMode,priority,timeToLive,mandatory,immediate);
         } else {
         	// Reference message case
             // Sequence is as follows 
@@ -599,7 +599,7 @@
         	
         	// Message.Transfer
         	Content data = new Content(Content.TypeEnum.REF_T, referenceId.getBytes()); 
-        	doMessageTransfer(messageHeaders,destination,data,message,deliveryMode,priority,timeToLive,immediate);
+        	doMessageTransfer(messageHeaders,destination,data,message,deliveryMode,priority,timeToLive,mandatory,immediate);
         	
         	//Message.Append
         	for(Iterator it = content.iterator(); it.hasNext();){
@@ -622,7 +622,7 @@
     }
     
     private void doMessageTransfer(MessageHeaders messageHeaders,AMQDestination destination, Content content, AbstractJMSMessage message, int deliveryMode, int priority,
-            long timeToLive, boolean immediate)throws JMSException{
+            long timeToLive, boolean mandatory, boolean immediate)throws JMSException{
     	try
         {
             AMQMethodBody methodBody = MessageTransferBody.createMethodBody(
@@ -638,6 +638,7 @@
                 destination.getExchangeName(),  // String exchange
                 messageHeaders.getExpiration(), // long expiration
                 immediate,                      // boolean immediate
+                mandatory,                      // boolean mandatory
                 messageHeaders.getMessageId(),  // String messageId
                 (short)priority,                // short priority
                 message.getJMSRedelivered(),    // boolean redelivered

Modified: incubator/qpid/branches/qpid.0-9/java/common/protocol-version.xml
URL: http://svn.apache.org/viewvc/incubator/qpid/branches/qpid.0-9/java/common/protocol-version.xml?view=diff&rev=501456&r1=501455&r2=501456
==============================================================================
--- incubator/qpid/branches/qpid.0-9/java/common/protocol-version.xml (original)
+++ incubator/qpid/branches/qpid.0-9/java/common/protocol-version.xml Tue Jan 30 08:40:41 2007
@@ -27,8 +27,8 @@
   <property name="generated.dir" location="${generated.path}/${generated.package}" />
   <property name="generated.timestamp" location="${generated.dir}/timestamp" />
   <property name="xml.spec.dir" location="${topDirectoryLocation}/../specs" />
-  <property name="xml.spec.deps" value="amqp.0-9.xml cluster.0-9.xml exchange-bound.0-9.xml amqp-nogen.0-9.xml" />
-  <property name="xml.spec.list" value="${xml.spec.dir}/amqp.0-9.xml ${xml.spec.dir}/cluster.0-9.xml ${xml.spec.dir}/exchange-bound.0-9.xml ${xml.spec.dir}/amqp-nogen.0-9.xml" />
+  <property name="xml.spec.deps" value="amqp.0-9.xml amqp-errata.0-9.xml cluster.0-9.xml exchange-bound.0-9.xml amqp-nogen.0-9.xml" />
+  <property name="xml.spec.list" value="${xml.spec.dir}/amqp.0-9.xml ${xml.spec.dir}/amqp-errata.0-9.xml ${xml.spec.dir}/cluster.0-9.xml ${xml.spec.dir}/exchange-bound.0-9.xml ${xml.spec.dir}/amqp-nogen.0-9.xml" />
   	
   <target name="generate" depends="compile_generator,check_generate_deps" unless="generation.notRequired">
     <mkdir dir="${generated.dir}"/>

Modified: incubator/qpid/branches/qpid.0-9/java/common/src/main/java/org/apache/qpid/framing/RequestManager.java
URL: http://svn.apache.org/viewvc/incubator/qpid/branches/qpid.0-9/java/common/src/main/java/org/apache/qpid/framing/RequestManager.java?view=diff&rev=501456&r1=501455&r2=501456
==============================================================================
--- incubator/qpid/branches/qpid.0-9/java/common/src/main/java/org/apache/qpid/framing/RequestManager.java (original)
+++ incubator/qpid/branches/qpid.0-9/java/common/src/main/java/org/apache/qpid/framing/RequestManager.java Tue Jan 30 08:40:41 2007
@@ -75,8 +75,6 @@
         long requestId = getNextRequestId(); // Get new request ID
         AMQFrame requestFrame = AMQRequestBody.createAMQFrame(channel, requestId,
             lastProcessedResponseId, requestMethodBody);
-        requestSentMap.put(requestId, methodListener);
-        protocolWriter.writeFrame(requestFrame);
         if (logger.isDebugEnabled())
         {
             logger.debug((serverFlag ? "SRV[" : "CLI[") + connectionId + "," + channel +
@@ -84,6 +82,8 @@
         }
         //System.out.println((serverFlag ? "SRV[" : "CLI[") + connectionId + "," + channel +
         //        "] TX REQ: Req[" + requestId + " " + lastProcessedResponseId + "]; " + requestMethodBody);
+        requestSentMap.put(requestId, methodListener);
+        protocolWriter.writeFrame(requestFrame);
         return requestId;
     }
 

Added: incubator/qpid/branches/qpid.0-9/specs/amqp-errata.0-9.xml
URL: http://svn.apache.org/viewvc/incubator/qpid/branches/qpid.0-9/specs/amqp-errata.0-9.xml?view=auto&rev=501456
==============================================================================
--- incubator/qpid/branches/qpid.0-9/specs/amqp-errata.0-9.xml (added)
+++ incubator/qpid/branches/qpid.0-9/specs/amqp-errata.0-9.xml Tue Jan 30 08:40:41 2007
@@ -0,0 +1,44 @@
+<?xml version = "1.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.
+ -
+ -->
+<amqp major = "0" minor = "9" port = "5672" comment = "AMQ Protocol version 0-9">
+
+  <constant name = "frame-request"    value = "9" />
+  <constant name = "frame-response"   value = "10" />
+
+  <!-- ==  MESSAGE  =========================================================== -->
+
+  <class name = "message" index = "120">
+    <method name = "transfer" index = "10">
+      <chassis name = "server" implement = "MUST" />
+      <chassis name = "client" implement = "MUST" />
+      <field name = "mandatory" type = "bit" label = "indicate mandatory routing">
+        <doc>
+          This flag tells the server how to react if the message cannot be
+          routed to a queue.  If this flag is set, the server will return an
+          unroutable message with a Return method.  If this flag is zero, the
+          server silently drops the message.
+        </doc>
+      </field>   
+    </method>
+  </class>
+
+</amqp>

Modified: incubator/qpid/branches/qpid.0-9/specs/amqp.0-9.xml
URL: http://svn.apache.org/viewvc/incubator/qpid/branches/qpid.0-9/specs/amqp.0-9.xml?view=diff&rev=501456&r1=501455&r2=501456
==============================================================================
--- incubator/qpid/branches/qpid.0-9/specs/amqp.0-9.xml (original)
+++ incubator/qpid/branches/qpid.0-9/specs/amqp.0-9.xml Tue Jan 30 08:40:41 2007
@@ -169,8 +169,6 @@
   <constant name = "frame-oob-body"   value = "6" />
   <constant name = "frame-trace"      value = "7" />
   <constant name = "frame-heartbeat"  value = "8" />
-  <constant name = "frame-request"    value = "9" />
-  <constant name = "frame-response"   value = "10" />
 
   <!-- Protocol constants -->
   <constant name = "frame-min-size"   value = "4096" />