BUG-865: removed unused utility methods from SchemaContextUtil
[yangtools.git] / code-generator / binding-generator-impl / src / main / java / org / opendaylight / yangtools / sal / binding / generator / impl / RuntimeGeneratedMappingServiceImpl.java
index d2fbe8b7e90b41ae437aa4fd026e68426d86471b..e475a6164336e03c2b3f09c6b0eca3117dbe004b 100644 (file)
@@ -7,38 +7,29 @@
  */
 package org.opendaylight.yangtools.sal.binding.generator.impl;
 
-import com.google.common.base.Objects;
-import com.google.common.base.Optional;
-import com.google.common.collect.HashMultimap;
-import com.google.common.util.concurrent.SettableFuture;
-
+import java.net.URI;
 import java.util.AbstractMap.SimpleEntry;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Future;
+import java.util.concurrent.ExecutionException;
 
 import javassist.ClassPool;
 
-import org.eclipse.xtext.xbase.lib.Exceptions;
-import org.eclipse.xtext.xbase.lib.Extension;
+import javax.annotation.concurrent.GuardedBy;
+
 import org.opendaylight.yangtools.binding.generator.util.BindingGeneratorUtil;
 import org.opendaylight.yangtools.binding.generator.util.ReferencedTypeImpl;
 import org.opendaylight.yangtools.binding.generator.util.Types;
 import org.opendaylight.yangtools.sal.binding.generator.api.ClassLoadingStrategy;
-import org.opendaylight.yangtools.sal.binding.generator.impl.BindingGeneratorImpl;
-import org.opendaylight.yangtools.sal.binding.generator.impl.GeneratedClassLoadingStrategy;
-import org.opendaylight.yangtools.sal.binding.generator.impl.LazyGeneratedCodecRegistry;
-import org.opendaylight.yangtools.sal.binding.generator.impl.ModuleContext;
-import org.opendaylight.yangtools.sal.binding.generator.impl.SchemaLock;
-import org.opendaylight.yangtools.sal.binding.generator.impl.TransformerGenerator;
 import org.opendaylight.yangtools.sal.binding.generator.util.YangSchemaUtils;
 import org.opendaylight.yangtools.sal.binding.model.api.Type;
 import org.opendaylight.yangtools.sal.binding.model.api.type.builder.GeneratedTypeBuilder;
@@ -46,6 +37,8 @@ import org.opendaylight.yangtools.yang.binding.Augmentation;
 import org.opendaylight.yangtools.yang.binding.BindingMapping;
 import org.opendaylight.yangtools.yang.binding.DataContainer;
 import org.opendaylight.yangtools.yang.binding.DataObject;
+import org.opendaylight.yangtools.yang.binding.InstanceIdentifier.IdentifiableItem;
+import org.opendaylight.yangtools.yang.binding.InstanceIdentifier.Item;
 import org.opendaylight.yangtools.yang.binding.RpcService;
 import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.data.api.CompositeNode;
@@ -60,7 +53,9 @@ import org.opendaylight.yangtools.yang.data.impl.codec.BindingIndependentMapping
 import org.opendaylight.yangtools.yang.data.impl.codec.CodecRegistry;
 import org.opendaylight.yangtools.yang.data.impl.codec.DataContainerCodec;
 import org.opendaylight.yangtools.yang.data.impl.codec.DeserializationException;
+import org.opendaylight.yangtools.yang.data.impl.codec.InstanceIdentifierCodec;
 import org.opendaylight.yangtools.yang.data.impl.codec.ValueWithQName;
+import org.opendaylight.yangtools.yang.model.api.AugmentationSchema;
 import org.opendaylight.yangtools.yang.model.api.Module;
 import org.opendaylight.yangtools.yang.model.api.RpcDefinition;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
@@ -73,100 +68,83 @@ import org.opendaylight.yangtools.yang.model.util.SchemaContextUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+
 public class RuntimeGeneratedMappingServiceImpl implements BindingIndependentMappingService, SchemaContextListener,
-        SchemaLock, AutoCloseable, SchemaContextHolder {
+        SchemaLock, AutoCloseable, SchemaContextHolder, TypeResolver {
 
     private static final Logger LOG = LoggerFactory.getLogger(RuntimeGeneratedMappingServiceImpl.class);
 
-    private ClassPool pool;
+    private final ConcurrentMap<Type, Set<QName>> serviceTypeToRpc = new ConcurrentHashMap<>();
 
-    @Extension
-    private TransformerGenerator binding;
+    /**
+     * This is map of types which users are waiting for.
+     */
+    @GuardedBy("this")
+    private final Multimap<Type, SettableFuture<Type>> promisedTypes = HashMultimap.create();
 
-    @Extension
-    private LazyGeneratedCodecRegistry registry;
+    private final ClassLoadingStrategy classLoadingStrategy;
 
-    private final ConcurrentMap<Type, Type> typeDefinitions = new ConcurrentHashMap<>();
+    private final AbstractTransformerGenerator binding;
+    private final LazyGeneratedCodecRegistry registry;
+    private final ClassPool pool;
+
+    /*
+     * FIXME: updated here, access from AbstractTransformer
+     */
+    private final Map<Type, AugmentationSchema> typeToAugmentation = new ConcurrentHashMap<>();
     private final ConcurrentMap<Type, GeneratedTypeBuilder> typeToDefinition = new ConcurrentHashMap<>();
     private final ConcurrentMap<Type, SchemaNode> typeToSchemaNode = new ConcurrentHashMap<>();
-    private final ConcurrentMap<Type, Set<QName>> serviceTypeToRpc = new ConcurrentHashMap<>();
-    private final HashMultimap<Type, SettableFuture<Type>> promisedTypes = HashMultimap.create();
-    private final ClassLoadingStrategy classLoadingStrategy;
-    private SchemaContext schemaContext;
+    private final Map<SchemaPath, GeneratedTypeBuilder> pathToType = new ConcurrentHashMap<>();
 
-    public RuntimeGeneratedMappingServiceImpl() {
-        this(GeneratedClassLoadingStrategy.getTCCLClassLoadingStrategy());
-    }
+    // FIXME: need to figure these out
+    private final ConcurrentMap<Type, Type> typeDefinitions = new ConcurrentHashMap<>();
+    private SchemaContext schemaContext;
 
-    public RuntimeGeneratedMappingServiceImpl(ClassLoadingStrategy strat) {
-        classLoadingStrategy = strat;
+    public RuntimeGeneratedMappingServiceImpl(final ClassPool pool) {
+        this(pool, GeneratedClassLoadingStrategy.getTCCLClassLoadingStrategy());
     }
 
-    public ClassPool getPool() {
-        return this.pool;
-    }
+    public RuntimeGeneratedMappingServiceImpl(final ClassPool pool, final ClassLoadingStrategy strat) {
+        this.pool = Preconditions.checkNotNull(pool);
+        this.classLoadingStrategy = Preconditions.checkNotNull(strat);
+        // FIXME: this escapes constructor
+        binding = new TransformerGenerator(this, pool);
+        registry = new LazyGeneratedCodecRegistry(this, binding, classLoadingStrategy);
+        binding.setListener(registry);
 
-    public void setPool(final ClassPool pool) {
-        this.pool = pool;
+        // if (ctx !== null) {
+        // listenerRegistration = ctx.registerService(SchemaServiceListener,
+        // this, new Hashtable<String, String>());
+        // }
     }
 
     @Override
-    public SchemaContext getSchemaContext() {
+    public synchronized SchemaContext getSchemaContext() {
         return schemaContext;
     }
 
-    public void setSchemaContext(final SchemaContext schemaContext) {
-        this.schemaContext = schemaContext;
-    }
-
-    public TransformerGenerator getBinding() {
-        return this.binding;
-    }
-
-    public void setBinding(final TransformerGenerator binding) {
-        this.binding = binding;
-    }
-
-    public LazyGeneratedCodecRegistry getRegistry() {
-        return registry;
-    }
-
-    public void setRegistry(final LazyGeneratedCodecRegistry registry) {
-        this.registry = registry;
-    }
-
-    public ConcurrentMap<Type, GeneratedTypeBuilder> getTypeToDefinition() {
-        return typeToDefinition;
-    }
-
-    public ConcurrentMap<Type, Type> getTypeDefinitions() {
-        return typeDefinitions;
-    }
-
-    public ConcurrentMap<Type, SchemaNode> getTypeToSchemaNode() {
-        return typeToSchemaNode;
-    }
-
-    public ConcurrentMap<Type, Set<QName>> getServiceTypeToRpc() {
-        return serviceTypeToRpc;
-    }
-
     @Override
-    public void onGlobalContextUpdated(final SchemaContext arg0) {
-        this.setSchemaContext(arg0);
-        this.recreateBindingContext(arg0);
-        LazyGeneratedCodecRegistry _registry = this.getRegistry();
-        _registry.onGlobalContextUpdated(arg0);
+    public synchronized void onGlobalContextUpdated(final SchemaContext context) {
+        this.schemaContext = Preconditions.checkNotNull(context);
+        this.recreateBindingContext(context);
+        this.registry.onGlobalContextUpdated(context);
     }
 
-    private void recreateBindingContext(SchemaContext schemaContext) {
+    @GuardedBy("this")
+    private void recreateBindingContext(final SchemaContext schemaContext) {
         BindingGeneratorImpl newBinding = new BindingGeneratorImpl();
         newBinding.generateTypes(schemaContext);
 
         for (Map.Entry<Module, ModuleContext> entry : newBinding.getModuleContexts().entrySet()) {
 
             registry.onModuleContextAdded(schemaContext, entry.getKey(), entry.getValue());
-            binding.getPathToType().putAll(entry.getValue().getChildNodes());
+            pathToType.putAll(entry.getValue().getChildNodes());
             Module module = entry.getKey();
             ModuleContext context = entry.getValue();
             updateBindingFor(context.getChildNodes(), schemaContext);
@@ -187,26 +165,27 @@ public class RuntimeGeneratedMappingServiceImpl implements BindingIndependentMap
             for (Map.Entry<SchemaPath, Type> typedef : typedefs.entrySet()) {
                 Type value = typedef.getValue();
                 Type typeRef = new ReferencedTypeImpl(value.getPackageName(), value.getName());
-                binding.getTypeDefinitions().put(typeRef, value);
+                typeDefinitions.put(typeRef, value);
                 TypeDefinition<?> schemaNode = YangSchemaUtils.findTypeDefinition(schemaContext, typedef.getKey());
                 if (schemaNode != null) {
 
-                    binding.getTypeToSchemaNode().put(typeRef, schemaNode);
+                    typeToSchemaNode.put(typeRef, schemaNode);
                 } else {
                     LOG.error("Type definition for {} is not available", value);
                 }
             }
             List<GeneratedTypeBuilder> augmentations = context.getAugmentations();
             for (GeneratedTypeBuilder augmentation : augmentations) {
-                binding.getTypeToDefinition().put(augmentation, augmentation);
+                typeToDefinition.put(augmentation, augmentation);
             }
-            binding.getTypeToAugmentation().putAll(context.getTypeToAugmentation());
+            typeToAugmentation.putAll(context.getTypeToAugmentation());
             for (GeneratedTypeBuilder augmentation : augmentations) {
                 updatePromisedSchemas(augmentation);
             }
         }
     }
 
+    @Override
     public CompositeNode toDataDom(final DataObject data) {
         return toCompositeNodeImpl(data);
     }
@@ -231,15 +210,15 @@ public class RuntimeGeneratedMappingServiceImpl implements BindingIndependentMap
         }
     }
 
-    private CompositeNode toCompositeNodeImpl(DataObject object) {
+    private CompositeNode toCompositeNodeImpl(final DataObject object) {
         Class<? extends DataContainer> cls = object.getImplementedInterface();
         waitForSchema(cls);
         DataContainerCodec<DataObject> codec = (DataContainerCodec<DataObject>) registry.getCodecForDataObject(cls);
         return codec.serialize(new ValueWithQName<DataObject>(null, object));
     }
 
-    private CompositeNode toCompositeNodeImpl(org.opendaylight.yangtools.yang.data.api.InstanceIdentifier identifier,
-            DataObject object) {
+    private CompositeNode toCompositeNodeImpl(final org.opendaylight.yangtools.yang.data.api.InstanceIdentifier identifier,
+            final DataObject object) {
         PathArgument last = identifier.getPath().get(identifier.getPath().size() - 1);
         Class<? extends DataContainer> cls = object.getImplementedInterface();
         waitForSchema(cls);
@@ -248,7 +227,7 @@ public class RuntimeGeneratedMappingServiceImpl implements BindingIndependentMap
     }
 
     private CompositeNode toCompositeNodeImplAugument(
-            org.opendaylight.yangtools.yang.data.api.InstanceIdentifier identifier, DataObject object) {
+            final org.opendaylight.yangtools.yang.data.api.InstanceIdentifier identifier, final DataObject object) {
 
         // val cls = object.implementedInterface;
         // waitForSchema(cls);
@@ -262,66 +241,84 @@ public class RuntimeGeneratedMappingServiceImpl implements BindingIndependentMap
             for (Map.Entry<QName, Object> predicate : predicates.getKeyValues().entrySet()) {
                 newNodes.add(new SimpleNodeTOImpl<Object>(predicate.getKey(), null, predicate.getValue()));
             }
-            newNodes.addAll(ret.getChildren());
+            newNodes.addAll(ret.getValue());
             return new CompositeNodeTOImpl(last.getNodeType(), null, newNodes);
         }
         return ret;
     }
 
     @Override
-    public void waitForSchema(Class class1) {
-        if (registry.isCodecAvailable(class1)) {
-            return;
+    public void waitForSchema(final Class<?> cls) {
+        final ListenableFuture<Type> f = getSchemaDefinition(cls);
+        if (f != null) {
+            try {
+                f.get();
+            } catch (InterruptedException | ExecutionException e) {
+                LOG.warn("Waiting for schema for class {} failed", cls, e);
+                throw new IllegalStateException(String.format("Failed to get schema for %s", cls), e);
+            }
+            LOG.info("Schema for {} became available, thread unblocked", cls);
         }
-        Type ref = Types.typeForClass(class1);
-        getSchemaWithRetry(ref);
     }
 
+    @Override
     public InstanceIdentifier toDataDom(
             final org.opendaylight.yangtools.yang.binding.InstanceIdentifier<? extends DataObject> path) {
-        for (final org.opendaylight.yangtools.yang.binding.InstanceIdentifier.PathArgument arg : path.getPath()) {
+        for (final org.opendaylight.yangtools.yang.binding.InstanceIdentifier.PathArgument arg : path.getPathArguments()) {
             this.waitForSchema(arg.getType());
         }
-        return registry.getInstanceIdentifierCodec().serialize(path);
+
+        final InstanceIdentifierCodec c = registry.getInstanceIdentifierCodec();
+        Preconditions.checkState(c != null, "InstanceIdentifierCodec not present");
+        return c.serialize(path);
     }
 
     @Override
     public DataObject dataObjectFromDataDom(
             final org.opendaylight.yangtools.yang.binding.InstanceIdentifier<? extends DataObject> path,
-            final CompositeNode node) {
-        return (DataObject) dataObjectFromDataDom(path.getTargetType(), node);
-    }
+            final CompositeNode domData) throws DeserializationException {
+        if (domData == null) {
+            return null;
+        }
 
-    @Override
-    public org.opendaylight.yangtools.yang.binding.InstanceIdentifier<? extends Object> fromDataDom(
-            final InstanceIdentifier entry) {
         try {
-            return tryDeserialization(new Callable<org.opendaylight.yangtools.yang.binding.InstanceIdentifier<? extends Object>>() {
-                @Override
-                public org.opendaylight.yangtools.yang.binding.InstanceIdentifier<? extends Object> call() {
-                    return getRegistry().getInstanceIdentifierCodec().deserialize(entry);
-                }
-            });
-        } catch (Throwable _e) {
-            throw Exceptions.sneakyThrow(_e);
+            final Class<? extends DataContainer> container = path.getTargetType();
+            // FIXME: deprecate use without iid
+            final org.opendaylight.yangtools.yang.binding.InstanceIdentifier<? extends DataObject> wildcardedPath = createWildcarded(path);
+
+            final DataContainerCodec<? extends DataContainer> transformer = registry.getCodecForDataObject(container);
+            Preconditions.checkState(transformer != null, "Failed to find codec for type %s", container);
+
+            final ValueWithQName<? extends DataContainer> deserialize = transformer.deserialize(domData, wildcardedPath);
+            if (deserialize == null) {
+                return null;
+            }
+
+            return (DataObject) deserialize.getValue();
+        } catch (Exception e) {
+            LOG.warn("Failed to deserialize path {} data {}", path, domData);
+            throw new DeserializationException("Data deserialization failed", e);
         }
     }
 
     @Override
-    public CodecRegistry getCodecRegistry() {
-        return this.getRegistry();
-    }
-
-    private static <T> T tryDeserialization(Callable<T> deserializationBlock) throws DeserializationException {
+    public org.opendaylight.yangtools.yang.binding.InstanceIdentifier<? extends Object> fromDataDom(final InstanceIdentifier entry) throws DeserializationException {
         try {
-            return deserializationBlock.call();
+            final InstanceIdentifierCodec c = registry.getInstanceIdentifierCodec();
+            Preconditions.checkState(c != null, "InstanceIdentifierCodec not present");
+            return c.deserialize(entry);
         } catch (Exception e) {
-            // FIXME: Make this block providing more information.
-            throw new DeserializationException(e);
+            LOG.warn("Failed to deserialize entry {}", entry);
+            throw new DeserializationException("Entry deserialization failed", e);
         }
     }
 
-    private void updateBindingFor(Map<SchemaPath, GeneratedTypeBuilder> map, SchemaContext module) {
+    @Override
+    public CodecRegistry getCodecRegistry() {
+        return this.registry;
+    }
+
+    private void updateBindingFor(final Map<SchemaPath, GeneratedTypeBuilder> map, final SchemaContext module) {
         for (Map.Entry<SchemaPath, GeneratedTypeBuilder> entry : map.entrySet()) {
             SchemaNode schemaNode = SchemaContextUtil.findDataSchemaNode(module, entry.getKey());
 
@@ -335,25 +332,9 @@ public class RuntimeGeneratedMappingServiceImpl implements BindingIndependentMap
         }
     }
 
-    public void init() {
-        binding = new TransformerGenerator(pool);
-        registry = new LazyGeneratedCodecRegistry(this, classLoadingStrategy);
-
-        registry.setGenerator(binding);
-        // binding.staticFieldsInitializer = registry
-        binding.setListener(registry);
-        binding.setTypeToDefinition(typeToDefinition);
-        binding.setTypeToSchemaNode(typeToSchemaNode);
-        binding.setTypeDefinitions(typeDefinitions);
-
-        // if (ctx !== null) {
-        // listenerRegistration = ctx.registerService(SchemaServiceListener,
-        // this, new Hashtable<String, String>());
-        // }
-    }
 
     @Override
-    public Set<QName> getRpcQNamesFor(Class<? extends RpcService> service) {
+    public Set<QName> getRpcQNamesFor(final Class<? extends RpcService> service) {
         Set<QName> serviceRef = serviceTypeToRpc.get(new ReferencedTypeImpl(service.getPackage().getName(), service
                 .getSimpleName()));
         if (serviceRef == null) {
@@ -362,71 +343,61 @@ public class RuntimeGeneratedMappingServiceImpl implements BindingIndependentMap
         return serviceRef;
     }
 
-    private void getSchemaWithRetry(Type type) {
-        try {
+    private ListenableFuture<Type> getSchemaDefinition(final Class<?> cls) {
+        final Type type = Types.typeForClass(cls);
+        synchronized (this) {
             if (typeToDefinition.containsKey(type)) {
-                return;
+                return null;
             }
-            LOG.info("Thread blocked waiting for schema for: {}", type.getFullyQualifiedName());
-            waitForTypeDefinition(type).get();
-            LOG.info("Schema for {} became available, thread unblocked", type.getFullyQualifiedName());
-        } catch (Throwable t) {
-            Exceptions.sneakyThrow(t);
+
+            LOG.info("Thread is going to wait for schema for: {}", type.getFullyQualifiedName());
+            final SettableFuture<Type> f = SettableFuture.create();
+            promisedTypes.put(type, f);
+            return f;
         }
     }
 
-    private Future<Type> waitForTypeDefinition(final Type type) {
-        final SettableFuture<Type> future = SettableFuture.<Type> create();
-        promisedTypes.put(type, future);
-        return future;
-    }
+    @GuardedBy("this")
+    private void updatePromisedSchemas(final Type builder) {
+        final Type ref = new ReferencedTypeImpl(builder.getPackageName(), builder.getName());
+        final Collection<SettableFuture<Type>> futures = promisedTypes.get(ref);
 
-    private void updatePromisedSchemas(Type builder) {
-        Type ref = new ReferencedTypeImpl(builder.getPackageName(), builder.getName());
-        Set<SettableFuture<Type>> futures = promisedTypes.get(ref);
-        if (futures == null || futures.isEmpty()) {
-            return;
-        }
-        for (SettableFuture<Type> future : futures) {
-            future.set(builder);
+        if (futures != null) {
+            for (SettableFuture<Type> future : futures) {
+                future.set(builder);
+            }
+            promisedTypes.removeAll(builder);
         }
-        promisedTypes.removeAll(builder);
     }
 
     @Override
-    public void close() throws Exception {
+    public void close() {
+        // Nothing to do
     }
 
     @Override
     public DataContainer dataObjectFromDataDom(final Class<? extends DataContainer> container,
             final CompositeNode domData) {
+        // FIXME: Add check for valids inputs
+        // which are Notification and Rpc Input / Rpc Output
+
+        org.opendaylight.yangtools.yang.binding.InstanceIdentifier<? extends DataContainer> id = org.opendaylight.yangtools.yang.binding.InstanceIdentifier
+                .create((Class) container);
+        Preconditions.checkNotNull(id, "Failed to create path for type %s", container);
+
         try {
-            return tryDeserialization(new Callable<DataObject>() {
-                @Override
-                public DataObject call() throws Exception {
-                    if (Objects.equal(domData, null)) {
-                        return null;
-                    }
-                    final DataContainerCodec<? extends DataContainer> transformer = getRegistry()
-                            .getCodecForDataObject(container);
-                    ValueWithQName<? extends DataContainer> deserialize = transformer.deserialize(domData);
-                    DataContainer value = null;
-                    if (deserialize != null) {
-                        value = deserialize.getValue();
-                    }
-                    return ((DataObject) value);
-                }
-            });
-        } catch (Throwable _e) {
-            throw Exceptions.sneakyThrow(_e);
+            return dataObjectFromDataDom(id, domData);
+        } catch (DeserializationException e) {
+            LOG.warn("Conversion of class {} path {} data {} failed", container, id, domData, e);
+            throw new IllegalStateException("Failed to create data object", e);
         }
     }
 
     @Override
-    public Optional<Class<? extends RpcService>> getRpcServiceClassFor(String namespace, String revision) {
+    public synchronized Optional<Class<? extends RpcService>> getRpcServiceClassFor(final String namespace, final String revision) {
         Module module = null;
         if (schemaContext != null) {
-            module = schemaContext.findModuleByName(namespace, QName.parseRevision(revision));
+            module = schemaContext.findModuleByNamespaceAndRevision(URI.create(namespace), QName.parseRevision(revision));
         }
         if (module == null) {
             return Optional.absent();
@@ -434,8 +405,7 @@ public class RuntimeGeneratedMappingServiceImpl implements BindingIndependentMap
         try {
             Optional<Type> rpcTypeName = getRpcServiceType(module);
             if (rpcTypeName.isPresent()) {
-                Class<?> rpcClass = binding.getClassLoadingStrategy().loadClass(
-                        rpcTypeName.get().getFullyQualifiedName());
+                Class<?> rpcClass = classLoadingStrategy.loadClass(rpcTypeName.get().getFullyQualifiedName());
                 return Optional.<Class<? extends RpcService>> of((Class<? extends RpcService>) rpcClass);
             }
         } catch (Exception e) {
@@ -444,7 +414,7 @@ public class RuntimeGeneratedMappingServiceImpl implements BindingIndependentMap
         return Optional.absent();
     }
 
-    public Optional<Type> getRpcServiceType(Module module) {
+    public Optional<Type> getRpcServiceType(final Module module) {
         String namespace = BindingGeneratorUtil.moduleNamespaceToPackageName(module);
         if (module.getRpcs().isEmpty()) {
             return Optional.<Type> absent();
@@ -453,4 +423,37 @@ public class RuntimeGeneratedMappingServiceImpl implements BindingIndependentMap
                 + BindingMapping.RPC_SERVICE_SUFFIX));
     }
 
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    private static final org.opendaylight.yangtools.yang.binding.InstanceIdentifier<? extends DataObject> createWildcarded(
+            final org.opendaylight.yangtools.yang.binding.InstanceIdentifier<? extends DataObject> path) {
+
+        LinkedList<org.opendaylight.yangtools.yang.binding.InstanceIdentifier.PathArgument> wildcardedArgs = new LinkedList<>();
+        for(org.opendaylight.yangtools.yang.binding.InstanceIdentifier.PathArgument pathArg : path.getPathArguments()) {
+            if(pathArg instanceof IdentifiableItem<?,?>) {
+                pathArg = new Item(pathArg.getType());
+            }
+            wildcardedArgs.add(pathArg);
+        }
+        return org.opendaylight.yangtools.yang.binding.InstanceIdentifier.create(wildcardedArgs);
+    }
+
+    @Override
+    public final AugmentationSchema getAugmentation(final Type type) {
+        return typeToAugmentation.get(type);
+    }
+
+    @Override
+    public final GeneratedTypeBuilder getDefinition(final Type type) {
+        return typeToDefinition.get(type);
+    }
+
+    @Override
+    public final SchemaNode getSchemaNode(final Type type) {
+        return typeToSchemaNode.get(type);
+    }
+
+    @Override
+    public final GeneratedTypeBuilder getTypeBuilder(final SchemaPath path) {
+        return pathToType.get(path);
+    }
 }