You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@qpid.apache.org by ac...@apache.org on 2008/03/11 23:13:12 UTC

svn commit: r636126 - in /incubator/qpid/trunk/qpid/cpp: rubygen/0-10/ src/ src/qpid/ src/qpid/amqp_0_10/ src/qpid/framing/ src/qpid/sys/ src/qpid/sys/posix/ src/tests/

Author: aconway
Date: Tue Mar 11 15:13:10 2008
New Revision: 636126

URL: http://svn.apache.org/viewvc?rev=636126&view=rev
Log:
rubygen/0-10/specification.rb:
 - Simplified enum mapping/encoding.
 - struct encoding
 - ostream << operators
  
src/qpid/Serializer.h
 - free funciton serialization
 - separate Encoder/Decoder for const correctness
 - split() to allow separate encode/decode for complex cases.

src/qpid/amqp_0_10/Assembly.cpp, Assembly.h: AMQP 0-10 final Assembly

src/qpid/amqp_0_10/Codec.h
 - Replaced enable_if with overloads, simpler & more flexible.

src/qpid/amqp_0_10/Frame.cpp, .h: AMQP 0-10 final frame.

src/qpid/amqp_0_10/Holder.h:
 - provide const and non-const apply 

src/qpid/amqp_0_10/Segment.cpp, .h: AMQP 0-10 final Segment.

src/qpid/amqp_0_10/apply.h
 - ConstApplyFunctor for const apply.

src/qpid/amqp_0_10/built_in_types.h
 - SerializableString encoding
 
src/qpid/amqp_0_10/complex_types.cpp, .h
 - const application
 - Action base class for command/control.
 
src/qpid/framing/AMQBody.h
 - removed 0-10 final changes, moving integration point down the stack.
 
src/qpid/sys/Shlib.h
 - removed unused and uncompilable (on some compilers) function.
 
src/qpid/sys/Time.h, .cpp
 - ostream << for AbsTime and Duration.
 
src/tests/Assembly.cpp, Segment.cpp, apply.cpp, serialize.cpp: testing new code.

Added:
    incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.cpp   (with props)
    incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.h   (with props)
    incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.cpp   (with props)
    incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.h   (with props)
    incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.cpp   (with props)
    incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.h   (with props)
Modified:
    incubator/qpid/trunk/qpid/cpp/rubygen/0-10/specification.rb
    incubator/qpid/trunk/qpid/cpp/src/Makefile.am
    incubator/qpid/trunk/qpid/cpp/src/qpid/Serializer.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Codec.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Holder.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/apply.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/built_in_types.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/complex_types.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/complex_types.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/sys/Shlib.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/sys/Time.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/sys/posix/Time.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/apply.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/serialize.cpp

Modified: incubator/qpid/trunk/qpid/cpp/rubygen/0-10/specification.rb
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/rubygen/0-10/specification.rb?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/rubygen/0-10/specification.rb (original)
+++ incubator/qpid/trunk/qpid/cpp/rubygen/0-10/specification.rb Tue Mar 11 15:13:10 2008
@@ -15,11 +15,13 @@
     genl
     typename=d.name.typename
     if d.enum
-      scope("enum #{typename}Enum {", "};") { 
+      scope("enum #{typename} {", "};") { 
         genl d.enum.choices.map { |c|
           "#{c.name.constname} = #{c.value}" }.join(",\n")
       }
-      genl "typedef Enum<#{typename}Enum, uint8_t> #{typename};"
+      scope("inline SerializableEnum<#{typename}, uint8_t> serializable(#{typename}& e) {") {
+        genl "return SerializableEnum<#{typename}>(e);"
+      }
     else
       genl "typedef #{d.amqp2cpp} #{typename};"
     end
@@ -52,11 +54,20 @@
       ctor_decl(x.classname, x.parameters) unless x.fields.empty?
       genl "void accept(Visitor&);" 
       genl "void accept(ConstVisitor&) const;"
+      if (x.fields.empty?)
+        genl "template <class S> void serialize(S&) {}"
+      else
+        scope("template <class S> void serialize(S& s) {") {
+          gen "s"; x.fields.each { |f| gen "(#{f.cppname})"}; genl ";"
+        }
+      end
       genl
       yield if block
     }
+    genl "std::ostream& operator << (std::ostream&, const #{x.classname}&);"
   end
 
+  # FIXME aconway 2008-03-10: packing, coding
   def action_struct_cpp(x)
     genl
     genl "const char* #{x.classname}::NAME=\"#{x.fqname}\";"
@@ -73,6 +84,12 @@
       genl "void #{x.classname}::accept(Visitor& v) {  v.visit(*this); }"
       genl "void #{x.classname}::accept(ConstVisitor& v) const { v.visit(*this); }"
     end
+    genl
+    scope("std::ostream& operator << (std::ostream& o, const #{x.classname}&#{"x" unless x.fields.empty?}) {") {
+      genl "return o << \"[#{x.fqname}\";";
+      x.fields.each{ |f| genl "o << \" #{f.name}=\" << x.#{f.cppname};" }
+      genl "o << \"];\";"
+    }
   end
 
   # structs
@@ -87,13 +104,6 @@
       function_defn("template <class T> void invoke", ["T& target"]) {
         genl "target.#{a.funcname}(#{a.values.join(', ')});"
       }
-      if (a.fields.empty?)
-        genl "template <class S> void serialize(S&) {}"
-      else
-        scope("template <class S> void serialize(S& s) {") {
-          gen "s"; a.fields.each { |f| gen "(#{f.cppname})"}; genl ";"
-        }
-      end
     }
   end
   
@@ -108,6 +118,7 @@
       include "#{@dir}/built_in_types"
       include "#{@dir}/complex_types"
       include "<boost/call_traits.hpp>"
+      include "<iosfwd>"
       genl "using boost::call_traits;"
       namespace(@ns) {
         # Top level 
@@ -133,6 +144,7 @@
 
     cpp_file("#{@dir}/specification") { 
       include "#{@dir}/specification"
+      include "<iostream>"
       # FIXME aconway 2008-03-04: add Struct visitors.
       ["Command","Control"].each { |x| include "#{@dir}/Apply#{x}" }
       namespace(@ns) { 

Modified: incubator/qpid/trunk/qpid/cpp/src/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/Makefile.am?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/Makefile.am (original)
+++ incubator/qpid/trunk/qpid/cpp/src/Makefile.am Tue Mar 11 15:13:10 2008
@@ -109,6 +109,10 @@
   qpid/amqp_0_10/Holder.h \
   qpid/amqp_0_10/Codec.h \
   qpid/amqp_0_10/Decimal.h \
+  qpid/amqp_0_10/Segment.h \
+  qpid/amqp_0_10/Segment.cpp \
+  qpid/amqp_0_10/Assembly.h \
+  qpid/amqp_0_10/Assembly.cpp \
   qpid/Serializer.h \
   qpid/framing/AccumulatedAck.cpp \
   qpid/framing/AMQBody.cpp \

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/Serializer.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/Serializer.h?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/Serializer.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/Serializer.h Tue Mar 11 15:13:10 2008
@@ -23,59 +23,71 @@
  */
 
 #include <boost/utility/enable_if.hpp>
-#include <boost/static_assert.hpp>
-#include <boost/type_traits/is_class.hpp>
-#include <algorithm>
+#include <boost/type_traits/is_base_and_derived.hpp>
 
 namespace qpid {
+namespace serialize { 
 
-// FIXME aconway 2008-03-03: Doc - esp decoding
-template <class Derived> class Serializer {
-  public:
-    typedef Serializer result_type; // unary functor requirement.
+// FIXME aconway 2008-03-03: Document.
+// Encoder/Decoder concept: add op() for primitive types, raw(),
+// op()(Iter, Iter). Note split, encode, decode.
+// 
 
-    static const bool IS_DECODER=false;
+// FIXME aconway 2008-03-09: document - non-intrusive serialzation.
+// Default rule calls member. Enums must provide an override rule.
 
-    /** Generic handler for class objects, call serialize() */
-    template <class T>
-    typename boost::enable_if<boost::is_class<T>, Derived&>::type
-    operator()(T& t) {
-        t.serialize(self());
-        return self();
-    }
+/** Overload for types that do not provide a serialize() member.*/
+template <class T> T& serializable(T& t) { return t; }
+
+template <class Derived> class Encoder {
+  public:
+    typedef Derived& result_type; // unary functor requirement.
 
-    /** Generic handler for const class objects, call serialize() */
+    /** Default op() calls serializable() free function */
     template <class T>
-    typename boost::enable_if<boost::is_class<T>, Derived&>::type
-    operator()(const T& t) {
-        assert(!Derived::IS_DECODER); // We won't modify the value.
-        // const_cast so we don't need 2 serialize() members for every class.
-        const_cast<T&>(t).serialize(self());
-        return self();
+    Derived& operator()(const T& t) {
+        serializable(const_cast<T&>(t)).serialize(self()); return self();
     }
 
-    template <class T, bool=false> struct Split {
-        Split(Derived& s, T& t) { t.encode(s); }
-    };
-    
-    template <class T> struct Split<T,true> {
-        Split(Derived& s, T& t) { t.decode(s); }
-    };
-    /**
-     * Called by classes that want to receive separate
-     * encode()/decode() calls.
-     */
+    /** Split serialize() into encode()/decode() */
     template <class T>
-    void split(T& t) { Split<T, Derived::IS_DECODER>(self(),t); }
-
+    Derived& split(const T& t) { t.encode(self()); return self(); }
+    
   private:
     Derived& self() { return *static_cast<Derived*>(this); }
 };
 
+template <class Derived> class Decoder {
+  public:
+    typedef Derived& result_type; // unary functor requirement.
+
+    /** Default op() calls serializable() free function */
+    template <class T>
+    Derived& operator()(T& t) {
+        serializable(t).serialize(self()); return self();
+    }
 
+    /** Split serialize() into encode()/decode() */
+    template <class T>
+    Derived& split(T& t) { t.decode(self()); return self(); }
+    
 
+  private:
+    Derived& self() { return *static_cast<Derived*>(this); }
+};
 
+/** Serialize a type by converting it to/from another type */
+template <class Type, class AsType>
+struct SerializeAs {
+    Type& value;
+    SerializeAs(Type & t) : value(t) {}
+    template <class S> void serialize(S& s) { s.split(*this); }
+    template <class S> void encode(S& s) const { s(AsType(value)); }
+    template <class S> void decode(S& s) { AsType x; s(x); value=x; }
+};
 
-} // namespace qpid
+}} // namespace qpid::serialize
 
+// FIXME aconway 2008-03-09: rename to serialize.h
+// 
 #endif  /*!QPID_SERIALIZER_H*/

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.cpp?rev=636126&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.cpp (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.cpp Tue Mar 11 15:13:10 2008
@@ -0,0 +1,82 @@
+/*
+ *
+ * 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.
+ *
+ */
+#include "Assembly.h"
+#include "qpid/Exception.h"
+#include "Codec.h"
+#include "qpid/framing/Blob.h"
+
+namespace qpid {
+namespace amqp_0_10 {
+
+using framing::in_place;
+
+Assembly::Assembly(const Command& c) : command(in_place<Command::Holder>(c)) {}
+Assembly::Assembly(const Control& c) : control(in_place<Control::Holder>(c)) {}
+
+void Assembly::add(const Frame& f) {
+    switch (f.getType()) {
+      case COMMAND: {
+          Segment& s = segments[ACTION_SEG];
+          s.add(f);
+          if (s.isComplete()) {
+              command = in_place<Command::Holder>();
+              Codec::decode(s.begin())(*command);
+              if (f.testFlags(Frame::LAST_SEGMENT)) {
+                  segments[HEADER_SEG].setMissing();
+                  segments[BODY_SEG].setMissing();
+              }
+          }
+          break;
+      }
+      case CONTROL: {
+          Segment& s = segments[ACTION_SEG];
+          s.add(f);
+          if (s.isComplete()) {
+              control = in_place<Control::Holder>();
+              Codec::decode(s.begin())(*control);
+              if (f.testFlags(Frame::LAST_SEGMENT)) {
+                  segments[HEADER_SEG].setMissing();
+                  segments[BODY_SEG].setMissing();
+              }
+          }
+          break;
+      }
+      case HEADER: {
+          Segment& s = segments[HEADER_SEG];
+          s.add(f);
+          if (s.isComplete()) {
+              header = in_place<Header>();
+              Codec::decode(*header);
+              if (f.testFlags(Frame::LAST_SEGMENT)) {
+                  segments[BODY_SEG].setMissing();
+              }
+          }
+          break;
+      }
+      case BODY: {
+          Segment& s = segments[BODY_SEG];
+          s.add(f);
+          break;
+      }
+    }
+}
+
+}} // namespace qpid::amqp_0_10

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.cpp
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.cpp
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.h?rev=636126&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.h (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.h Tue Mar 11 15:13:10 2008
@@ -0,0 +1,71 @@
+#ifndef QPID_AMQP_0_10_ASSEMBLY_H
+#define QPID_AMQP_0_10_ASSEMBLY_H
+
+/*
+ *
+ * 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.
+ *
+ */
+
+#include "qpid/amqp_0_10/CommandHolder.h"
+#include "qpid/amqp_0_10/ControlHolder.h"
+#include "Segment.h"
+#include <boost/optional.hpp>
+
+namespace qpid {
+namespace amqp_0_10 {
+
+// FIXME aconway 2008-03-06: TODO
+struct Header {
+    template <class S> void serialize(S&) {}
+};
+
+class Assembly
+{
+  public:
+    enum SegmentIndex { ACTION_SEG, HEADER_SEG, BODY_SEG };
+    
+    Assembly() {}
+    Assembly(const Command& c);
+    Assembly(const Control& c);
+
+    Segment& getSegment(int i) { return segments[i]; }
+    const Segment& getSegment(int i) const { return segments[i]; }
+    
+    const Command* getCommand() const { return command ? command->get() : 0; }
+    const Control* getControl() const { return control ? control->get() : 0; }
+    const Header* getHeader() const { return header.get_ptr(); }
+
+    void setCommand(const Command& c) { *command = c; }
+    void setControl(const Control& c) { *control = c; }
+    void setHeader(const Header& h) { header = h; }
+    
+    void add(const Frame& f);
+
+    bool isComplete() const { return segments[BODY_SEG].isComplete(); }
+    
+  private:
+    Segment segments[3];
+    boost::optional<Command::Holder> command;
+    boost::optional<Control::Holder> control;
+    boost::optional<Header> header;
+};
+
+}} // namespace qpid::amqp_0_10
+
+#endif  /*!QPID_AMQP_0_10_ASSEMBLY_H*/

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.h
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Assembly.h
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Codec.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Codec.h?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Codec.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Codec.h Tue Mar 11 15:13:10 2008
@@ -28,34 +28,55 @@
 #include <boost/type_traits/is_float.hpp>
 #include <boost/type_traits/is_arithmetic.hpp>
 #include <boost/detail/endian.hpp>
+#include <boost/static_assert.hpp>
 
 namespace qpid {
 namespace amqp_0_10 {
+
+#ifdef BOOST_LITTLE_ENDIAN
+template <class T> void endianize(T& t) {
+    char*p =reinterpret_cast<char*>(&t);
+    std::reverse(p, p+sizeof(T));
+}
+#else
+template <class T> void endianize(T&) {}
+#endif
+
 /**
  * AMQP 0-10 encoding and decoding.
  */
-struct Codec {                  // FIXME aconway 2008-02-29: drop this wrapper?
+struct Codec {
+
+    // FIXME aconway 2008-02-29: drop this wrapper, rename to
+    // IteratorEncoder, IteratorDecoder?
 
     /** Encode to an output byte iterator */
     template <class OutIter>
-    class Encode : public Serializer<Encode<OutIter> > {
+    class Encode : public serialize::Encoder<Encode<OutIter> >
+    {
       public:
         Encode(OutIter o) : out(o) {}
 
-        using Serializer<Encode<OutIter> >::operator();
+        using serialize::Encoder<Encode<OutIter> >::operator();
 
-        template <class T>
-        typename boost::enable_if<boost::is_integral<T>, Encode&>::type
-        operator()(T x) {
-            endianize(x);
-            raw(&x, sizeof(x));
-            return *this;
-        }
+        // FIXME aconway 2008-03-10:  wrong encoding, need packing support
+        Encode& operator()(bool x) { *out++=x; return *this;} 
+
+        Encode& operator()(char x) { *out++=x; return *this; }
+        Encode& operator()(int8_t x) { *out++=x; return *this; }
+        Encode& operator()(uint8_t x) { *out++=x; return *this; }
+
+        Encode& operator()(int16_t x) { return endian(x); }
+        Encode& operator()(int32_t x) { return endian(x); }
+        Encode& operator()(int64_t x) { return endian(x); }
+
+        Encode& operator()(uint16_t x) { return endian(x); }
+        Encode& operator()(uint32_t x) { return endian(x); }
+        Encode& operator()(uint64_t x) { return endian(x); }
+
+        Encode& operator()(float x) { return endian(x); }
+        Encode& operator()(double x) { return endian(x); }
 
-        // FIXME aconway 2008-02-20: correct float encoading?
-        template <class T>
-        typename boost::enable_if<boost::is_float<T>, Encode&>::type
-        operator()(const T& x) { raw(&x, sizeof(x)); return *this; }
 
         template <class Iter> Encode& operator()(Iter begin, Iter end) {
             std::for_each(begin, end, *this);
@@ -63,42 +84,45 @@
         }
 
         void raw(const void* p, size_t n) {
-            std::copy((const char*)p, (const char*)p+n, out); 
+            std::copy((const char*)p, (const char*)p+n, out);
+            out += n;
         }
-        
+
+        OutIter pos() const { return out; }
+
       private:
+
+        template <class T> Encode& endian(T x) {
+            endianize(x); raw(&x, sizeof(x)); return *this;
+        }
+
         OutIter out;
     };
 
     template <class InIter>
-    class Decode : public Serializer<Decode<InIter> > {
+    class Decode : public serialize::Decoder<Decode<InIter> > {
       public:
         Decode(InIter i) : in(i) {}
 
-        static const bool IS_DECODER=true;
-        
-        using Serializer<Decode<InIter> >::operator();
+        using serialize::Decoder<Decode<InIter> >::operator();
         
-        template <class T>
-        typename boost::enable_if<boost::is_integral<T>, Decode&>::type
-        operator()(T& x) {
-            raw(&x, sizeof(x));
-            endianize(x);
-            return *this;
-        }
+        // FIXME aconway 2008-03-10:  wrong encoding, need packing support
+        Decode& operator()(bool& x) { x=*in++; return *this; }
 
-        template <class T>
-        typename boost::enable_if<boost::is_float<T>, Decode&>::type
-        operator()(T& x) { raw(&x, sizeof(x)); return *this; }
-
-        template<class T, class SizeType>
-        Decode& operator()(SerializableString<T,SizeType>& str) {
-            SizeType n;
-            (*this)(n);
-            str.resize(n);
-            std::for_each(str.begin(), str.end(), *this);
-            return *this;
-        }
+        Decode& operator()(char& x) { x=*in++; return *this; }
+        Decode& operator()(int8_t& x) { x=*in++; return *this; }
+        Decode& operator()(uint8_t& x) { x=*in++; return *this; }
+
+        Decode& operator()(int16_t& x) { return endian(x); }
+        Decode& operator()(int32_t& x) { return endian(x); }
+        Decode& operator()(int64_t& x) { return endian(x); }
+
+        Decode& operator()(uint16_t& x) { return endian(x); }
+        Decode& operator()(uint32_t& x) { return endian(x); }
+        Decode& operator()(uint64_t& x) { return endian(x); }
+
+        Decode& operator()(float& x) { return endian(x); }
+        Decode& operator()(double& x) { return endian(x); }
 
         template <class Iter> Decode& operator()(Iter begin, Iter end) {
             std::for_each(begin, end, *this);
@@ -106,34 +130,47 @@
         }
 
         void raw(void *p, size_t n) {
-            // FIXME aconway 2008-02-29: requires random access iterator,
-            // does this optimize to memcpy? Is there a better way?
             std::copy(in, in+n, (char*)p);
             in += n;
         }
 
+        InIter pos() const { return in; }
+
       private:
+
+        template <class T> Decode& endian(T& x) {
+            raw(&x, sizeof(x)); endianize(x); return *this;
+        }
+
         InIter in;
     };
 
     
-    class Size : public Serializer<Size> {
+    class Size : public serialize::Encoder<Size> {
       public:
         Size() : size(0) {}
 
         operator size_t() const { return size; }
 
-        using Serializer<Size>::operator();
+        using serialize::Encoder<Size>::operator();
+
+        // FIXME aconway 2008-03-10:  wrong encoding, need packing support
+        Size& operator()(bool x)  { size += sizeof(x); return *this; }
         
-        template <class T>
-        typename boost::enable_if<boost::is_arithmetic<T>, Size&>::type
-        operator()(const T&) { size += sizeof(T); return *this; }
-
-        template<class T, class SizeType>
-        Size& operator()(const SerializableString<T,SizeType>& str) {
-            size += sizeof(SizeType) + str.size()*sizeof(T);
-            return *this;
-        }
+        Size& operator()(char x)  { size += sizeof(x); return *this; }
+        Size& operator()(int8_t x)  { size += sizeof(x); return *this; }
+        Size& operator()(uint8_t x)  { size += sizeof(x); return *this; }
+
+        Size& operator()(int16_t x)  { size += sizeof(x); return *this; }
+        Size& operator()(int32_t x)  { size += sizeof(x); return *this; }
+        Size& operator()(int64_t x)  { size += sizeof(x); return *this; }
+
+        Size& operator()(uint16_t x)  { size += sizeof(x); return *this; }
+        Size& operator()(uint32_t x)  { size += sizeof(x); return *this; }
+        Size& operator()(uint64_t x)  { size += sizeof(x); return *this; }
+
+        Size& operator()(float x)  { size += sizeof(x); return *this; }
+        Size& operator()(double x)  { size += sizeof(x); return *this; }
 
         template <class Iter>
         Size& operator()(const Iter& a, const Iter& b) {
@@ -147,27 +184,16 @@
         size_t size;
     };
 
-    template <class OutIter> static Decode<OutIter> decode(const OutIter &i) {
-        return Decode<OutIter>(i);
+    // FIXME aconway 2008-03-11: rename to encoder(), decoder()
+    template <class InIter> static Decode<InIter> decode(const InIter &i) {
+        return Decode<InIter>(i);
     }
 
-    template <class InIter> static Encode<InIter> encode(InIter i) {
-        return Encode<InIter>(i);
+    template <class OutIter> static Encode<OutIter> encode(OutIter i) {
+        return Encode<OutIter>(i);
     }
 
     template <class T> static size_t size(const T& x) { return Size()(x); }
-
-  private:
-    template <class T> static inline void endianize(T& value) {
-#ifdef BOOST_LITTLE_ENDIAN
-        std::reverse((char*)&value, (char*)&value+sizeof(value));
-#else
-        (void)value;            // Avoid unused var warnings.
-#endif
-    }
-    static inline void endianize(char&) {}
-    static inline void endianize(uint8_t&) {}
-    static inline void endianize(int8_t&) {}
 };
 
 }} // namespace qpid::amqp_0_10

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.cpp?rev=636126&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.cpp (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.cpp Tue Mar 11 15:13:10 2008
@@ -0,0 +1,28 @@
+/*
+ *
+ * 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.
+ *
+ */
+#include "Frame.h"
+
+namespace qpid {
+namespace amqp_0_10 {
+
+bool Frame::match(const Frame& x) {
+}
+}} // namespace qpid::amqp_0_10

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.cpp
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.cpp
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.h?rev=636126&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.h (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.h Tue Mar 11 15:13:10 2008
@@ -0,0 +1,171 @@
+#ifndef QPID_AMQP_0_10_FRAME_H
+#define QPID_AMQP_0_10_FRAME_H
+
+/*
+ *
+ * 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.
+ *
+ */
+
+#include "qpid/amqp_0_10/built_in_types.h"
+#include <boost/shared_array.hpp>
+
+namespace qpid {
+namespace amqp_0_10 {
+
+class FrameHeader {
+  public:
+    enum Flags { FIRST_SEGMENT=8, LAST_SEGMENT=4, FIRST_FRAME=2, LAST_FRAME=1 };
+    enum ShortFlags { FS=8, LS=4, FF=2, LF=1 };
+
+
+    FrameHeader(uint8_t flags_=0, SegmentType type_=SegmentType(), uint8_t track_=0, uint16_t channel_=0)
+        : flags(flags_), type(type_), size_(), track(track_), channel(channel_)
+    {}
+
+    uint8_t getFlags() const { return flags; }
+    SegmentType getType() const { return type; }
+    uint16_t getSize() const { return size_; }
+    uint8_t getTrack() const { return track; }
+    uint16_t getChannel() const { return channel; }
+
+    void setFlags(uint8_t flags_) { flags=flags_; }
+    void setType(SegmentType type_)  { type=type_; }
+    void setTrack(uint8_t track_) { track=track_; }
+    void setChannel(uint8_t channel_) { channel=channel_; }
+
+    Flags testFlags(uint8_t f) const { return Flags(flags & f); }
+    void raiseFlags(uint8_t f) { flags |= f; }
+    void clearFlags(uint8_t f) { flags &= ~f; }
+
+    uint16_t size() const { return size_; }
+    bool empty() const { return size()==0; }
+
+    bool operator==(const FrameHeader& x)
+    { return memcmp(this, &x, sizeof(this)); }
+
+    template <class S> void serialize(S& s);
+
+  protected:
+    uint8_t flags;
+    SegmentType type;
+    uint16_t size_;
+    uint8_t track;
+    uint16_t channel;
+
+  private:
+};
+
+class Frame : public FrameHeader {
+  public:
+    Frame(uint8_t flags_=0, SegmentType type_=SegmentType(), uint8_t track_=0,
+          uint16_t channel_=0)
+        : FrameHeader(flags_, type_, track_, channel_), ref() {}
+
+    Frame(const FrameHeader& header) : FrameHeader(header), ref() {}
+
+    Frame(const FrameHeader& header, size_t s)
+        : FrameHeader(header), ref() { resize(s); }
+
+    Frame(const FrameHeader& header, const char* a, const char* b,
+          bool copyFlag=true) : FrameHeader(header), ref() {
+        if (copyFlag) copy(a,b); else refer(a,b);
+    }
+    
+    Frame(const FrameHeader& header, const char* p, std::size_t s,
+          bool copyFlag=true) : FrameHeader(header), ref() {
+        if (copyFlag) copy(p,p+s); else refer(p,p+s);
+    }
+    
+
+    /** Allocate a buffer of at least size bytes */
+    void resize(uint16_t size);
+
+    /** Make the frame refer to byte range [begin,end)
+     * Range is NOT COPIED, it must be valid for lifetime of Frame.
+     */
+    void refer(const char* begin, const char* end);
+
+    /** Allocate a buffer and copy range begin/end */
+    void copy(const char* begin, const char* end);
+
+    char* begin() { assert(!ref); return data.get(); }
+    const char* begin() const { return ref ? ref : data.get(); }
+    char* end() { return begin() + size(); }
+    const char* end() const { return begin() + size(); }
+
+    void clear() { data.reset();  ref = 0; size_= 0; }
+    
+    template <class S> void serialize(S& s);
+    template <class S> void encode(S& s) const;
+    template <class S> void decode(S& s);
+
+  private:
+    boost::shared_array<char> data;
+    const char* ref;
+};
+
+inline void Frame::refer(const char* a, const char* b) {
+    data.reset();
+    ref = a;
+    size_ = b-a;
+}
+
+inline void Frame::copy(const char* a, const char* b) {
+    resize(b-a);
+    std::copy(a, b, begin());
+}
+
+inline void Frame::resize(uint16_t s) {
+    if (s > size() || ref) {
+        ref = 0;
+        data.reset(new char[s]);
+    }
+    size_=s;
+}
+
+template <class S> void FrameHeader::serialize(S& s) {
+    uint8_t pad8=0;
+    uint32_t pad32=0;
+    s(flags)(type)(size_)(pad8)(track)(channel)(pad32);
+}
+
+template <class S> void Frame::serialize(S& s) { s.split(*this); }
+
+template <class S> void Frame::encode(S& s) const {
+    s(static_cast<const FrameHeader&>(*this));
+    s.raw(begin(), size());
+}
+
+template <class S> void Frame::decode(S& s) {
+    try {
+        uint16_t oldSize = size_;
+        s(static_cast<FrameHeader&>(*this));
+        std::swap(oldSize, size_);
+        resize(oldSize);
+        s.raw(begin(), size());
+    } catch (...) {
+        clear();
+        throw;
+    }
+}
+
+}} // namespace qpid::amqp_0_10
+
+#endif  /*!QPID_AMQP_0_10_FRAME_H*/
+

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.h
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Frame.h
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Holder.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Holder.h?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Holder.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Holder.h Tue Mar 11 15:13:10 2008
@@ -29,18 +29,29 @@
 
 template <class DerivedHolder, class BaseHeld, size_t Size>
 struct Holder : public framing::Blob<Size, BaseHeld> {
-
     typedef framing::Blob<Size, BaseHeld> Base;
     
+    struct Assign : public ApplyFunctor<void> {
+        Holder& holder;
+        Assign(Holder& x) : holder(x) {}
+        template <class T> void operator()(const T& rhs) { holder=rhs; }
+    };
+
     Holder() {}
+    Holder(const BaseHeld& x) { *this=x; }
     template <class T> Holder(const T& value) : Base(value) {}
 
     using Base::operator=;
-
+    Holder& operator=(const BaseHeld& rhs) {
+        Assign assign(*this);
+        apply(assign, rhs);
+        return *this;
+    }
+    
     uint8_t getCode() const { return this->get()->getCode(); }
     uint8_t getClassCode() const { return this->get()->getClassCode(); }
     
-    template <class S> void encode(S& s) {
+    template <class S> void encode(S& s) const {
         s(getClassCode())(getCode());
     }
 

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.cpp?rev=636126&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.cpp (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.cpp Tue Mar 11 15:13:10 2008
@@ -0,0 +1,81 @@
+/*
+ *
+ * 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.
+ *
+ */
+#include "Segment.h"
+#include "qpid/Exception.h"
+#include <boost/ref.hpp>
+#include <numeric>
+
+namespace qpid {
+namespace amqp_0_10 {
+
+Segment::Segment() : missing() {}
+
+bool Segment::isComplete() const {
+    return missing || !frames.empty() && (frames.back().testFlags(Frame::LAST_FRAME));
+}
+
+Segment::const_iterator Segment::begin() const {
+    return Segment::const_iterator(frames, frames.begin(), frames.begin()->begin());
+}
+
+Segment::const_iterator Segment::end() const {
+    return const_iterator(frames, frames.end(), frames.end()->end());
+}
+
+namespace {
+Frame::Flags segFlags(const Frame& f) {
+    return f.testFlags(Frame::FIRST_SEGMENT | Frame::LAST_SEGMENT);
+}
+} // namespace
+
+void Segment::add(const Frame& frame) {
+    // FIXME aconway 2008-03-07:  ex types & messages.
+    if (isComplete()) throw Exception("cannot add frame to complete segment");
+    if (!frames.empty()) {
+        if (frame.testFlags(Frame::FIRST_FRAME) ||
+            frame.getType() != frames.front().getType() ||
+            segFlags(frames.front()) != segFlags(frame))
+            throw Exception("invalid frame");
+    }
+    frames.push_back(frame);
+}
+
+bool Segment::isFirst() const {
+    return !frames.empty() && frames.front().testFlags(Frame::FIRST_SEGMENT);
+}
+
+bool Segment::isLast() const {
+    return !frames.empty() && frames.front().testFlags(Frame::LAST_SEGMENT);
+}
+
+namespace {
+size_t accumulate_size(size_t total, const Frame& f) {
+    return total+f.size();
+}
+}
+
+size_t Segment::size() const {
+    return std::accumulate(frames.begin(), frames.end(), 0, &accumulate_size);
+}
+
+}} // namespace qpid::amqp_0_10
+
+

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.cpp
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.cpp
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.h?rev=636126&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.h (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.h Tue Mar 11 15:13:10 2008
@@ -0,0 +1,127 @@
+#ifndef QPID_AMQP_0_10_SEGMENT_H
+#define QPID_AMQP_0_10_SEGMENT_H
+
+/*
+ *
+ * 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.
+ *
+ */
+
+#include "Frame.h"
+#include "qpid/InlineVector.h"
+#include "qpid/amqp_0_10/built_in_types.h"
+#include <boost/iterator/iterator_facade.hpp>
+#include <boost/range/iterator_range.hpp>
+#include <vector>
+
+namespace qpid {
+namespace amqp_0_10 {
+
+/**
+ * Represents a (possibly incomplete) header or body segment.
+ */
+class Segment {
+  public:
+    // TODO aconway 2008-03-07: optimize - use ISlist instead of vector.
+    typedef std::vector<Frame> Frames; 
+    struct const_iterator;
+
+    Segment();
+    
+    bool isComplete() const;
+    bool isFirst() const;
+    bool isLast() const;
+
+    const Frames& getFrames() const { return frames; }
+
+    void add(const Frame& frame);
+
+    /** Mark an empty segment as complete.
+     *@pre frames.empty().
+     */
+    void setMissing() { assert(frames.empty()); missing = true; }
+
+    size_t size() const; ///< Content size.
+
+    /** Iterate over content as a continuous range of bytes. */
+    const_iterator begin() const;
+    const_iterator end() const;
+
+    // TODO aconway 2008-03-07: Provide output iterator
+    // that extends the segment as basis for ostream (XML exchange)
+  private:
+    bool missing;
+    Frames frames;
+};
+
+class Segment::const_iterator : public boost::iterator_facade<
+    Segment::const_iterator, const char, boost::forward_traversal_tag>
+{
+  public:
+    const_iterator() : frames(), p() {}
+
+  private:
+#ifndef NDEBUG
+    void invariant() const {
+        assert(frames);
+        assert(frames->begin() <= i);
+        assert(i <= frames->end());
+        assert(i->begin() <= p);
+        assert(p <= i->end());
+    }
+    void valid() const {
+        invariant();
+        assert(p < i->end());
+    }
+#endif
+    const_iterator(const Frames& f, Frames::const_iterator pos, const char* ptr)
+        : frames(&f), i(pos), p(ptr) { skip_empty(); }
+
+    const char& dereference() const { valid(); return *p; }
+    bool equal(const const_iterator& x) const { return p == x.p; }
+    void increment() { valid(); ++p; skip_empty(); }
+    void advance(ptrdiff_t n) {
+        ptrdiff_t r = i->end() - p;
+        while (n > r) {
+            assert(i != frames->end());
+            n -= r;
+            p = (++i)->begin();
+            r = i->size();
+        }
+        p += n;
+        skip_empty();
+    }
+    void skip_empty() {
+        invariant();
+        while (p == i->end() && i != frames->end())
+            p = (++i)->begin();
+        invariant();
+    }
+
+    const Frames* frames;
+    Frames::const_iterator i;
+    const char* p;
+
+  friend class Segment;
+  friend class boost::iterator_core_access;
+};
+
+
+}} // namespace qpid::amqp_0_10
+
+#endif  /*!QPID_AMQP_0_10_SEGMENT_H*/

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.h
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/Segment.h
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/apply.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/apply.h?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/apply.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/apply.h Tue Mar 11 15:13:10 2008
@@ -79,11 +79,7 @@
     return visitor.getResult();
 }
 
-template <class R, bool Const=false> struct ApplyFunctor {
-    typedef R result_type;
-    static const bool IS_CONST=Const;
-};
-template <class R> struct ConstApplyFunctor : public ApplyFunctor<R, true> {};
+template <class R> struct ApplyFunctor { typedef R result_type; };
 
 }} // namespace qpid::amqp_0_10
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/built_in_types.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/built_in_types.h?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/built_in_types.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/built_in_types.h Tue Mar 11 15:13:10 2008
@@ -42,16 +42,16 @@
 typedef bool Bit;
 typedef bool Boolean;
 typedef char Char;
+typedef int8_t Int8;
 typedef int16_t Int16;
 typedef int32_t Int32;
 typedef int64_t Int64;
-typedef int8_t Int8;
+typedef uint8_t Bin8;
+typedef uint8_t Uint8;
 typedef uint16_t Uint16;
 typedef uint32_t CharUtf32 ;
 typedef uint32_t Uint32;
 typedef uint64_t Uint64;
-typedef uint8_t Bin8;
-typedef uint8_t Uint8;
 
 template <size_t N> struct Bin : public boost::array<char, N> {
     template <class S> void serialize(S& s) { s.raw(this->begin(), this->size()); }
@@ -81,9 +81,19 @@
 template <class T, class SizeType>
 struct SerializableString : public std::basic_string<T> {
     using std::basic_string<T>::operator=;
-    template <class S> void serialize(S& s) {
+
+    template <class S> void serialize(S& s) { s.split(*this); }
+
+    template <class S> void encode(S& s) const {
         s(SizeType(this->size()))(this->begin(), this->end());
     }
+
+    template <class S> void decode(S& s) {
+        SizeType newSize;
+        s(newSize);
+        this->resize(newSize);
+        s(this->begin(), this->end());
+    }
 };
 
 // TODO aconway 2008-02-29: separate ostream ops
@@ -114,21 +124,29 @@
 struct List  { template <class S> void serialize(S&) {} };
 struct Struct32  { template <class S> void serialize(S&) {} };
 
-// Serializable enum support
-template <class E, class Store> struct Enum {
-    Store value;
-    Enum() {}
-    Enum(E v) : value(v) {}
-    Enum(Store v) : value(v) {}
-    Enum& operator=(E v) { value=v; return *this; }
-    Enum& operator=(Store v) { value=v; return *this; }
-    operator E() const { return value; }
-    operator Store() const { return value; }
-    template <class S> void serialize(S& s) { s(value); }
+// FIXME aconway 2008-03-10: dummy ostream operators
+template <class T> std::ostream& operator<<(std::ostream& o, const Array<T>&) { return o; }
+inline std::ostream& operator<<(std::ostream& o, const ByteRanges&) { return o; }
+inline std::ostream& operator<<(std::ostream& o, const Map&) { return o; }
+inline std::ostream& operator<<(std::ostream& o, const SequenceSet&) { return o; }
+inline std::ostream& operator<<(std::ostream& o, const List&) { return o; }
+inline std::ostream& operator<<(std::ostream& o, const Struct32&) { return o; }
+
+/** Serialization helper for enums */
+template <class Enum, class Int=uint8_t>
+struct SerializableEnum {
+    Enum& value;
+    SerializableEnum(Enum & e) : value(e) {}
+    template <class S> void serialize(S& s) { s.split(*this); }
+    template <class S> void encode(S& s) const { s(Int(value)); }
+    template <class S> void decode(S& s) { Int i; s(i); value=Enum(i); }
 };
 
-enum SegmentTypeEnum { CONTROL, COMMAND, HEADER, BODY };
-typedef Enum<SegmentTypeEnum, uint8_t> SegmentType;
+enum SegmentType { CONTROL, COMMAND, HEADER, BODY };
+
+inline SerializableEnum<SegmentType> serializable(SegmentType& st) {
+    return SerializableEnum<SegmentType>(st);
+}
 
 }} // namespace qpid::amqp_0_10
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/complex_types.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/complex_types.cpp?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/complex_types.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/complex_types.cpp Tue Mar 11 15:13:10 2008
@@ -29,7 +29,7 @@
 // Functors for getting static values from a visitable base type.
 
 #define QPID_STATIC_VALUE_GETTER(NAME, TYPE, VALUE) \
-    struct NAME : public ConstApplyFunctor<TYPE> {  \
+    struct NAME : public ApplyFunctor<TYPE> {  \
         template <class T> TYPE operator()(const T&) const { return T::VALUE; }\
     }
 

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/complex_types.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/complex_types.h?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/complex_types.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/amqp_0_10/complex_types.h Tue Mar 11 15:13:10 2008
@@ -22,6 +22,8 @@
  *
  */
 
+#include "built_in_types.h"
+
 namespace qpid {
 namespace amqp_0_10 {
 
@@ -37,12 +39,31 @@
     virtual void accept(ConstVisitor&) const = 0;
 };
 
+struct Command;
+struct Control;
+
+struct Action {  // Marker for commands & controls
+    virtual ~Action() {}
+    virtual Command* getCommand() { return 0; }
+    virtual Control* getControl() { return 0; }
+
+    virtual const Command* getCommand() const {
+        return const_cast<Action*>(this)->getCommand();
+    }
+    virtual const Control* getControl() const {
+        return const_cast<Action*>(this)->getControl();
+    }
+};
+
 struct CommandVisitor;
 struct ConstCommandVisitor;
 struct CommandHolder;
 struct Command
-    : public Visitable<CommandVisitor, ConstCommandVisitor, CommandHolder>
+    : public Action,
+      public Visitable<CommandVisitor, ConstCommandVisitor, CommandHolder>
 {
+    using Action::getCommand;
+    Command* getCommand() { return this; }
     uint8_t getCode() const;
     uint8_t getClassCode() const;
     const char* getName() const;
@@ -53,8 +74,11 @@
 struct ConstControlVisitor;
 struct ControlHolder;
 struct Control
-    : public Visitable<ControlVisitor, ConstControlVisitor, ControlHolder>
+    : public Action,
+      public Visitable<ControlVisitor, ConstControlVisitor, ControlHolder>
 {
+    using Action::getControl;
+    Control* getControl() { return this; }
     uint8_t getCode() const;
     uint8_t getClassCode() const;
     const char* getName() const;
@@ -74,6 +98,10 @@
 };
 
 
+template <SegmentType E> struct ActionType;
+template <> struct ActionType<CONTROL> { typedef Control type; };
+template <> struct ActionType<COMMAND> { typedef Command type; };
+    
 }} // namespace qpid::amqp_0_10
 
 #endif  /*!QPID_AMQP_0_10_COMPLEX_TYPES_H*/

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/framing/AMQBody.h Tue Mar 11 15:13:10 2008
@@ -22,8 +22,6 @@
  *
  */
 #include "qpid/framing/amqp_types.h"
-#include "qpid/amqp_0_10/built_in_types.h"
-#include "qpid/amqp_0_10/complex_types.h"
 
 #include <ostream>
 
@@ -62,12 +60,6 @@
     virtual AMQMethodBody* getMethod() { return 0; }
     virtual const AMQMethodBody* getMethod() const { return 0; }
 
-    virtual amqp_0_10::Command* getCommand() { return 0; }
-    virtual const amqp_0_10::Command* getCommand() const { return 0; }
-
-    virtual amqp_0_10::Control* getControl() { return 0; }
-    virtual const amqp_0_10::Control* getControl() const { return 0; }
-
     /** Match if same type and same class/method ID for methods */
     static bool match(const AMQBody& , const AMQBody& );
 };
@@ -78,9 +70,7 @@
     METHOD_BODY = 1,
     HEADER_BODY = 2,
     CONTENT_BODY = 3,
-    HEARTBEAT_BODY = 8,
-    COMMAND_BODY = 100+amqp_0_10::COMMAND,
-    CONTROL_BODY = 100+amqp_0_10::CONTROL
+    HEARTBEAT_BODY = 8
 };
 
 }} // namespace qpid::framing

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/sys/Shlib.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/sys/Shlib.h?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/sys/Shlib.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/sys/Shlib.h Tue Mar 11 15:13:10 2008
@@ -56,12 +56,6 @@
                                        this->getSymbol(symbol)));
     }
     
-    /** Look up symbol in shared library. */
-        template <class T*>
-    void* getSymbol(const std::string& symbol) {
-        return getSymbol<T>(symbol.c_str());
-    }
-
   private:
     void* handle;
     void load(const char* libname);

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/sys/Time.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/sys/Time.h?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/sys/Time.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/sys/Time.h Tue Mar 11 15:13:10 2008
@@ -24,6 +24,7 @@
 
 #include <stdint.h>
 #include <limits>
+#include <iosfwd>
 
 namespace qpid {
 namespace sys {
@@ -54,6 +55,8 @@
   friend bool operator>(const AbsTime& a, const AbsTime& b);
 };
 
+std::ostream& operator << (std::ostream&, const AbsTime&);
+
 class Duration {
     static int64_t max() { return std::numeric_limits<int64_t>::max(); }
     int64_t nanosecs;
@@ -67,6 +70,7 @@
     inline operator int64_t() const;
 };
 
+std::ostream& operator << (std::ostream&, const Duration&);
 
 AbsTime::AbsTime(const AbsTime& t, const Duration& d) :
     time_ns(d == Duration::max() ? max() : t.time_ns+d.nanosecs)

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/sys/posix/Time.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/sys/posix/Time.cpp?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/sys/posix/Time.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/sys/posix/Time.cpp Tue Mar 11 15:13:10 2008
@@ -22,7 +22,7 @@
 #include "PrivatePosix.h"
 
 #include "qpid/sys/Time.h"
-
+#include <ostream>
 #include <time.h>
 #include <sys/time.h>
 
@@ -40,7 +40,7 @@
 struct timespec& toTimespec(struct timespec& ts, const Duration& t) {
     ts.tv_sec  = t / TIME_SEC;
     ts.tv_nsec = t % TIME_SEC;
-    return ts;
+    return ts; 
 }
 
 struct timeval& toTimeval(struct timeval& tv, const Duration& t) {
@@ -51,6 +51,30 @@
 
 Duration toTime(const struct timespec& ts) {
     return ts.tv_sec*TIME_SEC + ts.tv_nsec;
+}
+
+std::ostream& operator<<(std::ostream& o, const Duration& d) {
+    return o << int64_t(d) << "ns";   
+}
+
+std::ostream& operator<<(std::ostream& o, const AbsTime& t) {
+    static const char * month_abbrevs[] = {
+        "jan", "feb", "mar", "apr", "may", "jun", "jul", "aug", "sep", "oct", "nov", "dec"
+    };
+    struct tm * timeinfo;
+    time_t rawtime(t.timeValue()/TIME_SEC);
+    timeinfo = localtime (&rawtime);
+    char time_string[100];
+    sprintf ( time_string,
+              "%d-%s-%02d %02d:%02d:%02d",
+              1900 + timeinfo->tm_year,
+              month_abbrevs[timeinfo->tm_mon],
+              timeinfo->tm_mday,
+              timeinfo->tm_hour,
+              timeinfo->tm_min,
+              timeinfo->tm_sec
+    );
+    return o << time_string;
 }
 
 }}

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/apply.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/apply.cpp?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/apply.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/apply.cpp Tue Mar 11 15:13:10 2008
@@ -26,7 +26,7 @@
 
 using  namespace qpid::amqp_0_10;
 
-struct GetCode : public ConstApplyFunctor<uint8_t> {
+struct GetCode : public ApplyFunctor<uint8_t> {
     template <class T> uint8_t operator()(const T&) const { return T::CODE; }
 };
 

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/serialize.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/serialize.cpp?rev=636126&r1=636125&r2=636126&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/serialize.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/serialize.cpp Tue Mar 11 15:13:10 2008
@@ -24,6 +24,7 @@
 #include "qpid/amqp_0_10/Codec.h"
 #include "qpid/amqp_0_10/specification.h"
 #include "qpid/amqp_0_10/ControlHolder.h"
+#include "qpid/amqp_0_10/Frame.h"
 
 #include <boost/test/test_case_template.hpp>
 #include <boost/type_traits/is_arithmetic.hpp>
@@ -57,6 +58,10 @@
 
 } // qpid
 
+namespace std {
+// Dummy += for back inserters so we can use them with the decoder.
+template <class C> back_insert_iterator<C>& operator+=(back_insert_iterator<C>& bi, size_t) { return bi; }
+}
 
 QPID_AUTO_TEST_SUITE(SerializeTestSuite)
 
@@ -73,12 +78,14 @@
 
 typedef mpl::vector<Bit, Boolean, Char, Int32, Int64, Int8, Uint16, CharUtf32, Uint32, Uint64, Bin8, Uint8>::type IntegralTypes;
 typedef mpl::vector<Bin1024, Bin128, Bin16, Bin256, Bin32, Bin40, Bin512, Bin64, Bin72>::type BinTypes;
+// FIXME aconway 2008-03-07: float encoding
 typedef mpl::vector<Double, Float>::type FloatTypes;
 typedef mpl::vector<SequenceNo, Uuid, Datetime, Dec32, Dec64> FixedSizeClassTypes;
 typedef mpl::vector<Vbin8, Str8Latin, Str8, Str8Utf16, Vbin16, Str16Latin, Str16, Str16Utf16, Vbin32> VariableSizeTypes;
 
 
-typedef concat4<IntegralTypes, BinTypes, FloatTypes, FixedSizeClassTypes>::type FixedSizeTypes;
+// FIXME aconway 2008-03-07: float encoding
+typedef concat3<IntegralTypes, BinTypes, /*FloatTypes, */ FixedSizeClassTypes>::type FixedSizeTypes;
 typedef concat2<FixedSizeTypes, VariableSizeTypes>::type AllTypes;
 
 // TODO aconway 2008-02-20: should test 64 bit integrals for order also.
@@ -133,7 +140,9 @@
     bool encoded, decoded;
     char value;
     TestMe(char v) : encoded(), decoded(), value(v) {}
-    template <class S> void encode(S& s) { encoded=true; s(value); }
+    template <class S> void encode(S& s) const {
+        const_cast<TestMe*>(this)->encoded=true; s(value);
+    }
     template <class S> void decode(S& s) { decoded=true; s(value); }
     template <class S> void serialize(S& s) { s.split(*this); }
 };
@@ -172,6 +181,48 @@
     BOOST_CHECK_EQUAL(tune.maxFrameSize, 2u);
     BOOST_CHECK_EQUAL(tune.heartbeatMin, 3u);
     BOOST_CHECK_EQUAL(tune.heartbeatMax, 4u);
+}
+
+BOOST_AUTO_TEST_CASE(testFrameEncodeDecode) {
+    static const int overhead=12;
+    string data;
+    Frame r, c;
+    char d1[]="abcdefg";
+    r.refer(d1, d1+sizeof(d1));
+    r.setFlags(Frame::FIRST_FRAME);
+    r.setType(CONTROL);
+    r.setChannel(32);
+    r.setTrack(1);
+    char d2[]="01234567";
+    c.copy(d2, d2+sizeof(d2));
+
+    BOOST_CHECK_EQUAL(overhead+sizeof(d1), Codec::size(r));
+    BOOST_CHECK_EQUAL(overhead+sizeof(d2), Codec::size(c));
+    Codec::encode(std::back_inserter(data))(r)(c);
+    BOOST_CHECK_EQUAL(data.size(), Codec::size(r)+Codec::size(c));
+
+    FrameHeader fh;
+    std::string::iterator i = Codec::decode(data.begin())(fh).pos();
+    size_t s = fh.size();
+    BOOST_CHECK_EQUAL(s, sizeof(d1));
+    BOOST_CHECK_EQUAL(std::string(i, i+s), std::string(d1, d1+s));
+
+                      
+    Frame f1, f2;
+    Codec::decode(data.begin())(f1)(f2);
+    BOOST_CHECK_EQUAL(f1.size(), sizeof(d1));
+    BOOST_CHECK_EQUAL(std::string(f1.begin(), f1.size()),
+                      std::string(d1, sizeof(d1)));
+    BOOST_CHECK_EQUAL(f1.size(), r.size());
+    BOOST_CHECK_EQUAL(f1.getFlags(), Frame::FIRST_FRAME);
+    BOOST_CHECK_EQUAL(f1.getType(), CONTROL);
+    BOOST_CHECK_EQUAL(f1.getChannel(), 32);
+    BOOST_CHECK_EQUAL(f1.getTrack(), 1);
+
+    BOOST_CHECK_EQUAL(f2.size(), c.size());
+    BOOST_CHECK_EQUAL(std::string(f2.begin(), f2.end()),
+                      std::string(d2, d2+sizeof(d2)));
+    
 }
 
 QPID_AUTO_TEST_SUITE_END()