Move getRpcMethodToSchema()
[mdsal.git] / binding / mdsal-binding-dom-adapter / src / main / java / org / opendaylight / mdsal / binding / dom / adapter / RpcServiceAdapter.java
index a43bcb0629c298c1c457a8d69062632bc3db2ba5..18b2f2e41fdb324a51f61c924f7a4e318782505e 100644 (file)
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
-
 package org.opendaylight.mdsal.binding.dom.adapter;
 
-import org.opendaylight.mdsal.dom.spi.RpcRoutingStrategy;
+import static java.util.Objects.requireNonNull;
 
-import org.opendaylight.mdsal.dom.api.DOMRpcException;
-import org.opendaylight.mdsal.dom.api.DOMRpcResult;
-import org.opendaylight.mdsal.dom.api.DOMRpcService;
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableBiMap;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.util.concurrent.CheckedFuture;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
+import java.util.Map;
 import java.util.Map.Entry;
-import org.opendaylight.yangtools.binding.data.codec.impl.BindingNormalizedNodeCodecRegistry;
-import org.opendaylight.yangtools.yang.binding.DataContainer;
+import org.eclipse.jdt.annotation.NonNull;
+import org.opendaylight.mdsal.binding.runtime.api.RpcRuntimeType;
+import org.opendaylight.mdsal.binding.spec.naming.BindingMapping;
+import org.opendaylight.mdsal.binding.spec.reflect.BindingReflections;
+import org.opendaylight.mdsal.dom.api.DOMRpcService;
 import org.opendaylight.yangtools.yang.binding.DataObject;
-import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 import org.opendaylight.yangtools.yang.binding.RpcService;
-import org.opendaylight.yangtools.yang.binding.util.BindingReflections;
-import org.opendaylight.yangtools.yang.common.QName;
-import org.opendaylight.yangtools.yang.common.RpcResult;
-import org.opendaylight.yangtools.yang.common.RpcResultBuilder;
-import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
-import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifier;
-import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
-import org.opendaylight.yangtools.yang.data.api.schema.LeafNode;
-import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
-import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
-import org.opendaylight.yangtools.yang.model.api.RpcDefinition;
-import org.opendaylight.yangtools.yang.model.api.SchemaPath;
+import org.opendaylight.yangtools.yang.model.api.stmt.RpcEffectiveStatement;
 
 class RpcServiceAdapter implements InvocationHandler {
-
     private final ImmutableMap<Method, RpcInvocationStrategy> rpcNames;
-    private final Class<? extends RpcService> type;
-    private final BindingToNormalizedNodeCodec codec;
-    private final DOMRpcService delegate;
-    private final RpcService proxy;
+    private final @NonNull Class<? extends RpcService> type;
+    private final @NonNull AdapterContext adapterContext;
+    private final @NonNull DOMRpcService delegate;
+    private final @NonNull RpcService facade;
 
-    RpcServiceAdapter(final Class<? extends RpcService> type, final BindingToNormalizedNodeCodec codec,
+    RpcServiceAdapter(final Class<? extends RpcService> type, final AdapterContext adapterContext,
             final DOMRpcService domService) {
-        this.type = Preconditions.checkNotNull(type);
-        this.codec = Preconditions.checkNotNull(codec);
-        this.delegate = Preconditions.checkNotNull(domService);
-        final ImmutableMap.Builder<Method, RpcInvocationStrategy> rpcBuilder = ImmutableMap.builder();
-        for (final Entry<Method, RpcDefinition> rpc : codec.getRpcMethodToSchema(type).entrySet()) {
-            rpcBuilder.put(rpc.getKey(), createStrategy(rpc.getKey(), rpc.getValue()));
+        this.type = requireNonNull(type);
+        this.adapterContext = requireNonNull(adapterContext);
+        delegate = requireNonNull(domService);
+        facade = (RpcService) Proxy.newProxyInstance(type.getClassLoader(), new Class[] {type}, this);
+
+        final var methods = getRpcMethodToSchema(adapterContext.currentSerializer(), type);
+        final var rpcBuilder = ImmutableMap.<Method, RpcInvocationStrategy>builderWithExpectedSize(methods.size());
+        for (var entry : methods.entrySet()) {
+            final var method = entry.getKey();
+            rpcBuilder.put(method, RpcInvocationStrategy.of(this, method, entry.getValue()));
         }
         rpcNames = rpcBuilder.build();
-        proxy = (RpcService) Proxy.newProxyInstance(type.getClassLoader(), new Class[] {type}, this);
     }
 
-    private ListenableFuture<RpcResult<?>> invoke0(final SchemaPath schemaPath, final NormalizedNode<?, ?> input) {
-        final CheckedFuture<DOMRpcResult, DOMRpcException> result = delegate.invokeRpc(schemaPath, input);
-        if(result instanceof LazyDOMRpcResultFuture) {
-            return ((LazyDOMRpcResultFuture) result).getBindingFuture();
-        }
-
-        return transformFuture(schemaPath, result, codec.getCodecFactory());
+    final @NonNull CurrentAdapterSerializer currentSerializer() {
+        return adapterContext.currentSerializer();
     }
 
-    private RpcInvocationStrategy createStrategy(final Method method, final RpcDefinition schema) {
-        final RpcRoutingStrategy strategy = RpcRoutingStrategy.from(schema);
-        if (strategy.isContextBasedRouted()) {
-            return new RoutedStrategy(schema.getPath(), method, strategy.getLeaf());
-        }
-        return new NonRoutedStrategy(schema.getPath());
+    final @NonNull DOMRpcService delegate() {
+        return delegate;
     }
 
-    RpcService getProxy() {
-        return proxy;
+    final @NonNull RpcService facade() {
+        return facade;
     }
 
     @Override
-    public Object invoke(final Object proxy, final Method method, final Object[] args) throws Throwable {
-
-        final RpcInvocationStrategy rpc = rpcNames.get(method);
-        if (rpc != null) {
-            if (method.getParameterTypes().length == 0) {
-                return rpc.invokeEmpty();
-            }
+    public Object invoke(final Object proxy, final Method method, final Object[] args) {
+        final var strategy = rpcNames.get(method);
+        if (strategy != null) {
             if (args.length != 1) {
                 throw new IllegalArgumentException("Input must be provided.");
             }
-            return rpc.invoke((DataObject) args[0]);
+            return strategy.invoke((DataObject) requireNonNull(args[0]));
         }
 
-        if (isObjectMethod(method)) {
-            return callObjectMethod(proxy, method, args);
-        }
-        throw new UnsupportedOperationException("Method " + method.toString() + "is unsupported.");
-    }
-
-    private static boolean isObjectMethod(final Method m) {
-        switch (m.getName()) {
+        switch (method.getName()) {
             case "toString":
-                return (m.getReturnType().equals(String.class) && m.getParameterTypes().length == 0);
+                if (method.getReturnType().equals(String.class) && method.getParameterCount() == 0) {
+                    return type.getName() + "$Adapter{delegate=" + delegate.toString() + "}";
+                }
+                break;
             case "hashCode":
-                return (m.getReturnType().equals(int.class) && m.getParameterTypes().length == 0);
+                if (method.getReturnType().equals(int.class) && method.getParameterCount() == 0) {
+                    return System.identityHashCode(proxy);
+                }
+                break;
             case "equals":
-                return (m.getReturnType().equals(boolean.class) && m.getParameterTypes().length == 1 && m
-                        .getParameterTypes()[0] == Object.class);
+                if (method.getReturnType().equals(boolean.class) && method.getParameterCount() == 1
+                        && method.getParameterTypes()[0] == Object.class) {
+                    return proxy == args[0];
+                }
+                break;
             default:
-                return false;
+                break;
         }
-    }
 
-    private Object callObjectMethod(final Object self, final Method m, final Object[] args) {
-        switch (m.getName()) {
-            case "toString":
-                return type.getName() + "$Adapter{delegate=" + delegate.toString() + "}";
-            case "hashCode":
-                return System.identityHashCode(self);
-            case "equals":
-                return (self == args[0]);
-            default:
-                return null;
-        }
+        throw new UnsupportedOperationException("Method " + method.toString() + "is unsupported.");
     }
 
-    private static ListenableFuture<RpcResult<?>> transformFuture(final SchemaPath rpc,
-            final ListenableFuture<DOMRpcResult> domFuture, final BindingNormalizedNodeCodecRegistry codec) {
-        return Futures.transform(domFuture, new Function<DOMRpcResult, RpcResult<?>>() {
-            @Override
-            public RpcResult<?> apply(final DOMRpcResult input) {
-                final NormalizedNode<?, ?> domData = input.getResult();
-                final DataObject bindingResult;
-                if (domData != null) {
-                    final SchemaPath rpcOutput = rpc.createChild(QName.create(rpc.getLastComponent(), "output"));
-                    bindingResult = codec.fromNormalizedNodeRpcData(rpcOutput, (ContainerNode) domData);
-                } else {
-                    bindingResult = null;
+    // FIXME: This should be probably part of BindingRuntimeContext and RpcServices perhaps should have their own
+    //        RuntimeType
+    private static ImmutableBiMap<Method, RpcRuntimeType> getRpcMethodToSchema(
+            final CurrentAdapterSerializer serializer, final Class<? extends RpcService> key) {
+        final var runtimeContext = serializer.getRuntimeContext();
+        final var types = runtimeContext.getTypes();
+        final var qnameModule = BindingReflections.getQNameModule(key);
+
+        // We are dancing a bit here to reconstruct things a RpcServiceRuntimeType could easily hold
+        final var module = runtimeContext.getEffectiveModelContext().findModuleStatement(qnameModule)
+            .orElseThrow(() -> new IllegalStateException("No module found for " + qnameModule + " service " + key));
+        return module.streamEffectiveSubstatements(RpcEffectiveStatement.class)
+            .map(rpc -> {
+                final var rpcName = rpc.argument();
+                final var inputClz = runtimeContext.getRpcInput(rpcName);
+                final var methodName = BindingMapping.getRpcMethodName(rpcName);
+
+                final Method method;
+                try {
+                    method = key.getMethod(methodName, inputClz);
+                } catch (NoSuchMethodException e) {
+                    throw new IllegalStateException("Cannot find RPC method for " + rpc, e);
                 }
-                return RpcResult.class.cast(RpcResultBuilder.success(bindingResult).build());
-            }
-        });
-    }
-
-    private abstract class RpcInvocationStrategy {
-
-        private final SchemaPath rpcName;
-
-        protected RpcInvocationStrategy(final SchemaPath path) {
-            rpcName = path;
-        }
-
-        final ListenableFuture<RpcResult<?>> invoke(final DataObject input) {
-            return invoke0(rpcName, serialize(input));
-        }
-
-        abstract NormalizedNode<?, ?> serialize(DataObject input);
-
-        final ListenableFuture<RpcResult<?>> invokeEmpty() {
-            return invoke0(rpcName, null);
-        }
-
-        final SchemaPath getRpcName() {
-            return rpcName;
-        }
-
-    }
-
-    private final class NonRoutedStrategy extends RpcInvocationStrategy {
-
-        protected NonRoutedStrategy(final SchemaPath path) {
-            super(path);
-        }
-
-        @Override
-        NormalizedNode<?, ?> serialize(final DataObject input) {
-            return LazySerializedContainerNode.create(getRpcName(), input, codec.getCodecRegistry());
-        }
-
-    }
-
-    private final class RoutedStrategy extends RpcInvocationStrategy {
-
-        private final ContextReferenceExtractor refExtractor;
-        private final NodeIdentifier contextName;
-
-        protected RoutedStrategy(final SchemaPath path, final Method rpcMethod, final QName leafName) {
-            super(path);
-            final Class<? extends DataContainer> inputType = BindingReflections.resolveRpcInputClass(rpcMethod).get();
-            refExtractor = ContextReferenceExtractor.from(inputType);
-            this.contextName = new NodeIdentifier(leafName);
-        }
-
-        @Override
-        NormalizedNode<?, ?> serialize(final DataObject input) {
-            final InstanceIdentifier<?> bindingII = refExtractor.extract(input);
-            if (bindingII != null) {
-                final YangInstanceIdentifier yangII = codec.toYangInstanceIdentifierCached(bindingII);
-                final LeafNode<?> contextRef = ImmutableNodes.leafNode(contextName, yangII);
-                return LazySerializedContainerNode.withContextRef(getRpcName(), input, contextRef,
-                        codec.getCodecRegistry());
-            }
-            return LazySerializedContainerNode.create(getRpcName(), input, codec.getCodecRegistry());
-        }
 
+                final var type = types.schemaTreeChild(rpcName);
+                if (!(type instanceof RpcRuntimeType rpcType)) {
+                    throw new IllegalStateException("Unexpected run-time type " + type + " for " + rpcName);
+                }
+                return Map.entry(method, rpcType);
+            })
+            .collect(ImmutableBiMap.toImmutableBiMap(Entry::getKey, Entry::getValue));
     }
 }