Untangle BindingCodecContext/BindingNormalizedNodeCodecRegistry
[mdsal.git] / binding / mdsal-binding-dom-codec / src / main / java / org / opendaylight / mdsal / binding / dom / codec / impl / BindingCodecContext.java
index d0bc62deead2c15f7ac328814f1c623bd09e4646..edde979cb8623aad87591c77a3b66e259b6541cb 100644 (file)
@@ -7,8 +7,17 @@
  */
 package org.opendaylight.mdsal.binding.dom.codec.impl;
 
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Verify.verify;
+import static java.util.Objects.requireNonNull;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
 import com.google.common.collect.ImmutableMap;
+import java.io.IOException;
+import java.lang.reflect.Field;
 import java.lang.reflect.Method;
 import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Type;
@@ -19,45 +28,48 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.concurrent.Callable;
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
+import java.util.concurrent.ExecutionException;
+import org.eclipse.jdt.annotation.NonNull;
+import org.eclipse.jdt.annotation.Nullable;
+import org.opendaylight.binding.runtime.api.BindingRuntimeContext;
 import org.opendaylight.mdsal.binding.dom.codec.api.BindingCodecTree;
 import org.opendaylight.mdsal.binding.dom.codec.api.BindingCodecTreeNode;
+import org.opendaylight.mdsal.binding.dom.codec.api.BindingDataObjectCodecTreeNode;
+import org.opendaylight.mdsal.binding.dom.codec.api.BindingStreamEventWriter;
 import org.opendaylight.mdsal.binding.dom.codec.impl.NodeCodecContext.CodecContextFactory;
-import org.opendaylight.mdsal.binding.generator.util.BindingRuntimeContext;
+import org.opendaylight.mdsal.binding.dom.codec.loader.CodecClassLoader;
+import org.opendaylight.mdsal.binding.dom.codec.spi.BindingSchemaMapping;
 import org.opendaylight.mdsal.binding.model.api.GeneratedType;
 import org.opendaylight.mdsal.binding.spec.reflect.BindingReflections;
-import org.opendaylight.yangtools.concepts.Codec;
+import org.opendaylight.yangtools.concepts.Delegator;
+import org.opendaylight.yangtools.concepts.IllegalArgumentCodec;
 import org.opendaylight.yangtools.concepts.Immutable;
 import org.opendaylight.yangtools.util.ClassLoaderUtils;
 import org.opendaylight.yangtools.yang.binding.Action;
-import org.opendaylight.yangtools.yang.binding.BindingMapping;
-import org.opendaylight.yangtools.yang.binding.BindingStreamEventWriter;
 import org.opendaylight.yangtools.yang.binding.DataContainer;
 import org.opendaylight.yangtools.yang.binding.DataObject;
-import org.opendaylight.yangtools.yang.binding.DataObjectSerializer;
 import org.opendaylight.yangtools.yang.binding.Identifiable;
 import org.opendaylight.yangtools.yang.binding.Identifier;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
-import org.opendaylight.yangtools.yang.binding.InstanceIdentifier.IdentifiableItem;
 import org.opendaylight.yangtools.yang.binding.Notification;
+import org.opendaylight.yangtools.yang.binding.OpaqueObject;
 import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
-import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifierWithPredicates;
 import org.opendaylight.yangtools.yang.data.api.schema.stream.NormalizedNodeStreamWriter;
+import org.opendaylight.yangtools.yang.model.api.AnydataSchemaNode;
+import org.opendaylight.yangtools.yang.model.api.AnyxmlSchemaNode;
 import org.opendaylight.yangtools.yang.model.api.DataNodeContainer;
 import org.opendaylight.yangtools.yang.model.api.DataSchemaNode;
 import org.opendaylight.yangtools.yang.model.api.DocumentedNode.WithStatus;
 import org.opendaylight.yangtools.yang.model.api.LeafListSchemaNode;
 import org.opendaylight.yangtools.yang.model.api.LeafSchemaNode;
 import org.opendaylight.yangtools.yang.model.api.ListSchemaNode;
-import org.opendaylight.yangtools.yang.model.api.SchemaNode;
 import org.opendaylight.yangtools.yang.model.api.SchemaPath;
 import org.opendaylight.yangtools.yang.model.api.TypeDefinition;
 import org.opendaylight.yangtools.yang.model.api.TypedDataSchemaNode;
-import org.opendaylight.yangtools.yang.model.api.type.BooleanTypeDefinition;
-import org.opendaylight.yangtools.yang.model.api.type.EmptyTypeDefinition;
+import org.opendaylight.yangtools.yang.model.api.stmt.SchemaNodeIdentifier.Absolute;
 import org.opendaylight.yangtools.yang.model.api.type.IdentityrefTypeDefinition;
 import org.opendaylight.yangtools.yang.model.api.type.InstanceIdentifierTypeDefinition;
 import org.opendaylight.yangtools.yang.model.api.type.LeafrefTypeDefinition;
@@ -65,21 +77,57 @@ import org.opendaylight.yangtools.yang.model.api.type.UnionTypeDefinition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-final class BindingCodecContext implements CodecContextFactory, BindingCodecTree, Immutable {
+final class BindingCodecContext implements CodecContextFactory, BindingCodecTree, DataObjectSerializerRegistry,
+        Immutable {
+    private final class DataObjectSerializerProxy implements DataObjectSerializer, Delegator<DataObjectStreamer<?>> {
+        private final @NonNull DataObjectStreamer<?> delegate;
+
+        DataObjectSerializerProxy(final DataObjectStreamer<?> delegate) {
+            this.delegate = requireNonNull(delegate);
+        }
+
+        @Override
+        public DataObjectStreamer<?> getDelegate() {
+            return delegate;
+        }
+
+        @Override
+        public void serialize(final DataObject obj, final BindingStreamEventWriter stream) throws IOException {
+            delegate.serialize(BindingCodecContext.this, obj, stream);
+        }
+    }
+
     private static final Logger LOG = LoggerFactory.getLogger(BindingCodecContext.class);
 
-    private final Codec<YangInstanceIdentifier, InstanceIdentifier<?>> instanceIdentifierCodec;
-    private final Codec<QName, Class<?>> identityCodec;
-    private final BindingNormalizedNodeCodecRegistry registry;
+    private final LoadingCache<Class<?>, DataObjectStreamer<?>> streamers = CacheBuilder.newBuilder().build(
+        new CacheLoader<Class<?>, DataObjectStreamer<?>>() {
+            @Override
+            public DataObjectStreamer<?> load(final Class<?> key) throws ReflectiveOperationException {
+                final Class<?> streamer = DataObjectStreamerGenerator.generateStreamer(loader, BindingCodecContext.this,
+                    key);
+                final Field instance = streamer.getDeclaredField(DataObjectStreamerGenerator.INSTANCE_FIELD);
+                return (DataObjectStreamer<?>) instance.get(null);
+            }
+        });
+    private final LoadingCache<Class<?>, DataObjectSerializer> serializers = CacheBuilder.newBuilder().build(
+        new CacheLoader<Class<?>, DataObjectSerializer>() {
+            @Override
+            public DataObjectSerializer load(final Class<?> key) throws ExecutionException {
+                return new DataObjectSerializerProxy(streamers.get(key));
+            }
+        });
+
+    private final @NonNull CodecClassLoader loader = CodecClassLoader.create();
+    private final InstanceIdentifierCodec instanceIdentifierCodec;
+    private final @NonNull IdentityCodec identityCodec;
     private final BindingRuntimeContext context;
     private final SchemaRootCodecContext<?> root;
 
-    BindingCodecContext(final BindingRuntimeContext context, final BindingNormalizedNodeCodecRegistry registry) {
-        this.context = Preconditions.checkNotNull(context, "Binding Runtime Context is required.");
+    BindingCodecContext(final BindingRuntimeContext context) {
+        this.context = requireNonNull(context, "Binding Runtime Context is required.");
         this.root = SchemaRootCodecContext.create(this);
         this.identityCodec = new IdentityCodec(context);
         this.instanceIdentifierCodec = new InstanceIdentifierCodec(this);
-        this.registry = Preconditions.checkNotNull(registry);
     }
 
     @Override
@@ -87,18 +135,33 @@ final class BindingCodecContext implements CodecContextFactory, BindingCodecTree
         return context;
     }
 
-    Codec<YangInstanceIdentifier, InstanceIdentifier<?>> getInstanceIdentifierCodec() {
+    @Override
+    public CodecClassLoader getLoader() {
+        return loader;
+    }
+
+    InstanceIdentifierCodec getInstanceIdentifierCodec() {
         return instanceIdentifierCodec;
     }
 
-    public Codec<QName, Class<?>> getIdentityCodec() {
+    @Override
+    public IdentityCodec getIdentityCodec() {
         return identityCodec;
     }
 
-    @SuppressWarnings({"rawtypes", "unchecked"})
     @Override
     public DataObjectSerializer getEventStreamSerializer(final Class<?> type) {
-        return registry.getSerializer((Class) type);
+        return serializers.getUnchecked(type);
+    }
+
+    @Override
+    public DataObjectStreamer<?> getDataObjectSerializer(final Class<?> type) {
+        return streamers.getUnchecked(type);
+    }
+
+    @Override
+    public DataObjectSerializer getSerializer(final Class<? extends DataObject> type) {
+        return serializers.getUnchecked(type);
     }
 
     public Entry<YangInstanceIdentifier, BindingStreamEventWriter> newWriter(final InstanceIdentifier<?> path,
@@ -128,7 +191,7 @@ final class BindingCodecContext implements CodecContextFactory, BindingCodecTree
         DataContainerCodecContext<?,?> currentNode = root;
         for (final InstanceIdentifier.PathArgument bindingArg : binding.getPathArguments()) {
             currentNode = currentNode.bindingPathArgumentChild(bindingArg, builder);
-            Preconditions.checkArgument(currentNode != null, "Supplied Instance Identifier %s is not valid.", binding);
+            checkArgument(currentNode != null, "Supplied Instance Identifier %s is not valid.", binding);
         }
         return currentNode;
     }
@@ -145,16 +208,16 @@ final class BindingCodecContext implements CodecContextFactory, BindingCodecTree
      *         binding representation (choice, case, leaf).
      *
      */
-    @Nullable NodeCodecContext<?> getCodecContextNode(final @Nonnull YangInstanceIdentifier dom,
+    @Nullable BindingDataObjectCodecTreeNode<?> getCodecContextNode(final @NonNull YangInstanceIdentifier dom,
             final @Nullable Collection<InstanceIdentifier.PathArgument> bindingArguments) {
-        NodeCodecContext<?> currentNode = root;
+        NodeCodecContext currentNode = root;
         ListNodeCodecContext<?> currentList = null;
 
         for (final YangInstanceIdentifier.PathArgument domArg : dom.getPathArguments()) {
-            Preconditions.checkArgument(currentNode instanceof DataContainerCodecContext<?,?>,
+            checkArgument(currentNode instanceof DataContainerCodecContext,
                 "Unexpected child of non-container node %s", currentNode);
-            final DataContainerCodecContext<?,?> previous = (DataContainerCodecContext<?,?>) currentNode;
-            final NodeCodecContext<?> nextNode = previous.yangPathArgumentChild(domArg);
+            final DataContainerCodecContext<?,?> previous = (DataContainerCodecContext<?, ?>) currentNode;
+            final NodeCodecContext nextNode = previous.yangPathArgumentChild(domArg);
 
             /*
              * List representation in YANG Instance Identifier consists of two
@@ -165,7 +228,7 @@ final class BindingCodecContext implements CodecContextFactory, BindingCodecTree
              * Identifier as Item or IdentifiableItem
              */
             if (currentList != null) {
-                Preconditions.checkArgument(currentList == nextNode,
+                checkArgument(currentList == nextNode,
                         "List should be referenced two times in YANG Instance Identifier %s", dom);
 
                 // We entered list, so now we have all information to emit
@@ -183,13 +246,13 @@ final class BindingCodecContext implements CodecContextFactory, BindingCodecTree
                 // We do not add path argument for choice, since
                 // it is not supported by binding instance identifier.
                 currentNode = nextNode;
-            } else if (nextNode instanceof DataContainerCodecContext<?,?>) {
+            } else if (nextNode instanceof DataContainerCodecContext) {
                 if (bindingArguments != null) {
-                    bindingArguments.add(((DataContainerCodecContext<?,?>) nextNode).getBindingPathArgument(domArg));
+                    bindingArguments.add(((DataContainerCodecContext<?, ?>) nextNode).getBindingPathArgument(domArg));
                 }
                 currentNode = nextNode;
-            } else if (nextNode instanceof LeafNodeCodecContext) {
-                LOG.debug("Instance identifier referencing a leaf is not representable (%s)", dom);
+            } else if (nextNode instanceof ValueNodeCodecContext) {
+                LOG.debug("Instance identifier referencing a leaf is not representable ({})", dom);
                 return null;
             }
         }
@@ -197,11 +260,11 @@ final class BindingCodecContext implements CodecContextFactory, BindingCodecTree
         // Algorithm ended in list as whole representation
         // we sill need to emit identifier for list
         if (currentNode instanceof ChoiceNodeCodecContext) {
-            LOG.debug("Instance identifier targeting a choice is not representable (%s)", dom);
+            LOG.debug("Instance identifier targeting a choice is not representable ({})", dom);
             return null;
         }
         if (currentNode instanceof CaseNodeCodecContext) {
-            LOG.debug("Instance identifier targeting a case is not representable (%s)", dom);
+            LOG.debug("Instance identifier targeting a case is not representable ({})", dom);
             return null;
         }
 
@@ -211,7 +274,12 @@ final class BindingCodecContext implements CodecContextFactory, BindingCodecTree
             }
             return currentList;
         }
-        return currentNode;
+        if (currentNode != null) {
+            verify(currentNode instanceof BindingDataObjectCodecTreeNode, "Illegal return node %s for identifier %s",
+                currentNode, dom);
+            return (BindingDataObjectCodecTreeNode<?>) currentNode;
+        }
+        return null;
     }
 
     NotificationCodecContext<?> getNotificationContext(final SchemaPath notification) {
@@ -227,40 +295,40 @@ final class BindingCodecContext implements CodecContextFactory, BindingCodecTree
     }
 
     @Override
-    public ImmutableMap<String, LeafNodeCodecContext<?>> getLeafNodes(final Class<?> parentClass,
+    public ImmutableMap<Method, ValueNodeCodecContext> getLeafNodes(final Class<?> parentClass,
             final DataNodeContainer childSchema) {
         final Map<String, DataSchemaNode> getterToLeafSchema = new HashMap<>();
         for (final DataSchemaNode leaf : childSchema.getChildNodes()) {
-            if (leaf instanceof TypedDataSchemaNode) {
-                getterToLeafSchema.put(getGetterName(leaf, ((TypedDataSchemaNode) leaf).getType()), leaf);
+            if (leaf instanceof TypedDataSchemaNode || leaf instanceof AnyxmlSchemaNode
+                    || leaf instanceof AnydataSchemaNode) {
+                getterToLeafSchema.put(BindingSchemaMapping.getGetterMethodName(leaf), leaf);
             }
         }
         return getLeafNodesUsingReflection(parentClass, getterToLeafSchema);
     }
 
-    private static String getGetterName(final SchemaNode node, final TypeDefinition<?> typeDef) {
-        final String suffix = BindingMapping.getGetterSuffix(node.getQName());
-        // Bug 8903: If it is a derived type of boolean or empty, not an inner type, then the return type
-        // of method would be the generated type of typedef not build-in types, so here it should be 'get'.
-        if ((typeDef instanceof BooleanTypeDefinition || typeDef instanceof EmptyTypeDefinition)
-                && (typeDef.getPath().equals(node.getPath()) || typeDef.getBaseType() == null)) {
-            return "is" + suffix;
-        }
-        return "get" + suffix;
-    }
-
-    private ImmutableMap<String, LeafNodeCodecContext<?>> getLeafNodesUsingReflection(final Class<?> parentClass,
-            final Map<String, DataSchemaNode> getterToLeafSchema) {
-        final Map<String, LeafNodeCodecContext<?>> leaves = new HashMap<>();
+    private ImmutableMap<Method, ValueNodeCodecContext> getLeafNodesUsingReflection(
+            final Class<?> parentClass, final Map<String, DataSchemaNode> getterToLeafSchema) {
+        final Map<Method, ValueNodeCodecContext> leaves = new HashMap<>();
         for (final Method method : parentClass.getMethods()) {
-            if (method.getParameterTypes().length == 0) {
+            if (method.getParameterCount() == 0) {
                 final DataSchemaNode schema = getterToLeafSchema.get(method.getName());
-                final Class<?> valueType;
+
+                final ValueNodeCodecContext valueNode;
                 if (schema instanceof LeafSchemaNode) {
-                    valueType = method.getReturnType();
+                    final LeafSchemaNode leafSchema = (LeafSchemaNode) schema;
+
+                    final Class<?> valueType = method.getReturnType();
+                    final IllegalArgumentCodec<Object, Object> codec = getCodec(valueType, leafSchema.getType());
+                    valueNode = LeafNodeCodecContext.of(leafSchema, codec, method.getName(), valueType,
+                        context.getSchemaContext());
                 } else if (schema instanceof LeafListSchemaNode) {
-                    final Type genericType = ClassLoaderUtils.getFirstGenericParameter(method.getGenericReturnType());
+                    final Optional<Type> optType = ClassLoaderUtils.getFirstGenericParameter(
+                        method.getGenericReturnType());
+                    checkState(optType.isPresent(), "Failed to find return type for %s", method);
 
+                    final Class<?> valueType;
+                    final Type genericType = optType.get();
                     if (genericType instanceof Class<?>) {
                         valueType = (Class<?>) genericType;
                     } else if (genericType instanceof ParameterizedType) {
@@ -268,38 +336,38 @@ final class BindingCodecContext implements CodecContextFactory, BindingCodecTree
                     } else {
                         throw new IllegalStateException("Unexpected return type " + genericType);
                     }
+
+                    final LeafListSchemaNode leafListSchema = (LeafListSchemaNode) schema;
+                    final IllegalArgumentCodec<Object, Object> codec = getCodec(valueType, leafListSchema.getType());
+                    valueNode = new LeafSetNodeCodecContext(leafListSchema, codec, method.getName());
+                } else if (schema instanceof AnyxmlSchemaNode) {
+                    valueNode = new OpaqueNodeCodecContext.Anyxml<>((AnyxmlSchemaNode) schema, method.getName(),
+                            opaqueReturnType(method), loader);
+                } else if (schema instanceof AnydataSchemaNode) {
+                    valueNode = new OpaqueNodeCodecContext.Anydata<>((AnydataSchemaNode) schema, method.getName(),
+                            opaqueReturnType(method), loader);
                 } else {
-                    // We do not have schema for leaf, so we will ignore it (eg. getClass, getImplementedInterface).
+                    verify(schema == null, "Unhandled schema %s for method %s", schema, method);
+                    // We do not have schema for leaf, so we will ignore it (e.g. getClass).
                     continue;
                 }
-                final Codec<Object, Object> codec = getCodec(valueType, schema);
-                final LeafNodeCodecContext<?> leafNode = new LeafNodeCodecContext<>(schema, codec, method,
-                        context.getSchemaContext());
-                leaves.put(schema.getQName().getLocalName(), leafNode);
+
+                leaves.put(method, valueNode);
             }
         }
         return ImmutableMap.copyOf(leaves);
     }
 
-    private Codec<Object, Object> getCodec(final Class<?> valueType, final DataSchemaNode schema) {
-        Preconditions.checkArgument(schema instanceof TypedDataSchemaNode, "Unsupported leaf node type %s", schema);
-
-        return getCodec(valueType, ((TypedDataSchemaNode)schema).getType());
-    }
-
-    Codec<Object, Object> getCodec(final Class<?> valueType, final TypeDefinition<?> instantiatedType) {
+    // FIXME: this is probably not right w.r.t. nulls
+    IllegalArgumentCodec<Object, Object> getCodec(final Class<?> valueType, final TypeDefinition<?> instantiatedType) {
         if (Class.class.equals(valueType)) {
             @SuppressWarnings({ "unchecked", "rawtypes" })
-            final Codec<Object, Object> casted = (Codec) identityCodec;
+            final IllegalArgumentCodec<Object, Object> casted = (IllegalArgumentCodec) identityCodec;
             return casted;
         } else if (InstanceIdentifier.class.equals(valueType)) {
             @SuppressWarnings({ "unchecked", "rawtypes" })
-            final Codec<Object, Object> casted = (Codec) instanceIdentifierCodec;
+            final IllegalArgumentCodec<Object, Object> casted = (IllegalArgumentCodec) instanceIdentifierCodec;
             return casted;
-        } else if (Boolean.class.equals(valueType)) {
-            if (instantiatedType instanceof EmptyTypeDefinition) {
-                return ValueTypeCodec.EMPTY_CODEC;
-            }
         } else if (BindingReflections.isBindingClass(valueType)) {
             return getCodecForBindingClass(valueType, instantiatedType);
         }
@@ -307,55 +375,66 @@ final class BindingCodecContext implements CodecContextFactory, BindingCodecTree
     }
 
     @SuppressWarnings("checkstyle:illegalCatch")
-    private Codec<Object, Object> getCodecForBindingClass(final Class<?> valueType, final TypeDefinition<?> typeDef) {
+    // FIXME: this is probably not right w.r.t. nulls
+    private IllegalArgumentCodec<Object, Object> getCodecForBindingClass(final Class<?> valueType,
+            final TypeDefinition<?> typeDef) {
         if (typeDef instanceof IdentityrefTypeDefinition) {
             return ValueTypeCodec.encapsulatedValueCodecFor(valueType, typeDef, identityCodec);
         } else if (typeDef instanceof InstanceIdentifierTypeDefinition) {
             return ValueTypeCodec.encapsulatedValueCodecFor(valueType, typeDef, instanceIdentifierCodec);
         } else if (typeDef instanceof UnionTypeDefinition) {
-            final Callable<UnionTypeCodec> loader = UnionTypeCodec.loader(valueType, (UnionTypeDefinition) typeDef,
+            final Callable<UnionTypeCodec> unionLoader = UnionTypeCodec.loader(valueType, (UnionTypeDefinition) typeDef,
                 this);
             try {
-                return loader.call();
+                return unionLoader.call();
             } catch (final Exception e) {
                 throw new IllegalStateException("Unable to load codec for " + valueType, e);
             }
         } else if (typeDef instanceof LeafrefTypeDefinition) {
             final Entry<GeneratedType, WithStatus> typeWithSchema = context.getTypeWithSchema(valueType);
             final WithStatus schema = typeWithSchema.getValue();
-            Preconditions.checkState(schema instanceof TypeDefinition<?>);
+            checkState(schema instanceof TypeDefinition);
             return getCodec(valueType, (TypeDefinition<?>) schema);
         }
         return ValueTypeCodec.getCodecFor(valueType, typeDef);
     }
 
     @Override
-    public Codec<NodeIdentifierWithPredicates, IdentifiableItem<?, ?>> getPathArgumentCodec(final Class<?> listClz,
-            final ListSchemaNode schema) {
-        final Class<? extends Identifier<?>> identifier = ClassLoaderUtils.findFirstGenericArgument(listClz,
+    public IdentifiableItemCodec getPathArgumentCodec(final Class<?> listClz, final ListSchemaNode schema) {
+        final Optional<Class<Identifier<?>>> optIdentifier = ClassLoaderUtils.findFirstGenericArgument(listClz,
                 Identifiable.class);
+        checkState(optIdentifier.isPresent(), "Failed to find identifier for %s", listClz);
+
+        final Class<Identifier<?>> identifier = optIdentifier.get();
         final Map<QName, ValueContext> valueCtx = new HashMap<>();
-        for (final LeafNodeCodecContext<?> leaf : getLeafNodes(identifier, schema).values()) {
+        for (final ValueNodeCodecContext leaf : getLeafNodes(identifier, schema).values()) {
             final QName name = leaf.getDomPathArgument().getNodeType();
             valueCtx.put(name, new ValueContext(identifier, leaf));
         }
-        return new IdentifiableItemCodec(schema, identifier, listClz, valueCtx);
+        return IdentifiableItemCodec.of(schema, identifier, listClz, valueCtx);
     }
 
     @SuppressWarnings("unchecked")
     @Override
-    public <T extends DataObject> BindingCodecTreeNode<T> getSubtreeCodec(final InstanceIdentifier<T> path) {
+    public <T extends DataObject> BindingDataObjectCodecTreeNode<T> getSubtreeCodec(final InstanceIdentifier<T> path) {
         // TODO Do we need defensive check here?
-        return (BindingCodecTreeNode<T>) getCodecContextNode(path, null);
+        return (BindingDataObjectCodecTreeNode<T>) getCodecContextNode(path, null);
     }
 
     @Override
-    public BindingCodecTreeNode<?> getSubtreeCodec(final YangInstanceIdentifier path) {
+    public BindingCodecTreeNode getSubtreeCodec(final YangInstanceIdentifier path) {
         return getCodecContextNode(path, null);
     }
 
     @Override
-    public BindingCodecTreeNode<?> getSubtreeCodec(final SchemaPath path) {
+    public BindingCodecTreeNode getSubtreeCodec(final Absolute path) {
         throw new UnsupportedOperationException("Not implemented yet.");
     }
+
+    @SuppressWarnings("rawtypes")
+    private static Class<? extends OpaqueObject> opaqueReturnType(final Method method) {
+        final Class<?> valueType = method.getReturnType();
+        verify(OpaqueObject.class.isAssignableFrom(valueType), "Illegal value type %s", valueType);
+        return valueType.asSubclass(OpaqueObject.class);
+    }
 }