Split out RFC8040-only constructs from yang-common
[netconf.git] / restconf / restconf-nb-rfc8040 / src / main / java / org / opendaylight / restconf / nb / rfc8040 / rests / services / impl / RestconfDataServiceImpl.java
1 /*
2  * Copyright (c) 2016 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.restconf.nb.rfc8040.rests.services.impl;
9
10 import static java.util.Objects.requireNonNull;
11 import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfDataServiceConstant.PostPutQueryParameters.INSERT;
12 import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfDataServiceConstant.PostPutQueryParameters.POINT;
13 import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfStreamsConstants.NOTIFICATION_STREAM;
14 import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfStreamsConstants.STREAMS_PATH;
15 import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfStreamsConstants.STREAM_ACCESS_PATH_PART;
16 import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfStreamsConstants.STREAM_LOCATION_PATH_PART;
17 import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfStreamsConstants.STREAM_PATH;
18 import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfStreamsConstants.STREAM_PATH_PART;
19
20 import com.google.common.annotations.VisibleForTesting;
21 import com.google.common.collect.ImmutableList;
22 import com.google.common.util.concurrent.Futures;
23 import com.google.common.util.concurrent.MoreExecutors;
24 import java.net.URI;
25 import java.time.Clock;
26 import java.time.LocalDateTime;
27 import java.time.format.DateTimeFormatter;
28 import java.util.HashMap;
29 import java.util.List;
30 import java.util.Map;
31 import java.util.Map.Entry;
32 import java.util.Optional;
33 import java.util.concurrent.CancellationException;
34 import java.util.concurrent.ExecutionException;
35 import javax.ws.rs.Path;
36 import javax.ws.rs.core.Response;
37 import javax.ws.rs.core.Response.Status;
38 import javax.ws.rs.core.UriInfo;
39 import org.eclipse.jdt.annotation.Nullable;
40 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
41 import org.opendaylight.mdsal.dom.api.DOMActionException;
42 import org.opendaylight.mdsal.dom.api.DOMActionResult;
43 import org.opendaylight.mdsal.dom.api.DOMActionService;
44 import org.opendaylight.mdsal.dom.api.DOMDataBroker;
45 import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
46 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteOperations;
47 import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
48 import org.opendaylight.mdsal.dom.api.DOMMountPoint;
49 import org.opendaylight.mdsal.dom.api.DOMMountPointService;
50 import org.opendaylight.mdsal.dom.api.DOMSchemaService;
51 import org.opendaylight.mdsal.dom.spi.SimpleDOMActionResult;
52 import org.opendaylight.restconf.common.context.InstanceIdentifierContext;
53 import org.opendaylight.restconf.common.context.NormalizedNodeContext;
54 import org.opendaylight.restconf.common.errors.RestconfDocumentedException;
55 import org.opendaylight.restconf.common.patch.PatchContext;
56 import org.opendaylight.restconf.common.patch.PatchStatusContext;
57 import org.opendaylight.restconf.nb.rfc8040.Rfc8040;
58 import org.opendaylight.restconf.nb.rfc8040.handlers.SchemaContextHandler;
59 import org.opendaylight.restconf.nb.rfc8040.legacy.NormalizedNodePayload;
60 import org.opendaylight.restconf.nb.rfc8040.legacy.QueryParameters;
61 import org.opendaylight.restconf.nb.rfc8040.rests.services.api.RestconfDataService;
62 import org.opendaylight.restconf.nb.rfc8040.rests.services.api.RestconfStreamsSubscriptionService;
63 import org.opendaylight.restconf.nb.rfc8040.rests.transactions.MdsalRestconfStrategy;
64 import org.opendaylight.restconf.nb.rfc8040.rests.transactions.RestconfStrategy;
65 import org.opendaylight.restconf.nb.rfc8040.rests.utils.DeleteDataTransactionUtil;
66 import org.opendaylight.restconf.nb.rfc8040.rests.utils.PatchDataTransactionUtil;
67 import org.opendaylight.restconf.nb.rfc8040.rests.utils.PlainPatchDataTransactionUtil;
68 import org.opendaylight.restconf.nb.rfc8040.rests.utils.PostDataTransactionUtil;
69 import org.opendaylight.restconf.nb.rfc8040.rests.utils.PutDataTransactionUtil;
70 import org.opendaylight.restconf.nb.rfc8040.rests.utils.ReadDataTransactionUtil;
71 import org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfDataServiceConstant;
72 import org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfDataServiceConstant.PostPutQueryParameters.Insert;
73 import org.opendaylight.restconf.nb.rfc8040.streams.Configuration;
74 import org.opendaylight.restconf.nb.rfc8040.streams.listeners.NotificationListenerAdapter;
75 import org.opendaylight.restconf.nb.rfc8040.utils.mapping.RestconfMappingNodeUtil;
76 import org.opendaylight.restconf.nb.rfc8040.utils.parser.ParserIdentifier;
77 import org.opendaylight.yang.gen.v1.urn.sal.restconf.event.subscription.rev140708.NotificationOutputTypeGrouping.NotificationOutputType;
78 import org.opendaylight.yangtools.concepts.Immutable;
79 import org.opendaylight.yangtools.yang.common.ErrorTag;
80 import org.opendaylight.yangtools.yang.common.ErrorType;
81 import org.opendaylight.yangtools.yang.common.QName;
82 import org.opendaylight.yangtools.yang.common.Revision;
83 import org.opendaylight.yangtools.yang.common.RpcError;
84 import org.opendaylight.yangtools.yang.common.RpcResultBuilder;
85 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
86 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifierWithPredicates;
87 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
88 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
89 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
90 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
91 import org.opendaylight.yangtools.yang.model.api.ActionDefinition;
92 import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext;
93 import org.opendaylight.yangtools.yang.model.api.ListSchemaNode;
94 import org.opendaylight.yangtools.yang.model.api.NotificationDefinition;
95 import org.opendaylight.yangtools.yang.model.api.SchemaNode;
96 import org.opendaylight.yangtools.yang.model.api.stmt.SchemaNodeIdentifier.Absolute;
97 import org.slf4j.Logger;
98 import org.slf4j.LoggerFactory;
99
100 /**
101  * Implementation of {@link RestconfDataService}.
102  */
103 @Path("/")
104 public class RestconfDataServiceImpl implements RestconfDataService {
105     // FIXME: we should be able to interpret 'point' and refactor this class into a behavior
106     private static final class QueryParams implements Immutable {
107         final @Nullable String point;
108         final @Nullable Insert insert;
109
110         QueryParams(final @Nullable Insert insert, final @Nullable String point) {
111             this.insert = insert;
112             this.point = point;
113         }
114     }
115
116     private static final Logger LOG = LoggerFactory.getLogger(RestconfDataServiceImpl.class);
117     private static final DateTimeFormatter FORMATTER = DateTimeFormatter.ofPattern("yyyy-MMM-dd HH:mm:ss");
118
119     private final RestconfStreamsSubscriptionService delegRestconfSubscrService;
120     private final SchemaContextHandler schemaContextHandler;
121     private final MdsalRestconfStrategy restconfStrategy;
122     private final DOMMountPointService mountPointService;
123     private final SubscribeToStreamUtil streamUtils;
124     private final DOMActionService actionService;
125     private final DOMDataBroker dataBroker;
126
127     public RestconfDataServiceImpl(final SchemaContextHandler schemaContextHandler,
128             final DOMDataBroker dataBroker, final DOMMountPointService  mountPointService,
129             final RestconfStreamsSubscriptionService delegRestconfSubscrService,
130             final DOMActionService actionService, final Configuration configuration) {
131         this.schemaContextHandler = requireNonNull(schemaContextHandler);
132         this.dataBroker = requireNonNull(dataBroker);
133         restconfStrategy = new MdsalRestconfStrategy(dataBroker);
134         this.mountPointService = requireNonNull(mountPointService);
135         this.delegRestconfSubscrService = requireNonNull(delegRestconfSubscrService);
136         this.actionService = requireNonNull(actionService);
137         streamUtils = configuration.isUseSSE() ? SubscribeToStreamUtil.serverSentEvents()
138                 : SubscribeToStreamUtil.webSockets();
139     }
140
141     @Override
142     public Response readData(final UriInfo uriInfo) {
143         return readData(null, uriInfo);
144     }
145
146     @Override
147     public Response readData(final String identifier, final UriInfo uriInfo) {
148         final EffectiveModelContext schemaContextRef = schemaContextHandler.get();
149         final InstanceIdentifierContext<?> instanceIdentifier = ParserIdentifier.toInstanceIdentifier(
150                 identifier, schemaContextRef, Optional.of(mountPointService));
151         final QueryParameters parameters = ReadDataTransactionUtil.parseUriParameters(instanceIdentifier, uriInfo);
152
153         final DOMMountPoint mountPoint = instanceIdentifier.getMountPoint();
154
155         // FIXME: this looks quite crazy, why do we even have it?
156         if (mountPoint == null && identifier != null && identifier.contains(STREAMS_PATH)
157             && !identifier.contains(STREAM_PATH_PART)) {
158             createAllYangNotificationStreams(schemaContextRef, uriInfo);
159         }
160
161         final List<YangInstanceIdentifier> fieldPaths = parameters.getFieldPaths();
162         final RestconfStrategy strategy = getRestconfStrategy(mountPoint);
163         final NormalizedNode node;
164         if (fieldPaths != null && !fieldPaths.isEmpty()) {
165             node = ReadDataTransactionUtil.readData(parameters.getContent(), instanceIdentifier.getInstanceIdentifier(),
166                     strategy, parameters.getWithDefault(), schemaContextRef, fieldPaths);
167         } else {
168             node = ReadDataTransactionUtil.readData(parameters.getContent(), instanceIdentifier.getInstanceIdentifier(),
169                     strategy, parameters.getWithDefault(), schemaContextRef);
170         }
171
172         // FIXME: this is utter craziness, refactor it properly!
173         if (identifier != null && identifier.contains(STREAM_PATH) && identifier.contains(STREAM_ACCESS_PATH_PART)
174                 && identifier.contains(STREAM_LOCATION_PATH_PART)) {
175             final String value = (String) node.body();
176             final String streamName = value.substring(value.indexOf(NOTIFICATION_STREAM + '/'));
177             delegRestconfSubscrService.subscribeToStream(streamName, uriInfo);
178         }
179         if (node == null) {
180             throw new RestconfDocumentedException(
181                     "Request could not be completed because the relevant data model content does not exist",
182                     ErrorType.PROTOCOL, ErrorTag.DATA_MISSING);
183         }
184
185         if (parameters.getContent().equals(RestconfDataServiceConstant.ReadData.ALL)
186                     || parameters.getContent().equals(RestconfDataServiceConstant.ReadData.CONFIG)) {
187             final QName type = node.getIdentifier().getNodeType();
188             return Response.status(Status.OK)
189                     .entity(new NormalizedNodeContext(instanceIdentifier, node, parameters))
190                     .header("ETag", '"' + type.getModule().getRevision().map(Revision::toString).orElse(null)
191                         + "-" + type.getLocalName() + '"')
192                     .header("Last-Modified", FORMATTER.format(LocalDateTime.now(Clock.systemUTC())))
193                     .build();
194         }
195
196         return Response.status(Status.OK)
197             .entity(new NormalizedNodeContext(instanceIdentifier, node, parameters))
198             .build();
199     }
200
201     private void createAllYangNotificationStreams(final EffectiveModelContext schemaContext, final UriInfo uriInfo) {
202         final DOMDataTreeWriteTransaction transaction = dataBroker.newWriteOnlyTransaction();
203         for (final NotificationDefinition notificationDefinition : schemaContext.getNotifications()) {
204             writeNotificationStreamToDatastore(schemaContext, uriInfo, transaction,
205                 CreateStreamUtil.createYangNotifiStream(notificationDefinition, schemaContext,
206                     NotificationOutputType.XML));
207             writeNotificationStreamToDatastore(schemaContext, uriInfo, transaction,
208                 CreateStreamUtil.createYangNotifiStream(notificationDefinition, schemaContext,
209                     NotificationOutputType.JSON));
210         }
211         try {
212             transaction.commit().get();
213         } catch (final InterruptedException | ExecutionException e) {
214             throw new RestconfDocumentedException("Problem while putting data to DS.", e);
215         }
216     }
217
218     private void writeNotificationStreamToDatastore(final EffectiveModelContext schemaContext,
219             final UriInfo uriInfo, final DOMDataTreeWriteOperations tx, final NotificationListenerAdapter listener) {
220         final URI uri = streamUtils.prepareUriByStreamName(uriInfo, listener.getStreamName());
221         final MapEntryNode mapToStreams = RestconfMappingNodeUtil.mapYangNotificationStreamByIetfRestconfMonitoring(
222                 listener.getSchemaPath().lastNodeIdentifier(), schemaContext.getNotifications(), null,
223                 listener.getOutputType(), uri);
224
225         tx.merge(LogicalDatastoreType.OPERATIONAL,
226             Rfc8040.restconfStateStreamPath(mapToStreams.getIdentifier()), mapToStreams);
227     }
228
229     @Override
230     public Response putData(final String identifier, final NormalizedNodePayload payload, final UriInfo uriInfo) {
231         requireNonNull(payload);
232
233         final QueryParams checkedParms = checkQueryParameters(uriInfo);
234
235         final InstanceIdentifierContext<? extends SchemaNode> iid = payload.getInstanceIdentifierContext();
236
237         validInputData(iid.getSchemaNode(), payload);
238         validTopLevelNodeName(iid.getInstanceIdentifier(), payload);
239         validateListKeysEqualityInPayloadAndUri(payload);
240
241         final DOMMountPoint mountPoint = payload.getInstanceIdentifierContext().getMountPoint();
242         final EffectiveModelContext ref = mountPoint == null
243                 ? schemaContextHandler.get() : modelContext(mountPoint);
244
245         final RestconfStrategy strategy = getRestconfStrategy(mountPoint);
246         return PutDataTransactionUtil.putData(payload, ref, strategy, checkedParms.insert, checkedParms.point);
247     }
248
249     private static QueryParams checkQueryParameters(final UriInfo uriInfo) {
250         boolean insertUsed = false;
251         boolean pointUsed = false;
252         Insert insert = null;
253         String point = null;
254
255         for (final Entry<String, List<String>> entry : uriInfo.getQueryParameters().entrySet()) {
256             switch (entry.getKey()) {
257                 case INSERT:
258                     if (insertUsed) {
259                         throw new RestconfDocumentedException("Insert parameter can be used only once.",
260                             ErrorType.PROTOCOL, ErrorTag.BAD_ELEMENT);
261                     }
262
263                     insertUsed = true;
264                     final String str = entry.getValue().get(0);
265                     insert = Insert.forValue(str);
266                     if (insert == null) {
267                         throw new RestconfDocumentedException("Unrecognized insert parameter value '" + str + "'",
268                             ErrorType.PROTOCOL, ErrorTag.BAD_ELEMENT);
269                     }
270                     break;
271                 case POINT:
272                     if (pointUsed) {
273                         throw new RestconfDocumentedException("Point parameter can be used only once.",
274                             ErrorType.PROTOCOL, ErrorTag.BAD_ELEMENT);
275                     }
276
277                     pointUsed = true;
278                     point = entry.getValue().get(0);
279                     break;
280                 default:
281                     throw new RestconfDocumentedException("Bad parameter for post: " + entry.getKey(),
282                             ErrorType.PROTOCOL, ErrorTag.BAD_ELEMENT);
283             }
284         }
285
286         checkQueryParams(insertUsed, pointUsed, insert);
287         return new QueryParams(insert, point);
288     }
289
290     private static void checkQueryParams(final boolean insertUsed, final boolean pointUsed, final Insert insert) {
291         if (pointUsed) {
292             if (!insertUsed) {
293                 throw new RestconfDocumentedException("Point parameter can't be used without Insert parameter.",
294                     ErrorType.PROTOCOL, ErrorTag.BAD_ELEMENT);
295             }
296
297             if (insert != Insert.BEFORE && insert != Insert.AFTER) {
298                 throw new RestconfDocumentedException(
299                     "Point parameter can be used only with 'after' or 'before' values of Insert parameter.",
300                     ErrorType.PROTOCOL, ErrorTag.BAD_ELEMENT);
301             }
302         }
303     }
304
305     @Override
306     public Response postData(final String identifier, final NormalizedNodePayload payload, final UriInfo uriInfo) {
307         return postData(payload, uriInfo);
308     }
309
310     @Override
311     public Response postData(final NormalizedNodePayload payload, final UriInfo uriInfo) {
312         requireNonNull(payload);
313         if (payload.getInstanceIdentifierContext().getSchemaNode() instanceof ActionDefinition) {
314             return invokeAction(payload);
315         }
316
317         final QueryParams checkedParms = checkQueryParameters(uriInfo);
318         final DOMMountPoint mountPoint = payload.getInstanceIdentifierContext().getMountPoint();
319         final RestconfStrategy strategy = getRestconfStrategy(mountPoint);
320         return PostDataTransactionUtil.postData(uriInfo, payload, strategy,
321                 getSchemaContext(mountPoint), checkedParms.insert, checkedParms.point);
322     }
323
324     @Override
325     public Response deleteData(final String identifier) {
326         final InstanceIdentifierContext<?> instanceIdentifier = ParserIdentifier.toInstanceIdentifier(
327                 identifier, schemaContextHandler.get(), Optional.of(mountPointService));
328
329         final DOMMountPoint mountPoint = instanceIdentifier.getMountPoint();
330         final RestconfStrategy strategy = getRestconfStrategy(mountPoint);
331         return DeleteDataTransactionUtil.deleteData(strategy, instanceIdentifier.getInstanceIdentifier());
332     }
333
334     @Override
335     public PatchStatusContext patchData(final String identifier, final PatchContext context, final UriInfo uriInfo) {
336         return patchData(context, uriInfo);
337     }
338
339     @Override
340     public PatchStatusContext patchData(final PatchContext context, final UriInfo uriInfo) {
341         final DOMMountPoint mountPoint = RestconfDocumentedException.throwIfNull(context,
342             ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE, "No patch documented provided")
343             .getInstanceIdentifierContext().getMountPoint();
344         final RestconfStrategy strategy = getRestconfStrategy(mountPoint);
345         return PatchDataTransactionUtil.patchData(context, strategy, getSchemaContext(mountPoint));
346     }
347
348     @Override
349     public Response patchData(final String identifier, final NormalizedNodePayload payload, final UriInfo uriInfo) {
350         requireNonNull(payload);
351
352         final InstanceIdentifierContext<? extends SchemaNode> iid = payload.getInstanceIdentifierContext();
353         validInputData(iid.getSchemaNode(), payload);
354         validTopLevelNodeName(iid.getInstanceIdentifier(), payload);
355         validateListKeysEqualityInPayloadAndUri(payload);
356
357         final DOMMountPoint mountPoint = payload.getInstanceIdentifierContext().getMountPoint();
358         final EffectiveModelContext ref = mountPoint == null
359                 ? schemaContextHandler.get() : modelContext(mountPoint);
360         final RestconfStrategy strategy = getRestconfStrategy(mountPoint);
361
362         return PlainPatchDataTransactionUtil.patchData(payload, strategy, ref);
363     }
364
365     private EffectiveModelContext getSchemaContext(final DOMMountPoint mountPoint) {
366         return mountPoint == null ? schemaContextHandler.get() : modelContext(mountPoint);
367     }
368
369     // FIXME: why is this synchronized?
370     public synchronized RestconfStrategy getRestconfStrategy(final DOMMountPoint mountPoint) {
371         if (mountPoint == null) {
372             return restconfStrategy;
373         }
374
375         return RestconfStrategy.forMountPoint(mountPoint).orElseThrow(() -> {
376             LOG.warn("Mount point {} does not expose a suitable access interface", mountPoint.getIdentifier());
377             return new RestconfDocumentedException("Could not find a supported access interface in mount point "
378                 + mountPoint.getIdentifier());
379         });
380     }
381
382     /**
383      * Invoke Action operation.
384      *
385      * @param payload {@link NormalizedNodeContext} - the body of the operation
386      * @return {@link NormalizedNodeContext} wrapped in {@link Response}
387      */
388     public Response invokeAction(final NormalizedNodeContext payload) {
389         final InstanceIdentifierContext<?> context = payload.getInstanceIdentifierContext();
390         final DOMMountPoint mountPoint = context.getMountPoint();
391         final Absolute schemaPath = Absolute.of(ImmutableList.copyOf(context.getSchemaNode().getPath()
392             .getPathFromRoot()));
393         final YangInstanceIdentifier yangIIdContext = context.getInstanceIdentifier();
394         final NormalizedNode data = payload.getData();
395
396         if (yangIIdContext.isEmpty()
397             && !RestconfDataServiceConstant.NETCONF_BASE_QNAME.equals(data.getIdentifier().getNodeType())) {
398             throw new RestconfDocumentedException("Instance identifier need to contain at least one path argument",
399                 ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE);
400         }
401
402         final DOMActionResult response;
403         final EffectiveModelContext schemaContextRef;
404         if (mountPoint != null) {
405             response = invokeAction((ContainerNode) data, schemaPath, yangIIdContext, mountPoint);
406             schemaContextRef = modelContext(mountPoint);
407         } else {
408             response = invokeAction((ContainerNode) data, schemaPath, yangIIdContext, actionService);
409             schemaContextRef = schemaContextHandler.get();
410         }
411         final DOMActionResult result = checkActionResponse(response);
412
413         ActionDefinition resultNodeSchema = null;
414         ContainerNode resultData = null;
415         if (result != null) {
416             final Optional<ContainerNode> optOutput = result.getOutput();
417             if (optOutput.isPresent()) {
418                 resultData = optOutput.get();
419                 resultNodeSchema = (ActionDefinition) context.getSchemaNode();
420             }
421         }
422
423         if (resultData != null && resultData.isEmpty()) {
424             return Response.status(Status.NO_CONTENT).build();
425         }
426
427         return Response.status(Status.OK)
428             .entity(new NormalizedNodeContext(
429                 new InstanceIdentifierContext<>(yangIIdContext, resultNodeSchema, mountPoint, schemaContextRef),
430                 resultData))
431             .build();
432     }
433
434
435     /**
436      * Invoking Action via mount point.
437      *
438      * @param mountPoint mount point
439      * @param data input data
440      * @param schemaPath schema path of data
441      * @return {@link DOMActionResult}
442      */
443     private static DOMActionResult invokeAction(final ContainerNode data,
444             final Absolute schemaPath, final YangInstanceIdentifier yangIId, final DOMMountPoint mountPoint) {
445         return invokeAction(data, schemaPath, yangIId, mountPoint.getService(DOMActionService.class)
446             .orElseThrow(() -> new RestconfDocumentedException("DomAction service is missing.")));
447     }
448
449     /**
450      * Invoke Action via ActionServiceHandler.
451      *
452      * @param data input data
453      * @param yangIId invocation context
454      * @param schemaPath schema path of data
455      * @param actionService action service to invoke action
456      * @return {@link DOMActionResult}
457      */
458     // FIXME: NETCONF-718: we should be returning a future here
459     private static DOMActionResult invokeAction(final ContainerNode data, final Absolute schemaPath,
460             final YangInstanceIdentifier yangIId, final DOMActionService actionService) {
461         return RestconfInvokeOperationsServiceImpl.checkedGet(Futures.catching(actionService.invokeAction(
462             schemaPath, new DOMDataTreeIdentifier(LogicalDatastoreType.OPERATIONAL, yangIId.getParent()), data),
463             DOMActionException.class,
464             cause -> new SimpleDOMActionResult(ImmutableList.of(RpcResultBuilder.newError(
465                 RpcError.ErrorType.RPC, "operation-failed", cause.getMessage()))),
466             MoreExecutors.directExecutor()));
467     }
468
469     /**
470      * Check the validity of the result.
471      *
472      * @param response response of Action
473      * @return {@link DOMActionResult} result
474      */
475     private static DOMActionResult checkActionResponse(final DOMActionResult response) {
476         if (response == null) {
477             return null;
478         }
479
480         try {
481             if (response.getErrors().isEmpty()) {
482                 return response;
483             }
484             LOG.debug("InvokeAction Error Message {}", response.getErrors());
485             throw new RestconfDocumentedException("InvokeAction Error Message ", null, response.getErrors());
486         } catch (final CancellationException e) {
487             final String errMsg = "The Action Operation was cancelled while executing.";
488             LOG.debug("Cancel Execution: {}", errMsg, e);
489             throw new RestconfDocumentedException(errMsg, ErrorType.RPC, ErrorTag.PARTIAL_OPERATION, e);
490         }
491     }
492
493     /**
494      * Valid input data with {@link SchemaNode}.
495      *
496      * @param schemaNode {@link SchemaNode}
497      * @param payload    input data
498      */
499     @VisibleForTesting
500     public static void validInputData(final SchemaNode schemaNode, final NormalizedNodeContext payload) {
501         if (schemaNode != null && payload.getData() == null) {
502             throw new RestconfDocumentedException("Input is required.", ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE);
503         } else if (schemaNode == null && payload.getData() != null) {
504             throw new RestconfDocumentedException("No input expected.", ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE);
505         }
506     }
507
508     /**
509      * Valid top level node name.
510      *
511      * @param path    path of node
512      * @param payload data
513      */
514     @VisibleForTesting
515     public static void validTopLevelNodeName(final YangInstanceIdentifier path, final NormalizedNodeContext payload) {
516         final String payloadName = payload.getData().getIdentifier().getNodeType().getLocalName();
517
518         if (path.isEmpty()) {
519             if (!payload.getData().getIdentifier().getNodeType().equals(
520                 RestconfDataServiceConstant.NETCONF_BASE_QNAME)) {
521                 throw new RestconfDocumentedException("Instance identifier has to contain at least one path argument",
522                         ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE);
523             }
524         } else {
525             final String identifierName = path.getLastPathArgument().getNodeType().getLocalName();
526             if (!payloadName.equals(identifierName)) {
527                 throw new RestconfDocumentedException(
528                         "Payload name (" + payloadName + ") is different from identifier name (" + identifierName + ")",
529                         ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE);
530             }
531         }
532     }
533
534
535     /**
536      * Validates whether keys in {@code payload} are equal to values of keys in
537      * {@code iiWithData} for list schema node.
538      *
539      * @throws RestconfDocumentedException if key values or key count in payload and URI isn't equal
540      */
541     @VisibleForTesting
542     public static void validateListKeysEqualityInPayloadAndUri(final NormalizedNodeContext payload) {
543         final InstanceIdentifierContext<?> iiWithData = payload.getInstanceIdentifierContext();
544         final PathArgument lastPathArgument = iiWithData.getInstanceIdentifier().getLastPathArgument();
545         final SchemaNode schemaNode = iiWithData.getSchemaNode();
546         final NormalizedNode data = payload.getData();
547         if (schemaNode instanceof ListSchemaNode) {
548             final List<QName> keyDefinitions = ((ListSchemaNode) schemaNode).getKeyDefinition();
549             if (lastPathArgument instanceof NodeIdentifierWithPredicates && data instanceof MapEntryNode) {
550                 final Map<QName, Object> uriKeyValues = ((NodeIdentifierWithPredicates) lastPathArgument).asMap();
551                 isEqualUriAndPayloadKeyValues(uriKeyValues, (MapEntryNode) data, keyDefinitions);
552             }
553         }
554     }
555
556     private static void isEqualUriAndPayloadKeyValues(final Map<QName, Object> uriKeyValues, final MapEntryNode payload,
557             final List<QName> keyDefinitions) {
558         final Map<QName, Object> mutableCopyUriKeyValues = new HashMap<>(uriKeyValues);
559         for (final QName keyDefinition : keyDefinitions) {
560             final Object uriKeyValue = RestconfDocumentedException.throwIfNull(
561                     mutableCopyUriKeyValues.remove(keyDefinition), ErrorType.PROTOCOL, ErrorTag.DATA_MISSING,
562                     "Missing key %s in URI.", keyDefinition);
563
564             final Object dataKeyValue = payload.getIdentifier().getValue(keyDefinition);
565
566             if (!uriKeyValue.equals(dataKeyValue)) {
567                 final String errMsg = "The value '" + uriKeyValue + "' for key '" + keyDefinition.getLocalName()
568                         + "' specified in the URI doesn't match the value '" + dataKeyValue
569                         + "' specified in the message body. ";
570                 throw new RestconfDocumentedException(errMsg, ErrorType.PROTOCOL, ErrorTag.INVALID_VALUE);
571             }
572         }
573     }
574
575     private static EffectiveModelContext modelContext(final DOMMountPoint mountPoint) {
576         return mountPoint.getService(DOMSchemaService.class)
577             .flatMap(svc -> Optional.ofNullable(svc.getGlobalContext()))
578             .orElse(null);
579     }
580 }