BUG-1690: catch wildcard InstanceIdentifiers
[controller.git] / opendaylight / md-sal / sal-binding-broker / src / main / java / org / opendaylight / controller / md / sal / binding / impl / AbstractForwardedTransaction.java
index a935a33a5eb26ee13371554165db996f390469b8..eadde73e42c0daf632da84dc84fadfc08a171f92 100644 (file)
  */
 package org.opendaylight.controller.md.sal.binding.impl;
 
-import java.util.ArrayList;
-import java.util.EnumMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-
-import javax.annotation.Nullable;
-
-import org.opendaylight.controller.md.sal.common.api.TransactionStatus;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.Futures;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncTransaction;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
-import org.opendaylight.controller.md.sal.common.impl.util.compat.DataNormalizationException;
-import org.opendaylight.controller.md.sal.common.impl.util.compat.DataNormalizationOperation;
+import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
 import org.opendaylight.controller.md.sal.dom.api.DOMDataReadTransaction;
-import org.opendaylight.controller.md.sal.dom.api.DOMDataReadWriteTransaction;
-import org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction;
 import org.opendaylight.yangtools.concepts.Delegator;
+import org.opendaylight.yangtools.concepts.Identifiable;
+import org.opendaylight.yangtools.util.concurrent.MappingCheckedFuture;
 import org.opendaylight.yangtools.yang.binding.DataObject;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
-import org.opendaylight.yangtools.yang.common.RpcResult;
-import org.opendaylight.yangtools.yang.data.api.InstanceIdentifier.PathArgument;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
-import org.opendaylight.yangtools.yang.data.impl.codec.DeserializationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Function;
-import com.google.common.base.Optional;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
+abstract class AbstractForwardedTransaction<T extends AsyncTransaction<YangInstanceIdentifier, NormalizedNode<?, ?>>>
+        implements Delegator<T>, Identifiable<Object> {
 
-public class AbstractForwardedTransaction<T extends AsyncTransaction<org.opendaylight.yangtools.yang.data.api.InstanceIdentifier, NormalizedNode<?, ?>>>
-        implements Delegator<T> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractForwardedTransaction.class);
     private final T delegate;
-    private final static CacheBuilder<Object, Object> CACHE_BUILDER = CacheBuilder.newBuilder()
-            .expireAfterWrite(10, TimeUnit.MILLISECONDS).maximumSize(100);
     private final BindingToNormalizedNodeCodec codec;
-    private final EnumMap<LogicalDatastoreType, Cache<InstanceIdentifier<?>, DataObject>> cacheMap;
 
-    protected AbstractForwardedTransaction(final T delegate, final BindingToNormalizedNodeCodec codec) {
-        super();
-        this.delegate = delegate;
-        this.codec = codec;
+    public AbstractForwardedTransaction(final T delegateTx, final BindingToNormalizedNodeCodec codec) {
+        this.delegate = Preconditions.checkNotNull(delegateTx, "Delegate must not be null");
+        this.codec = Preconditions.checkNotNull(codec, "Codec must not be null");
+    }
 
-        this.cacheMap = new EnumMap<>(LogicalDatastoreType.class);
-        cacheMap.put(LogicalDatastoreType.OPERATIONAL, CACHE_BUILDER.<InstanceIdentifier<?>, DataObject> build());
-        cacheMap.put(LogicalDatastoreType.CONFIGURATION, CACHE_BUILDER.<InstanceIdentifier<?>, DataObject> build());
 
+    @Override
+    public final  Object getIdentifier() {
+        return delegate.getIdentifier();
     }
 
     @Override
-    public T getDelegate() {
+    public final  T getDelegate() {
         return delegate;
     }
 
-    protected final BindingToNormalizedNodeCodec getCodec() {
-        return codec;
-    }
-
-    protected ListenableFuture<Optional<DataObject>> transformFuture(final LogicalDatastoreType store,
-            final InstanceIdentifier<?> path, final ListenableFuture<Optional<NormalizedNode<?, ?>>> future) {
-        return Futures.transform(future, new Function<Optional<NormalizedNode<?, ?>>, Optional<DataObject>>() {
-            @Nullable
-            @Override
-            public Optional<DataObject> apply(@Nullable final Optional<NormalizedNode<?, ?>> normalizedNode) {
-                if (normalizedNode.isPresent()) {
-                    final DataObject dataObject;
-                    try {
-                        dataObject = codec.toBinding(path, normalizedNode.get());
-                    } catch (DeserializationException e) {
-                        LOG.warn("Failed to create dataobject from node {}", normalizedNode.get(), e);
-                        throw new IllegalStateException("Failed to create dataobject", e);
-                    }
-
-                    if (dataObject != null) {
-                        updateCache(store, path, dataObject);
-                        return Optional.of(dataObject);
-                    }
-                }
-                return Optional.absent();
-            }
-        });
-    }
-
-    protected void doPut(final DOMDataWriteTransaction writeTransaction, final LogicalDatastoreType store,
-            final InstanceIdentifier<?> path, final DataObject data) {
-        invalidateCache(store, path);
-        final Entry<org.opendaylight.yangtools.yang.data.api.InstanceIdentifier, NormalizedNode<?, ?>> normalized = codec
-                .toNormalizedNode(path, data);
-        writeTransaction.put(store, normalized.getKey(), normalized.getValue());
-    }
-
-    protected void doPutWithEnsureParents(final DOMDataReadWriteTransaction writeTransaction,
-            final LogicalDatastoreType store, final InstanceIdentifier<?> path, final DataObject data) {
-        invalidateCache(store, path);
-        final Entry<org.opendaylight.yangtools.yang.data.api.InstanceIdentifier, NormalizedNode<?, ?>> normalized = codec
-                .toNormalizedNode(path, data);
-
-        final org.opendaylight.yangtools.yang.data.api.InstanceIdentifier normalizedPath = normalized.getKey();
-        ensureParentsByMerge(writeTransaction, store, normalizedPath, path);
-        LOG.debug("Tx: {} : Putting data {}", getDelegate().getIdentifier(), normalizedPath);
-        writeTransaction.put(store, normalizedPath, normalized.getValue());
-    }
-
-    protected void doMergeWithEnsureParents(final DOMDataReadWriteTransaction writeTransaction,
-            final LogicalDatastoreType store, final InstanceIdentifier<?> path, final DataObject data) {
-        invalidateCache(store, path);
-        final Entry<org.opendaylight.yangtools.yang.data.api.InstanceIdentifier, NormalizedNode<?, ?>> normalized = codec
-                .toNormalizedNode(path, data);
-
-        final org.opendaylight.yangtools.yang.data.api.InstanceIdentifier normalizedPath = normalized.getKey();
-        ensureParentsByMerge(writeTransaction, store, normalizedPath, path);
-        LOG.debug("Tx: {} : Merge data {}",getDelegate().getIdentifier(),normalizedPath);
-        writeTransaction.merge(store, normalizedPath, normalized.getValue());
-    }
-
-    private void ensureParentsByMerge(final DOMDataReadWriteTransaction writeTransaction,
-            final LogicalDatastoreType store,
-            final org.opendaylight.yangtools.yang.data.api.InstanceIdentifier normalizedPath,
-            final InstanceIdentifier<?> path) {
-        List<PathArgument> currentArguments = new ArrayList<>();
-        DataNormalizationOperation<?> currentOp = codec.getDataNormalizer().getRootOperation();
-        Iterator<PathArgument> iterator = normalizedPath.getPath().iterator();
-        while (iterator.hasNext()) {
-            PathArgument currentArg = iterator.next();
-            try {
-                currentOp = currentOp.getChild(currentArg);
-            } catch (DataNormalizationException e) {
-                throw new IllegalArgumentException(String.format("Invalid child encountered in path %s", path), e);
-            }
-            currentArguments.add(currentArg);
-            org.opendaylight.yangtools.yang.data.api.InstanceIdentifier currentPath = new org.opendaylight.yangtools.yang.data.api.InstanceIdentifier(
-                    currentArguments);
-
-            final Optional<NormalizedNode<?, ?>> d;
-            try {
-                d = writeTransaction.read(store, currentPath).get();
-            } catch (InterruptedException | ExecutionException e) {
-                LOG.error("Failed to read pre-existing data from store {} path {}", store, currentPath, e);
-                throw new IllegalStateException("Failed to read pre-existing data", e);
-            }
-
-            if (!d.isPresent() && iterator.hasNext()) {
-                writeTransaction.merge(store, currentPath, currentOp.createDefault(currentArg));
-            }
-        }
-    }
-
-    protected void doMerge(final DOMDataWriteTransaction writeTransaction, final LogicalDatastoreType store,
-            final InstanceIdentifier<?> path, final DataObject data) {
-        invalidateCache(store, path);
-        final Entry<org.opendaylight.yangtools.yang.data.api.InstanceIdentifier, NormalizedNode<?, ?>> normalized = codec
-                .toNormalizedNode(path, data);
-        writeTransaction.merge(store, normalized.getKey(), normalized.getValue());
-    }
-
-    protected void doDelete(final DOMDataWriteTransaction writeTransaction, final LogicalDatastoreType store,
-            final InstanceIdentifier<?> path) {
-        invalidateCache(store, path);
-        final org.opendaylight.yangtools.yang.data.api.InstanceIdentifier normalized = codec.toNormalized(path);
-        writeTransaction.delete(store, normalized);
-    }
-
-    protected ListenableFuture<RpcResult<TransactionStatus>> doCommit(final DOMDataWriteTransaction writeTransaction) {
-        return writeTransaction.commit();
+    @SuppressWarnings("unchecked")
+    protected final <S extends AsyncTransaction<YangInstanceIdentifier, NormalizedNode<?, ?>>> S getDelegateChecked(final Class<S> txType) {
+        Preconditions.checkState(txType.isInstance(delegate));
+        return (S) delegate;
     }
 
-    protected void doCancel(final DOMDataWriteTransaction writeTransaction) {
-        writeTransaction.cancel();
-    }
-
-    protected ListenableFuture<Optional<DataObject>> doRead(final DOMDataReadTransaction readTransaction,
-            final LogicalDatastoreType store, final InstanceIdentifier<?> path) {
-        final DataObject dataObject = getFromCache(store, path);
-        if (dataObject == null) {
-            final ListenableFuture<Optional<NormalizedNode<?, ?>>> future = readTransaction.read(store,
-                    codec.toNormalized(path));
-            return transformFuture(store, path, future);
-        } else {
-            return Futures.immediateFuture(Optional.of(dataObject));
-        }
-    }
-
-    private DataObject getFromCache(final LogicalDatastoreType store, final InstanceIdentifier<?> path) {
-        Cache<InstanceIdentifier<?>, DataObject> cache = cacheMap.get(store);
-        if (cache != null) {
-            return cache.getIfPresent(path);
-        }
-        return null;
+    protected final BindingToNormalizedNodeCodec getCodec() {
+        return codec;
     }
 
-    private void updateCache(final LogicalDatastoreType store, final InstanceIdentifier<?> path,
-            final DataObject dataObject) {
-        // Check if cache exists. If not create one.
-        Cache<InstanceIdentifier<?>, DataObject> cache = cacheMap.get(store);
-        if (cache == null) {
-            cache = CacheBuilder.newBuilder().maximumSize(1000).expireAfterWrite(1, TimeUnit.MINUTES).build();
-
-        }
+    protected final <D extends DataObject> CheckedFuture<Optional<D>,ReadFailedException> doRead(
+            final DOMDataReadTransaction readTx, final LogicalDatastoreType store,
+            final InstanceIdentifier<D> path) {
+        Preconditions.checkArgument(!path.isWildcarded(), "Invalid read of wildcarded path %s", path);
 
-        cache.put(path, dataObject);
+        return MappingCheckedFuture.create(
+                    Futures.transform(readTx.read(store, codec.toNormalized(path)),
+                                      codec.deserializeFunction(path)),
+                    ReadFailedException.MAPPER);
     }
-
-    private void invalidateCache(final LogicalDatastoreType store, final InstanceIdentifier<?> path) {
-        // FIXME: Optimization: invalidate only parents and children of path
-        Cache<InstanceIdentifier<?>, DataObject> cache = cacheMap.get(store);
-        cache.invalidateAll();
-        LOG.trace("Cache invalidated");
-    }
-
 }