Split out BindingDataObjectCodecTreeNode.streamChild()
[mdsal.git] / binding / mdsal-binding-dom-codec / src / main / java / org / opendaylight / mdsal / binding / dom / codec / impl / SchemaRootCodecContext.java
index b342d0e214cb4957a194effb0197b9ed353b44af..bc1b2d4b8d53a3548b273a2886cd24c6d06d7876 100644 (file)
  */
 package org.opendaylight.mdsal.binding.dom.codec.impl;
 
-import com.google.common.base.Optional;
-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.base.Throwables;
 import com.google.common.base.Verify;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Type;
 import java.util.List;
-import org.opendaylight.yangtools.yang.binding.BindingMapping;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import org.eclipse.jdt.annotation.NonNull;
+import org.eclipse.jdt.annotation.Nullable;
+import org.opendaylight.mdsal.binding.dom.codec.api.IncorrectNestingException;
+import org.opendaylight.mdsal.binding.model.api.JavaTypeName;
+import org.opendaylight.mdsal.binding.runtime.api.ActionRuntimeType;
+import org.opendaylight.mdsal.binding.runtime.api.BindingRuntimeContext;
+import org.opendaylight.mdsal.binding.runtime.api.BindingRuntimeTypes;
+import org.opendaylight.mdsal.binding.runtime.api.ChoiceRuntimeType;
+import org.opendaylight.mdsal.binding.runtime.api.CompositeRuntimeType;
+import org.opendaylight.mdsal.binding.runtime.api.ContainerLikeRuntimeType;
+import org.opendaylight.mdsal.binding.runtime.api.DataRuntimeType;
+import org.opendaylight.mdsal.binding.runtime.api.NotificationRuntimeType;
+import org.opendaylight.mdsal.binding.runtime.api.RuntimeType;
+import org.opendaylight.mdsal.binding.spec.naming.BindingMapping;
+import org.opendaylight.mdsal.binding.spec.reflect.BindingReflections;
+import org.opendaylight.yangtools.util.ClassLoaderUtils;
+import org.opendaylight.yangtools.yang.binding.Action;
 import org.opendaylight.yangtools.yang.binding.ChoiceIn;
 import org.opendaylight.yangtools.yang.binding.DataContainer;
 import org.opendaylight.yangtools.yang.binding.DataObject;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
+import org.opendaylight.yangtools.yang.binding.KeyedListAction;
 import org.opendaylight.yangtools.yang.binding.Notification;
-import org.opendaylight.yangtools.yang.binding.util.BindingReflections;
+import org.opendaylight.yangtools.yang.binding.RpcInput;
+import org.opendaylight.yangtools.yang.binding.RpcOutput;
 import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.common.QNameModule;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.model.api.ChoiceSchemaNode;
-import org.opendaylight.yangtools.yang.model.api.ContainerSchemaNode;
+import org.opendaylight.yangtools.yang.model.api.ContainerLike;
 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.Module;
-import org.opendaylight.yangtools.yang.model.api.NotificationDefinition;
 import org.opendaylight.yangtools.yang.model.api.RpcDefinition;
-import org.opendaylight.yangtools.yang.model.api.SchemaContext;
-import org.opendaylight.yangtools.yang.model.api.SchemaPath;
-import org.opendaylight.yangtools.yang.model.util.SchemaContextUtil;
-import org.opendaylight.yangtools.yang.model.util.SchemaNodeUtils;
-
-final class SchemaRootCodecContext<D extends DataObject> extends DataContainerCodecContext<D,SchemaContext> {
-
-    private final LoadingCache<Class<?>, DataContainerCodecContext<?,?>> childrenByClass = CacheBuilder.newBuilder()
-            .build(new CacheLoader<Class<?>, DataContainerCodecContext<?,?>>() {
-                @Override
-                public DataContainerCodecContext<?,?> load(final Class<?> key) {
-                    return createDataTreeChildContext(key);
-                }
-            });
+import org.opendaylight.yangtools.yang.model.api.stmt.SchemaNodeIdentifier.Absolute;
 
-    private final LoadingCache<Class<?>, ContainerNodeCodecContext<?>> rpcDataByClass = CacheBuilder.newBuilder().build(
-            new CacheLoader<Class<?>, ContainerNodeCodecContext<?>>() {
-                @Override
-                public ContainerNodeCodecContext<?> load(final Class<?> key) {
-                    return createRpcDataContext(key);
-                }
-            });
+final class SchemaRootCodecContext<D extends DataObject> extends DataContainerCodecContext<D, BindingRuntimeTypes> {
 
-    private final LoadingCache<Class<?>, NotificationCodecContext<?>> notificationsByClass = CacheBuilder.newBuilder()
-            .build(new CacheLoader<Class<?>, NotificationCodecContext<?>>() {
-                @Override
-                public NotificationCodecContext<?> load(final Class<?> key) {
-                    return createNotificationDataContext(key);
-                }
-            });
+    private final LoadingCache<Class<? extends DataObject>, DataContainerCodecContext<?, ?>> childrenByClass =
+        CacheBuilder.newBuilder().build(new CacheLoader<>() {
+            @Override
+            public DataContainerCodecContext<?, ?> load(final Class<? extends DataObject> key) {
+                return createDataTreeChildContext(key);
+            }
+        });
+
+    private final LoadingCache<Class<? extends Action<?, ?, ?>>, ActionCodecContext> actionsByClass =
+        CacheBuilder.newBuilder().build(new CacheLoader<>() {
+            @Override
+            public ActionCodecContext load(final Class<? extends Action<?, ?, ?>> key) {
+                return createActionContext(key);
+            }
+        });
 
     private final LoadingCache<Class<? extends DataObject>, ChoiceNodeCodecContext<?>> choicesByClass =
-            CacheBuilder.newBuilder().build(new CacheLoader<Class<? extends DataObject>, ChoiceNodeCodecContext<?>>() {
-                @Override
-                public ChoiceNodeCodecContext<?> load(final Class<? extends DataObject> key) {
-                    return createChoiceDataContext(key);
+        CacheBuilder.newBuilder().build(new CacheLoader<>() {
+            @Override
+            public ChoiceNodeCodecContext<?> load(final Class<? extends DataObject> key) {
+                return createChoiceDataContext(key);
+            }
+        });
+
+    private final LoadingCache<Class<?>, NotificationCodecContext<?>> notificationsByClass = CacheBuilder.newBuilder()
+        .build(new CacheLoader<Class<?>, NotificationCodecContext<?>>() {
+            @Override
+            public NotificationCodecContext<?> load(final Class<?> key) {
+                checkArgument(key.isInterface(), "Supplied class must be interface.");
+
+                // TODO: we should be able to work with bindingChild() instead of schemaTreeChild() here
+                final QName qname = BindingReflections.findQName(key);
+                final RuntimeType child = getType().schemaTreeChild(qname);
+                checkArgument(child instanceof NotificationRuntimeType, "Supplied %s is not valid notification",
+                    key);
+                return new NotificationCodecContext<>(key, (NotificationRuntimeType) child, factory());
+            }
+        });
+
+    private final LoadingCache<Class<?>, ContainerNodeCodecContext<?>> rpcDataByClass = CacheBuilder.newBuilder()
+        .build(new CacheLoader<Class<?>, ContainerNodeCodecContext<?>>() {
+            @Override
+            public ContainerNodeCodecContext<?> load(final Class<?> key) {
+                final BiFunction<BindingRuntimeTypes, QName, Optional<? extends ContainerLikeRuntimeType<?, ?>>> lookup;
+                if (RpcInput.class.isAssignableFrom(key)) {
+                    lookup = BindingRuntimeTypes::findRpcInput;
+                } else if (RpcOutput.class.isAssignableFrom(key)) {
+                    lookup = BindingRuntimeTypes::findRpcOutput;
+                } else {
+                    throw new IllegalArgumentException(key + " does not represent an RPC container");
                 }
-            });
-
-    private final LoadingCache<QName, DataContainerCodecContext<?,?>> childrenByQName = CacheBuilder.newBuilder().build(
-            new CacheLoader<QName, DataContainerCodecContext<?,?>>() {
-                @Override
-                public DataContainerCodecContext<?,?> load(final QName qname) {
-                    final DataSchemaNode childSchema = getSchema().getDataChildByName(qname);
-                    childNonNull(childSchema, qname,"Argument %s is not valid child of %s", qname,getSchema());
-                    if (childSchema instanceof DataNodeContainer || childSchema instanceof ChoiceSchemaNode) {
-                        @SuppressWarnings("unchecked")
-                        final Class<? extends DataObject> childCls = (Class<? extends DataObject>)
-                                factory().getRuntimeContext().getClassForSchema(childSchema);
-                        return streamChild(childCls);
-                    }
 
-                    throw new UnsupportedOperationException("Unsupported child type " + childSchema.getClass());
+                final CodecContextFactory factory = factory();
+                final BindingRuntimeContext context = factory.getRuntimeContext();
+
+                final QName qname = BindingReflections.findQName(key);
+                final QNameModule qnameModule = qname.getModule();
+                final Module module = context.getEffectiveModelContext().findModule(qnameModule)
+                    .orElseThrow(() -> new IllegalArgumentException("Failed to find module for " + qnameModule));
+                final String className = BindingMapping.getClassName(qname);
+
+                for (final RpcDefinition potential : module.getRpcs()) {
+                    final QName potentialQName = potential.getQName();
+                    /*
+                     * Check if rpc and class represents data from same module and then checks if rpc local name
+                     * produces same class name as class name appended with Input/Output based on QName associated
+                     * with binding class.
+                     *
+                     * FIXME: Rework this to have more precise logic regarding Binding Specification.
+                     */
+                    if (key.getSimpleName().equals(BindingMapping.getClassName(potentialQName) + className)) {
+                        final ContainerLike schema = getRpcDataSchema(potential, qname);
+                        checkArgument(schema != null, "Schema for %s does not define input / output.", potentialQName);
+
+                        final ContainerLikeRuntimeType<?, ?> type = lookup.apply(context.getTypes(), potentialQName)
+                            .orElseThrow(() -> new IllegalArgumentException("Cannot find runtime type for " + key));
+
+                        return (ContainerNodeCodecContext) DataContainerCodecPrototype.from(key,
+                            (ContainerLikeRuntimeType<?, ?>) type, factory).get();
+                    }
                 }
-            });
 
-    private final LoadingCache<SchemaPath, RpcInputCodec<?>> rpcDataByPath = CacheBuilder.newBuilder().build(
-        new CacheLoader<SchemaPath, RpcInputCodec<?>>() {
+                throw new IllegalArgumentException("Supplied class " + key + " is not valid RPC class.");
+            }
+        });
+
+    private final LoadingCache<QName, DataContainerCodecContext<?,?>> childrenByQName =
+        CacheBuilder.newBuilder().build(new CacheLoader<>() {
             @Override
-            public RpcInputCodec<?> load(final SchemaPath key) {
-                final ContainerSchemaNode schema = SchemaContextUtil.getRpcDataSchema(getSchema(), key);
-                @SuppressWarnings("unchecked")
-                final Class<? extends DataContainer> cls = (Class<? extends DataContainer>)
-                        factory().getRuntimeContext().getClassForSchema(schema);
-                return getRpc(cls);
+            public DataContainerCodecContext<?, ?> load(final QName qname) throws ClassNotFoundException {
+                final var type = getType();
+                final var child = childNonNull(type.schemaTreeChild(qname), qname,
+                    "Argument %s is not valid child of %s", qname, type);
+                if (!(child instanceof DataRuntimeType)) {
+                    throw IncorrectNestingException.create("Argument %s is not valid data tree child of %s", qname,
+                        type);
+                }
+
+                // TODO: improve this check?
+                final var childSchema = child.statement();
+                if (childSchema instanceof DataNodeContainer || childSchema instanceof ChoiceSchemaNode) {
+                    return streamChild(factory().getRuntimeContext().loadClass(child.javaType()));
+                }
+
+                throw new UnsupportedOperationException("Unsupported child type " + childSchema.getClass());
             }
         });
 
-    private final LoadingCache<SchemaPath, NotificationCodecContext<?>> notificationsByPath = CacheBuilder.newBuilder()
-            .build(new CacheLoader<SchemaPath, NotificationCodecContext<?>>() {
-                @Override
-                public NotificationCodecContext<?> load(final SchemaPath key) {
-                    final NotificationDefinition schema = SchemaContextUtil.getNotificationSchema(getSchema(), key);
-                    @SuppressWarnings("unchecked")
-                    final Class<? extends Notification> clz = (Class<? extends Notification>)
-                            factory().getRuntimeContext().getClassForSchema(schema);
-                    return getNotification(clz);
+    private final LoadingCache<Absolute, RpcInputCodec<?>> rpcDataByPath =
+        CacheBuilder.newBuilder().build(new CacheLoader<>() {
+            @Override
+            public RpcInputCodec<?> load(final Absolute key) {
+                final var rpcName = key.firstNodeIdentifier();
+                final var context = factory().getRuntimeContext();
+
+                final Class<? extends DataContainer> container;
+                switch (key.lastNodeIdentifier().getLocalName()) {
+                    case "input":
+                        container = context.getRpcInput(rpcName);
+                        break;
+                    case "output":
+                        container = context.getRpcOutput(rpcName);
+                        break;
+                    default:
+                        throw new IllegalArgumentException("Unhandled path " + key);
                 }
-            });
 
-    private SchemaRootCodecContext(final DataContainerCodecPrototype<SchemaContext> dataPrototype) {
+                return getRpc(container);
+            }
+        });
+
+    private final LoadingCache<Absolute, NotificationCodecContext<?>> notificationsByPath =
+        CacheBuilder.newBuilder().build(new CacheLoader<>() {
+            @Override
+            public NotificationCodecContext<?> load(final Absolute key) {
+                final Class<?> cls = factory().getRuntimeContext().getClassForSchema(key);
+                checkArgument(Notification.class.isAssignableFrom(cls), "Path %s does not represent a notification",
+                    key);
+                return getNotificationImpl(cls);
+            }
+        });
+
+    private SchemaRootCodecContext(final DataContainerCodecPrototype<BindingRuntimeTypes> dataPrototype) {
         super(dataPrototype);
     }
 
@@ -128,23 +217,20 @@ final class SchemaRootCodecContext<D extends DataObject> extends DataContainerCo
      * @return A new root node
      */
     static SchemaRootCodecContext<?> create(final CodecContextFactory factory) {
-        final DataContainerCodecPrototype<SchemaContext> prototype = DataContainerCodecPrototype.rootPrototype(factory);
-        return new SchemaRootCodecContext<>(prototype);
+        return new SchemaRootCodecContext<>(DataContainerCodecPrototype.rootPrototype(factory));
     }
 
+    @Override
+    public WithStatus getSchema() {
+        return getType().getEffectiveModelContext();
+    }
 
-    @SuppressWarnings("unchecked")
     @Override
+    @SuppressWarnings("unchecked")
     public <C extends DataObject> DataContainerCodecContext<C, ?> streamChild(final Class<C> childClass) {
-        /* FIXME: This is still not solved for RPCs
-         * TODO: Probably performance wise RPC, Data and Notification loading cache
-         *       should be merge for performance resons. Needs microbenchmark to
-         *       determine which is faster (keeping them separate or in same cache).
-         */
-        if (Notification.class.isAssignableFrom(childClass)) {
-            return (DataContainerCodecContext<C, ?>) getNotification((Class<? extends Notification>)childClass);
-        }
-        return (DataContainerCodecContext<C, ?>) getOrRethrow(childrenByClass,childClass);
+        final DataContainerCodecContext<?, ?> result = Notification.class.isAssignableFrom(childClass)
+            ? getNotificationImpl(childClass) : getOrRethrow(childrenByClass, childClass);
+        return (DataContainerCodecContext<C, ?>) result;
     }
 
     @Override
@@ -159,115 +245,129 @@ final class SchemaRootCodecContext<D extends DataObject> extends DataContainerCo
     }
 
     @Override
-    public D deserialize(final NormalizedNode<?, ?> normalizedNode) {
+    public D deserialize(final NormalizedNode normalizedNode) {
         throw new UnsupportedOperationException("Could not create Binding data representation for root");
     }
 
-    NotificationCodecContext<?> getNotification(final Class<? extends Notification> notification) {
-        return getOrRethrow(notificationsByClass, notification);
+    ActionCodecContext getAction(final Class<? extends Action<?, ?, ?>> action) {
+        return getOrRethrow(actionsByClass, action);
     }
 
-    NotificationCodecContext<?> getNotification(final SchemaPath notification) {
+    NotificationCodecContext<?> getNotification(final Absolute notification) {
         return getOrRethrow(notificationsByPath, notification);
     }
 
+    NotificationCodecContext<?> getNotification(final Class<? extends Notification<?>> notification) {
+        return getNotificationImpl(notification);
+    }
+
+    private NotificationCodecContext<?> getNotificationImpl(final Class<?> notification) {
+        return getOrRethrow(notificationsByClass, notification);
+    }
+
     ContainerNodeCodecContext<?> getRpc(final Class<? extends DataContainer> rpcInputOrOutput) {
         return getOrRethrow(rpcDataByClass, rpcInputOrOutput);
     }
 
-    RpcInputCodec<?> getRpc(final SchemaPath notification) {
-        return getOrRethrow(rpcDataByPath, notification);
+    RpcInputCodec<?> getRpc(final Absolute containerPath) {
+        return getOrRethrow(rpcDataByPath, containerPath);
     }
 
-    DataContainerCodecContext<?,?> createDataTreeChildContext(final Class<?> key) {
-        final QName qname = BindingReflections.findQName(key);
-        final DataSchemaNode childSchema = childNonNull(getSchema().getDataChildByName(qname), key,
+    DataContainerCodecContext<?, ?> createDataTreeChildContext(final Class<? extends DataObject> key) {
+        final RuntimeType childSchema = childNonNull(getType().bindingChild(JavaTypeName.create(key)), key,
             "%s is not top-level item.", key);
-        return DataContainerCodecPrototype.from(key, childSchema, factory()).get();
+        if (childSchema instanceof CompositeRuntimeType && childSchema instanceof DataRuntimeType) {
+            return DataContainerCodecPrototype.from(key, (CompositeRuntimeType) childSchema, factory()).get();
+        }
+        throw IncorrectNestingException.create("%s is not a valid data tree child of %s", key, this);
     }
 
-    ContainerNodeCodecContext<?> createRpcDataContext(final Class<?> key) {
-        Preconditions.checkArgument(DataContainer.class.isAssignableFrom(key));
-        final QName qname = BindingReflections.findQName(key);
-        final QNameModule qnameModule = qname.getModule();
-        final Module module = getSchema().findModule(qnameModule)
-                .orElseThrow(() -> new IllegalArgumentException("Failed to find module for " + qnameModule));
-        final String className = BindingMapping.getClassName(qname);
-
-        RpcDefinition rpc = null;
-        for (final RpcDefinition potential : module.getRpcs()) {
-            final QName potentialQName = potential.getQName();
-            /*
-             * Check if rpc and class represents data from same module and then
-             * checks if rpc local name produces same class name as class name
-             * appended with Input/Output based on QName associated with bidning
-             * class.
-             *
-             * FIXME: Rework this to have more precise logic regarding Binding
-             * Specification.
-             */
-            if (key.getSimpleName().equals(BindingMapping.getClassName(potentialQName) + className)) {
-                rpc = potential;
-                break;
-            }
+    ActionCodecContext createActionContext(final Class<? extends Action<?, ?, ?>> action) {
+        if (KeyedListAction.class.isAssignableFrom(action)) {
+            return prepareActionContext(2, 3, 4, action, KeyedListAction.class);
+        } else if (Action.class.isAssignableFrom(action)) {
+            return prepareActionContext(1, 2, 3, action, Action.class);
         }
-        Preconditions.checkArgument(rpc != null, "Supplied class %s is not valid RPC class.", key);
-        final ContainerSchemaNode schema = SchemaNodeUtils.getRpcDataSchema(rpc, qname);
-        Preconditions.checkArgument(schema != null, "Schema for %s does not define input / output.", rpc.getQName());
-        return (ContainerNodeCodecContext<?>) DataContainerCodecPrototype.from(key, schema, factory()).get();
+        throw new IllegalArgumentException("The specific action type does not exist for action " + action.getName());
     }
 
-    NotificationCodecContext<?> createNotificationDataContext(final Class<?> notificationType) {
-        Preconditions.checkArgument(Notification.class.isAssignableFrom(notificationType));
-        Preconditions.checkArgument(notificationType.isInterface(), "Supplied class must be interface.");
-        final QName qname = BindingReflections.findQName(notificationType);
-        /**
-         *  FIXME: After Lithium cleanup of yang-model-api, use direct call on schema context
-         *  to retrieve notification via index.
-         */
-        final NotificationDefinition schema = SchemaContextUtil.getNotificationSchema(getSchema(),
-                SchemaPath.create(true, qname));
-        Preconditions.checkArgument(schema != null, "Supplied %s is not valid notification", notificationType);
-
-        return new NotificationCodecContext<>(notificationType, schema, factory());
+    private ActionCodecContext prepareActionContext(final int inputOffset, final int outputOffset,
+            final int expectedArgsLength, final Class<? extends Action<?, ?, ?>> action, final Class<?> actionType) {
+        final Optional<ParameterizedType> optParamType = ClassLoaderUtils.findParameterizedType(action, actionType);
+        checkState(optParamType.isPresent(), "%s does not specialize %s", action, actionType);
+
+        final ParameterizedType paramType = optParamType.get();
+        final Type[] args = paramType.getActualTypeArguments();
+        checkArgument(args.length == expectedArgsLength, "Unexpected (%s) Action generatic arguments", args.length);
+        final ActionRuntimeType schema = factory().getRuntimeContext().getActionDefinition(action);
+        return new ActionCodecContext(
+            DataContainerCodecPrototype.from(asClass(args[inputOffset], RpcInput.class), schema.input(),
+                factory()).get(),
+            DataContainerCodecPrototype.from(asClass(args[outputOffset], RpcOutput.class), schema.output(),
+                factory()).get());
+    }
+
+    private static <T extends DataObject> Class<? extends T> asClass(final Type type, final Class<T> target) {
+        verify(type instanceof Class, "Type %s is not a class", type);
+        return ((Class<?>) type).asSubclass(target);
+    }
+
+    /**
+     * Returns RPC input or output schema based on supplied QName.
+     *
+     * @param rpc RPC Definition
+     * @param qname input or output QName with namespace same as RPC
+     * @return input or output schema. Returns null if RPC does not have input/output specified.
+     */
+    private static @Nullable ContainerLike getRpcDataSchema(final @NonNull RpcDefinition rpc,
+            final @NonNull QName qname) {
+        requireNonNull(rpc, "Rpc Schema must not be null");
+        switch (requireNonNull(qname, "QName must not be null").getLocalName()) {
+            case "input":
+                return rpc.getInput();
+            case "output":
+                return rpc.getOutput();
+            default:
+                throw new IllegalArgumentException("Supplied qname " + qname
+                        + " does not represent rpc input or output.");
+        }
     }
 
     ChoiceNodeCodecContext<?> createChoiceDataContext(final Class<? extends DataObject> caseType) {
         final Class<?> choiceClass = findCaseChoice(caseType);
-        Preconditions.checkArgument(choiceClass != null, "Class %s is not a valid case representation", caseType);
-        final DataSchemaNode schema = factory().getRuntimeContext().getSchemaDefinition(choiceClass);
-        Preconditions.checkArgument(schema instanceof ChoiceSchemaNode, "Class %s does not refer to a choice",
-            caseType);
+        checkArgument(choiceClass != null, "Class %s is not a valid case representation", caseType);
+        final CompositeRuntimeType schema = factory().getRuntimeContext().getSchemaDefinition(choiceClass);
+        checkArgument(schema instanceof ChoiceRuntimeType, "Class %s does not refer to a choice", caseType);
 
-        final DataContainerCodecContext<?, ChoiceSchemaNode> choice = DataContainerCodecPrototype.from(choiceClass,
-            (ChoiceSchemaNode)schema, factory()).get();
+        final DataContainerCodecContext<?, ChoiceRuntimeType> choice = DataContainerCodecPrototype.from(choiceClass,
+            (ChoiceRuntimeType)schema, factory()).get();
         Verify.verify(choice instanceof ChoiceNodeCodecContext);
         return (ChoiceNodeCodecContext<?>) choice;
     }
 
     @Override
-    protected Object deserializeObject(final NormalizedNode<?, ?> normalizedNode) {
+    protected Object deserializeObject(final NormalizedNode normalizedNode) {
         throw new UnsupportedOperationException("Unable to deserialize root");
     }
 
     @Override
     public InstanceIdentifier.PathArgument deserializePathArgument(final YangInstanceIdentifier.PathArgument arg) {
-        Preconditions.checkArgument(arg == null);
+        checkArgument(arg == null);
         return null;
     }
 
     @Override
     public YangInstanceIdentifier.PathArgument serializePathArgument(final InstanceIdentifier.PathArgument arg) {
-        Preconditions.checkArgument(arg == null);
+        checkArgument(arg == null);
         return null;
     }
 
     @Override
     public DataContainerCodecContext<?, ?> bindingPathArgumentChild(final InstanceIdentifier.PathArgument arg,
             final List<PathArgument> builder) {
-        final java.util.Optional<? extends Class<? extends DataObject>> caseType = arg.getCaseType();
+        final Optional<? extends Class<? extends DataObject>> caseType = arg.getCaseType();
         if (caseType.isPresent()) {
-            final Class<? extends DataObject> type = caseType.get();
+            final @NonNull Class<? extends DataObject> type = caseType.orElseThrow();
             final ChoiceNodeCodecContext<?> choice = choicesByClass.getUnchecked(type);
             choice.addYangPathArgument(arg, builder);
             final DataContainerCodecContext<?, ?> caze = choice.streamChild(type);
@@ -302,4 +402,4 @@ final class SchemaRootCodecContext<D extends DataObject> extends DataContainerCo
             throw e;
         }
     }
-}
\ No newline at end of file
+}