Object parsing can be private now
[bgpcep.git] / pcep / impl / src / main / java / org / opendaylight / protocol / pcep / impl / PCEPMessageFactory.java
index 0753ebea2319eaa2c6547e41cbda6890884f2bd5..627b1673602c291a128cd1bf338f2ce4a8bfaae3 100644 (file)
@@ -7,51 +7,30 @@
  */
 package org.opendaylight.protocol.pcep.impl;
 
-import java.util.HashMap;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.UnpooledByteBufAllocator;
+
+import java.util.ArrayList;
+import java.util.List;
 
 import org.opendaylight.protocol.framework.DeserializerException;
 import org.opendaylight.protocol.framework.DocumentedException;
-import org.opendaylight.protocol.framework.ProtocolMessage;
 import org.opendaylight.protocol.framework.ProtocolMessageFactory;
 import org.opendaylight.protocol.pcep.PCEPDeserializerException;
-import org.opendaylight.protocol.pcep.PCEPDocumentedException;
-import org.opendaylight.protocol.pcep.PCEPErrors;
-import org.opendaylight.protocol.pcep.PCEPMessage;
-import org.opendaylight.protocol.pcep.impl.message.PCCreateMessageParser;
-import org.opendaylight.protocol.pcep.impl.message.PCEPCloseMessageParser;
-import org.opendaylight.protocol.pcep.impl.message.PCEPErrorMessageParser;
-import org.opendaylight.protocol.pcep.impl.message.PCEPKeepAliveMessageParser;
-import org.opendaylight.protocol.pcep.impl.message.PCEPNotificationMessageParser;
-import org.opendaylight.protocol.pcep.impl.message.PCEPOpenMessageParser;
-import org.opendaylight.protocol.pcep.impl.message.PCEPRawMessage;
-import org.opendaylight.protocol.pcep.impl.message.PCEPReplyMessageParser;
-import org.opendaylight.protocol.pcep.impl.message.PCEPReportMessageParser;
-import org.opendaylight.protocol.pcep.impl.message.PCEPRequestMessageParser;
-import org.opendaylight.protocol.pcep.impl.message.PCEPUpdateRequestMessageParser;
-import org.opendaylight.protocol.pcep.impl.message.PCEPXRAddTunnelMessageParser;
-import org.opendaylight.protocol.pcep.impl.message.PCEPXRDeleteTunnelMessageParser;
-import org.opendaylight.protocol.pcep.message.PCCreateMessage;
-import org.opendaylight.protocol.pcep.message.PCEPCloseMessage;
-import org.opendaylight.protocol.pcep.message.PCEPErrorMessage;
-import org.opendaylight.protocol.pcep.message.PCEPKeepAliveMessage;
-import org.opendaylight.protocol.pcep.message.PCEPNotificationMessage;
-import org.opendaylight.protocol.pcep.message.PCEPOpenMessage;
-import org.opendaylight.protocol.pcep.message.PCEPReplyMessage;
-import org.opendaylight.protocol.pcep.message.PCEPReportMessage;
-import org.opendaylight.protocol.pcep.message.PCEPRequestMessage;
-import org.opendaylight.protocol.pcep.message.PCEPUpdateRequestMessage;
-import org.opendaylight.protocol.pcep.message.PCEPXRAddTunnelMessage;
-import org.opendaylight.protocol.pcep.message.PCEPXRDeleteTunnelMessage;
+import org.opendaylight.protocol.pcep.spi.MessageHandlerRegistry;
+import org.opendaylight.protocol.pcep.spi.MessageSerializer;
 import org.opendaylight.protocol.util.ByteArray;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.pcep.types.rev131005.Message;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
 import com.google.common.primitives.UnsignedBytes;
 
 /**
- * Factory for subclasses of {@link org.opendaylight.protocol.pcep.PCEPMessage PCEPMessage}
+ * A PCEP message parser which also does validation.
  */
-public class PCEPMessageFactory implements ProtocolMessageFactory {
+public final class PCEPMessageFactory implements ProtocolMessageFactory<Message> {
 
        private final static Logger logger = LoggerFactory.getLogger(PCEPMessageFactory.class);
 
@@ -62,78 +41,30 @@ public class PCEPMessageFactory implements ProtocolMessageFactory {
        public final static int COMMON_HEADER_LENGTH = 4; // bytes
 
        /**
-        * Type identifier for {@link org.opendaylight.protocol.pcep.PCEPMessage PCEPMessage}
+        * Current supported version of PCEP.
         */
-       public enum PCEPMessageType {
-               OPEN(1), NOTIFICATION(5), KEEPALIVE(2), RESPONSE(4), REQUEST(3), ERROR(6), CLOSE(7), UPDATE_REQUEST(11), STATUS_REPORT(10),
-               // TODO: replace with actual values by IANA
-               XR_ADD_TUNNEL(8), XR_DELETE_TUNNEL(9), PCCREATE(12);
-
-               private final int identifier;
-
-               PCEPMessageType(final int identifier) {
-                       this.identifier = identifier;
-               }
-
-               public int getIdentifier() {
-                       return this.identifier;
-               }
-
-               public static PCEPMessageType getFromInt(final int type) throws PCEPDeserializerException {
-
-                       for (final PCEPMessageType type_e : PCEPMessageType.values()) {
-                               if (type_e.getIdentifier() == type)
-                                       return type_e;
-                       }
+       public static final int PCEP_VERSION = 1;
 
-                       throw new PCEPDeserializerException("Unknown PCEPMessage Class identifier. Passed: " + type + "; Known: "
-                                       + PCEPMessageType.values() + ".");
-               }
-       }
-
-       private static class MapOfParsers extends HashMap<PCEPMessageType, PCEPMessageParser> {
-
-               private static final long serialVersionUID = -5715193806554448822L;
+       private static final int VERSION_SF_LENGTH = 3;
 
-               private final static MapOfParsers instance = new MapOfParsers();
+       private static final int VER_FLAGS_MF_LENGTH = 1;
+       private static final int TYPE_F_LENGTH = 1;
+       private static final int LENGTH_F_LENGTH = 2;
 
-               private MapOfParsers() {
-                       this.fillInMap();
-               }
+       private static final int VER_FLAGS_MF_OFFSET = 0;
+       private static final int TYPE_F_OFFSET = VER_FLAGS_MF_LENGTH + VER_FLAGS_MF_OFFSET;
+       private static final int LENGTH_F_OFFSET = TYPE_F_LENGTH + TYPE_F_OFFSET;
 
-               private void fillInMap() {
-                       this.put(PCEPMessageType.OPEN, new PCEPOpenMessageParser());
-                       this.put(PCEPMessageType.KEEPALIVE, new PCEPKeepAliveMessageParser());
-                       this.put(PCEPMessageType.NOTIFICATION, new PCEPNotificationMessageParser());
-                       this.put(PCEPMessageType.ERROR, new PCEPErrorMessageParser());
-                       this.put(PCEPMessageType.RESPONSE, new PCEPReplyMessageParser());
-                       this.put(PCEPMessageType.REQUEST, new PCEPRequestMessageParser());
-                       this.put(PCEPMessageType.UPDATE_REQUEST, new PCEPUpdateRequestMessageParser());
-                       this.put(PCEPMessageType.STATUS_REPORT, new PCEPReportMessageParser());
-                       this.put(PCEPMessageType.CLOSE, new PCEPCloseMessageParser());
-                       this.put(PCEPMessageType.XR_ADD_TUNNEL, new PCEPXRAddTunnelMessageParser());
-                       this.put(PCEPMessageType.XR_DELETE_TUNNEL, new PCEPXRDeleteTunnelMessageParser());
-                       this.put(PCEPMessageType.PCCREATE, new PCCreateMessageParser());
-               }
+       private final MessageHandlerRegistry registry;
 
-               public static MapOfParsers getInstance() {
-                       return instance;
-               }
+       public PCEPMessageFactory(final MessageHandlerRegistry registry) {
+               this.registry = Preconditions.checkNotNull(registry);
        }
 
-       /**
-        * 
-        * @param bytes assume array of bytes without common header
-        * @param msgHeader
-        * @return Parsed specific PCEPMessage
-        * @throws PCEPDeserializerException
-        * @throws PCEPDocumentedException
-        */
-
        @Override
-       public ProtocolMessage parse(final byte[] bytes) throws DeserializerException, DocumentedException {
-               if (bytes == null || bytes.length == 0)
-                       throw new IllegalArgumentException("Array of bytes is mandatory.");
+       public Message parse(final byte[] bytes) throws DeserializerException, DocumentedException {
+               Preconditions.checkArgument(bytes != null, "Bytes may not be null");
+               Preconditions.checkArgument(bytes.length != 0, "Bytes may not be empty");
 
                logger.trace("Attempt to parse message from bytes: {}", ByteArray.bytesToHexString(bytes));
 
@@ -143,84 +74,61 @@ public class PCEPMessageFactory implements ProtocolMessageFactory {
 
                final byte[] msgBody = ByteArray.cutBytes(bytes, TYPE_SIZE + 1 + LENGTH_SIZE);
 
-               if (msgBody.length != (msgLength - COMMON_HEADER_LENGTH))
-                       throw new DeserializerException("Size don't match size specified in header. Passed: " + msgBody.length + "; Expected: "
-                                       + (msgLength - COMMON_HEADER_LENGTH) + ". " + msgLength);
+               if (msgBody.length != msgLength - COMMON_HEADER_LENGTH) {
+                       throw new DeserializerException("Body size " + msgBody.length + " does not match header size "
+                                       + (msgLength - COMMON_HEADER_LENGTH));
+               }
+
+               final List<Message> errors = new ArrayList<>();
+               Message msg = null;
 
-               /*
-                * if PCEPObjectIdentifier.getObjectClassFromInt() dont't throws
-                * exception and if returned null we know the error type
-                */
-               PCEPMessageType msgType;
                try {
-                       msgType = PCEPMessageType.getFromInt(type);
+                       msg = this.registry.getMessageParser(type).parseMessage(msgBody, errors);
                } catch (final PCEPDeserializerException e) {
+                       logger.debug("Unexpected deserializer problem", e);
                        throw new DeserializerException(e.getMessage(), e);
                }
-               if (msgType == null)
-                       throw new DocumentedException("Unhandled message type " + type, new PCEPDocumentedException("Unhandled message type " + type, PCEPErrors.CAPABILITY_NOT_SUPPORTED));
 
-               PCEPMessage msg;
-               try {
-                       msg = new PCEPRawMessage(PCEPObjectFactory.parseObjects(msgBody), msgType);
-               } catch (final PCEPDeserializerException e) {
-                       throw new DeserializerException(e.getMessage(), e);
-               } catch (final PCEPDocumentedException e) {
-                       throw new DocumentedException(e.getMessage(), e);
+               if (!errors.isEmpty()) {
+                       // FIXME: we have a bunch of error messages, how can we send them back?
                }
+
                logger.debug("Message was parsed. {}", msg);
                return msg;
        }
 
        @Override
-       public byte[] put(final ProtocolMessage msg) {
-               final PCEPMessage pcepMsg = (PCEPMessage) msg;
-               if (pcepMsg == null)
+       public byte[] put(final Message msg) {
+               if (msg == null) {
                        throw new IllegalArgumentException("PCEPMessage is mandatory.");
-
-               final PCEPMessageType msgType;
-
-               if (pcepMsg instanceof PCEPOpenMessage) {
-                       msgType = PCEPMessageType.OPEN;
-               } else if (pcepMsg instanceof PCEPKeepAliveMessage) {
-                       msgType = PCEPMessageType.KEEPALIVE;
-               } else if (pcepMsg instanceof PCEPCloseMessage) {
-                       msgType = PCEPMessageType.CLOSE;
-               } else if (pcepMsg instanceof PCEPReplyMessage) {
-                       msgType = PCEPMessageType.RESPONSE;
-               } else if (pcepMsg instanceof PCEPRequestMessage) {
-                       msgType = PCEPMessageType.REQUEST;
-               } else if (pcepMsg instanceof PCEPNotificationMessage) {
-                       msgType = PCEPMessageType.NOTIFICATION;
-               } else if (pcepMsg instanceof PCEPErrorMessage) {
-                       msgType = PCEPMessageType.ERROR;
-               } else if (pcepMsg instanceof PCEPReportMessage) {
-                       msgType = PCEPMessageType.STATUS_REPORT;
-               } else if (pcepMsg instanceof PCEPUpdateRequestMessage) {
-                       msgType = PCEPMessageType.UPDATE_REQUEST;
-               } else if (pcepMsg instanceof PCEPXRAddTunnelMessage) {
-                       msgType = PCEPMessageType.XR_ADD_TUNNEL;
-               } else if (pcepMsg instanceof PCEPXRDeleteTunnelMessage) {
-                       msgType = PCEPMessageType.XR_DELETE_TUNNEL;
-               } else if (pcepMsg instanceof PCCreateMessage) {
-                       msgType = PCEPMessageType.PCCREATE;
-               } else {
-                       logger.error("Unknown instance of PCEPMessage. Message class: {}", pcepMsg.getClass());
-                       throw new IllegalArgumentException("Unknown instance of PCEPMessage. Passed " + pcepMsg.getClass());
                }
 
-               logger.trace("Serializing {}", msgType);
+               final ByteBuf buf = new UnpooledByteBufAllocator(false).buffer();
+
+               final MessageSerializer serializer = this.registry.getMessageSerializer(msg);
+
+               serializer.serializeMessage(msg, buf);
+
+               final byte[] msgBody = new byte[buf.readableBytes()];
+
+               buf.getBytes(0, msgBody);
+
+               final byte[] headerBytes = new byte[COMMON_HEADER_LENGTH];
+
+               // msgVer_Flag
+               headerBytes[VER_FLAGS_MF_OFFSET] = (byte) (PCEP_VERSION << (Byte.SIZE - VERSION_SF_LENGTH));
 
-               final byte[] msgBody = MapOfParsers.getInstance().get(msgType).put(pcepMsg);
+               // msgType
+               headerBytes[TYPE_F_OFFSET] = (byte) serializer.getMessageType();
 
-               final PCEPMessageHeader msgHeader = new PCEPMessageHeader(msgType.getIdentifier(), msgBody.length
-                               + PCEPMessageHeader.COMMON_HEADER_LENGTH, PCEPMessage.PCEP_VERSION);
+               // msgLength
+               System.arraycopy(ByteArray.intToBytes(msgBody.length + COMMON_HEADER_LENGTH), Integer.SIZE / Byte.SIZE - LENGTH_F_LENGTH,
+                               headerBytes, LENGTH_F_OFFSET, LENGTH_F_LENGTH);
 
-               final byte[] headerBytes = msgHeader.toBytes();
                final byte[] retBytes = new byte[headerBytes.length + msgBody.length];
 
                ByteArray.copyWhole(headerBytes, retBytes, 0);
-               ByteArray.copyWhole(msgBody, retBytes, PCEPMessageHeader.COMMON_HEADER_LENGTH);
+               ByteArray.copyWhole(msgBody, retBytes, COMMON_HEADER_LENGTH);
 
                return retBytes;
        }