BUG-997: Evolve the SchemaRegistry concepts
[yangtools.git] / yang / yang-parser-impl / src / main / java / org / opendaylight / yangtools / yang / parser / repo / URLSchemaContextResolver.java
index 8ee18e58a5a562d40bb65e0de26fef2b9ffebb58..47de6b1a540b5bd1b0a14014ce9a5364447977b3 100644 (file)
@@ -9,155 +9,78 @@ package org.opendaylight.yangtools.yang.parser.repo;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.common.base.Function;
+import com.google.common.annotations.Beta;
 import com.google.common.base.Objects.ToStringHelper;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.ImmutableMultimap;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Maps.EntryTransformer;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Multimaps;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.collect.ImmutableList;
 import com.google.common.util.concurrent.CheckedFuture;
-import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URL;
 import java.util.Collection;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.atomic.AtomicReference;
 
-import javax.annotation.concurrent.GuardedBy;
-import javax.annotation.concurrent.ThreadSafe;
-
-import org.antlr.v4.runtime.ParserRuleContext;
-import org.antlr.v4.runtime.tree.ParseTreeWalker;
-import org.opendaylight.yangtools.concepts.AbstractObjectRegistration;
-import org.opendaylight.yangtools.concepts.ObjectRegistration;
-import org.opendaylight.yangtools.yang.model.api.Module;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.yang.model.parser.api.YangSyntaxErrorException;
+import org.opendaylight.yangtools.yang.model.repo.api.MissingSchemaSourceException;
+import org.opendaylight.yangtools.yang.model.repo.api.SchemaContextFactory;
+import org.opendaylight.yangtools.yang.model.repo.api.SchemaRepository;
+import org.opendaylight.yangtools.yang.model.repo.api.SchemaResolutionException;
+import org.opendaylight.yangtools.yang.model.repo.api.SchemaSourceException;
+import org.opendaylight.yangtools.yang.model.repo.api.SchemaSourceFilter;
 import org.opendaylight.yangtools.yang.model.repo.api.SourceIdentifier;
 import org.opendaylight.yangtools.yang.model.repo.api.YangTextSchemaSource;
-import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceTransformationException;
-import org.opendaylight.yangtools.yang.parser.builder.impl.ModuleBuilder;
-import org.opendaylight.yangtools.yang.parser.impl.YangParserImpl;
-import org.opendaylight.yangtools.yang.parser.impl.YangParserListenerImpl;
-import org.opendaylight.yangtools.yang.parser.impl.util.YangModelDependencyInfo;
+import org.opendaylight.yangtools.yang.model.repo.spi.PotentialSchemaSource;
+import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceProvider;
+import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceRegistration;
+import org.opendaylight.yangtools.yang.model.repo.spi.SchemaSourceRegistry;
 import org.opendaylight.yangtools.yang.parser.util.ASTSchemaSource;
 import org.opendaylight.yangtools.yang.parser.util.TextToASTTransformer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@ThreadSafe
-public class URLSchemaContextResolver {
+@Beta
+public class URLSchemaContextResolver implements SchemaSourceProvider<YangTextSchemaSource> {
     private static final Logger LOG = LoggerFactory.getLogger(URLSchemaContextResolver.class);
-    private static final Function<ASTSchemaSource, YangModelDependencyInfo> EXTRACT_DEPINFO = new Function<ASTSchemaSource, YangModelDependencyInfo>() {
-        @Override
-        public YangModelDependencyInfo apply(final ASTSchemaSource input) {
-            return input.getDependencyInformation();
-        }
-    };
-    private static final EntryTransformer<SourceIdentifier, Collection<YangModelDependencyInfo>, YangModelDependencyInfo> SQUASH_DEPINFO =
-            new EntryTransformer<SourceIdentifier, Collection<YangModelDependencyInfo>, YangModelDependencyInfo>() {
-        @Override
-        public YangModelDependencyInfo transformEntry(final SourceIdentifier key, final Collection<YangModelDependencyInfo> value) {
-            // FIXME: validate that all the info objects are the same
-            return value.iterator().next();
-        }
-    };
-    private static final Function<ASTSchemaSource, ParserRuleContext> EXTRACT_AST = new Function<ASTSchemaSource, ParserRuleContext>() {
-        @Override
-        public ParserRuleContext apply(final ASTSchemaSource input) {
-            return input.getAST();
-        }
-    };
-    private static final EntryTransformer<SourceIdentifier, Collection<ParserRuleContext>, ParserRuleContext> SQUASH_AST =
-            new EntryTransformer<SourceIdentifier, Collection<ParserRuleContext>, ParserRuleContext>() {
-        @Override
-        public ParserRuleContext transformEntry(final SourceIdentifier key, final Collection<ParserRuleContext> value) {
-            // FIXME: validate that all the info objects are the same
-            return value.iterator().next();
-        }
-    };
-
-    @GuardedBy("this")
-    private final Multimap<SourceIdentifier, ASTSchemaSource> resolvedRegs = ArrayListMultimap.create();
-    private final AtomicReference<Optional<SchemaContext>> currentSchemaContext = new AtomicReference<>(Optional.<SchemaContext>absent());
-    private final Queue<URLRegistration> outstandingRegs = new ConcurrentLinkedQueue<>();
-    private final TextToASTTransformer transformer;
-    @GuardedBy("this")
-    private Object version = new Object();
-    @GuardedBy("this")
-    private Object contextVersion = version;
-
-    private final class URLRegistration extends AbstractObjectRegistration<URL> {
-        @GuardedBy("this")
-        private CheckedFuture<ASTSchemaSource, SchemaSourceTransformationException> future;
-        @GuardedBy("this")
-        private ASTSchemaSource result;
-
-        protected URLRegistration(final URL url, final CheckedFuture<ASTSchemaSource, SchemaSourceTransformationException> future) {
-            super(url);
-            this.future = Preconditions.checkNotNull(future);
-        }
-
-        private synchronized boolean setResult(final ASTSchemaSource result) {
-            if (future != null) {
-                this.result = result;
-                return true;
-            } else {
-                return false;
-            }
-        }
 
-        @Override
-        protected void removeRegistration() {
-            // Cancel the future, but it may already be completing
-            future.cancel(false);
-
-            synchronized (this) {
-                future = null;
-                outstandingRegs.remove(this);
-                if (result != null) {
-                    removeSchemaSource(result);
-                }
-            }
-        }
-    }
-
-    private URLSchemaContextResolver(final TextToASTTransformer transformer) {
-        this.transformer = Preconditions.checkNotNull(transformer);
+    private final Cache<SourceIdentifier, YangTextSchemaSource> sources = CacheBuilder.newBuilder().build();
+    private final Collection<SourceIdentifier> requiredSources = new ConcurrentLinkedDeque<>();
+    private final AtomicReference<Optional<SchemaContext>> currentSchemaContext =
+            new AtomicReference<>(Optional.<SchemaContext>absent());
+    private final SchemaSourceRegistry registry;
+    private final SchemaRepository repository;
+    private volatile Object version = new Object();
+    private volatile Object contextVersion = version;
+
+    private URLSchemaContextResolver(final SchemaRepository repository, final SchemaSourceRegistry registry) {
+        this.repository = Preconditions.checkNotNull(repository);
+        this.registry = Preconditions.checkNotNull(registry);
     }
 
     public static URLSchemaContextResolver create(final String name) {
-        final ThreadFactory f = new ThreadFactoryBuilder().setDaemon(true).setNameFormat(name + "yangparser-%d").build();
-        final ListeningExecutorService s = MoreExecutors.listeningDecorator(Executors.newSingleThreadExecutor(f));
-
-        return new URLSchemaContextResolver(TextToASTTransformer.create(s));
+        final SharedSchemaRepository sharedRepo = new SharedSchemaRepository(name);
+        return new URLSchemaContextResolver(sharedRepo, sharedRepo);
     }
 
     /**
      * Register a URL hosting a YANG Text file.
      *
      * @param url URL
+     * @throws YangSyntaxErrorException When the YANG file is syntactically invalid
+     * @throws IOException when the URL is not readable
+     * @throws SchemaSourceException When parsing encounters general error
      */
-    public ObjectRegistration<URL> registerSource(final URL url) {
+    public URLRegistration registerSource(final URL url) throws SchemaSourceException, IOException, YangSyntaxErrorException {
         checkArgument(url != null, "Supplied URL must not be null");
 
-        final SourceIdentifier id = SourceIdentifier.create(url.getFile().toString(), Optional.<String>absent());
-        final YangTextSchemaSource text = new YangTextSchemaSource(id) {
+        final SourceIdentifier guessedId = new SourceIdentifier(url.getFile(), Optional.<String>absent());
+        final YangTextSchemaSource text = new YangTextSchemaSource(guessedId) {
             @Override
             public InputStream openStream() throws IOException {
                 return url.openStream();
@@ -169,39 +92,27 @@ public class URLSchemaContextResolver {
             }
         };
 
-        final CheckedFuture<ASTSchemaSource, SchemaSourceTransformationException> ast = transformer.transformSchemaSource(text);
-        final URLRegistration reg = new URLRegistration(url, ast);
-        outstandingRegs.add(reg);
+        final ASTSchemaSource ast = TextToASTTransformer.TRANSFORMATION.apply(text).checkedGet();
+        LOG.trace("Resolved URL {} to source {}", url, ast);
 
-        Futures.addCallback(ast, new FutureCallback<ASTSchemaSource>() {
-            @Override
-            public void onSuccess(final ASTSchemaSource result) {
-                LOG.trace("Resolved URL {} to source {}", url, result);
+        final SourceIdentifier resolvedId = ast.getIdentifier();
+        final SchemaSourceRegistration<YangTextSchemaSource> reg = registry.registerSchemaSource(this,
+                PotentialSchemaSource.create(resolvedId, YangTextSchemaSource.class, 0));
 
-                outstandingRegs.remove(reg);
-                if (reg.setResult(result)) {
-                    addSchemaSource(result);
-                }
-            }
+        requiredSources.add(resolvedId);
+        LOG.trace("Added source {} to schema context requirements", resolvedId);
+        version = new Object();
 
+        return new AbstractURLRegistration(text) {
             @Override
-            public void onFailure(final Throwable t) {
-                LOG.warn("Failed to parse YANG text from {}, ignoring it", url, t);
-                outstandingRegs.remove(reg);
+            protected void removeRegistration() {
+                requiredSources.remove(resolvedId);
+                LOG.trace("Removed source {} from schema context requirements", resolvedId);
+                version = new Object();
+                reg.close();
+                sources.invalidate(resolvedId);
             }
-        });
-
-        return reg;
-    }
-
-    private synchronized void addSchemaSource(final ASTSchemaSource src) {
-        resolvedRegs.put(src.getIdentifier(), src);
-        version = new Object();
-    }
-
-    private synchronized void removeSchemaSource(final ASTSchemaSource src) {
-        resolvedRegs.put(src.getIdentifier(), src);
-        version = new Object();
+        };
     }
 
     /**
@@ -210,64 +121,57 @@ public class URLSchemaContextResolver {
      *         new schema context was successfully built.
      */
     public Optional<SchemaContext> getSchemaContext() {
-        while (true) {
-            Optional<SchemaContext> result;
-            final Multimap<SourceIdentifier, ASTSchemaSource> sources;
-            final Object v;
-            synchronized (this) {
-                result = currentSchemaContext.get();
-                if (version == contextVersion) {
-                    return result;
+        final SchemaContextFactory factory = repository.createSchemaContextFactory(SchemaSourceFilter.ALWAYS_ACCEPT);
+        Optional<SchemaContext> sc;
+        Object v;
+        do {
+            // Spin get stable context version
+            Object cv;
+            do {
+                cv = contextVersion;
+                sc = currentSchemaContext.get();
+                if (version == cv) {
+                    return sc;
                 }
+            } while (cv != contextVersion);
 
-                sources = ImmutableMultimap.copyOf(resolvedRegs);
+            // Version has been updated
+            Collection<SourceIdentifier> sources;
+            do {
                 v = version;
-            }
-
-            if (!sources.isEmpty()) {
-                final Map<SourceIdentifier, YangModelDependencyInfo> deps =
-                        Maps.transformEntries(Multimaps.transformValues(sources, EXTRACT_DEPINFO).asMap(), SQUASH_DEPINFO);
-
-                LOG.debug("Resolving dependency reactor {}", deps);
-                final DependencyResolver res = DependencyResolver.create(deps);
-                if (!res.getUnresolvedSources().isEmpty()) {
-                    LOG.debug("Omitting models {} due to unsatisfied imports {}", res.getUnresolvedSources(), res.getUnsatisfiedImports());
+                sources = ImmutableList.copyOf(requiredSources);
+            } while (v != version);
+
+            while (true) {
+                final CheckedFuture<SchemaContext, SchemaResolutionException> f = factory.createSchemaContext(sources);
+                try {
+                    sc = Optional.of(f.checkedGet());
+                    break;
+                } catch (SchemaResolutionException e) {
+                    LOG.info("Failed to fully assemble schema context for {}", sources, e);
+                    sources = e.getResolvedSources();
                 }
-
-                final Map<SourceIdentifier, ParserRuleContext> asts =
-                        Maps.transformEntries(Multimaps.transformValues(sources, EXTRACT_AST).asMap(), SQUASH_AST);
-
-                final ParseTreeWalker walker = new ParseTreeWalker();
-                final Map<SourceIdentifier, ModuleBuilder> sourceToBuilder = new LinkedHashMap<>();
-
-                for (Entry<SourceIdentifier, ParserRuleContext> entry : asts.entrySet()) {
-                    final YangParserListenerImpl yangModelParser = new YangParserListenerImpl(entry.getKey().getName());
-                    walker.walk(yangModelParser, entry.getValue());
-                    ModuleBuilder moduleBuilder = yangModelParser.getModuleBuilder();
-
-                    moduleBuilder.setSource(sources.get(entry.getKey()).iterator().next().getYangText());
-                    sourceToBuilder.put(entry.getKey(), moduleBuilder);
-                }
-                LOG.debug("Modules ready for integration");
-
-                final YangParserImpl parser = YangParserImpl.getInstance();
-                final Collection<Module> modules = parser.buildModules(sourceToBuilder.values());
-                LOG.debug("Integrated cross-references modules");
-
-                result = Optional.of(parser.assembleContext(modules));
-            } else {
-                result = Optional.absent();
             }
 
             synchronized (this) {
-                if (v == version) {
-                    currentSchemaContext.set(result);
-                    contextVersion = version;
-                    return result;
+                if (contextVersion == cv) {
+                    currentSchemaContext.set(sc);
+                    contextVersion = v;
                 }
-
-                LOG.debug("Context version {} expected {}, retry", version, v);
             }
+        } while (version == v);
+
+        return sc;
+    }
+
+    @Override
+    public CheckedFuture<YangTextSchemaSource, SchemaSourceException> getSource(final SourceIdentifier sourceIdentifier) {
+        final YangTextSchemaSource ret = sources.getIfPresent(sourceIdentifier);
+        if (ret == null) {
+            return Futures.<YangTextSchemaSource, SchemaSourceException>immediateFailedCheckedFuture(
+                    new MissingSchemaSourceException("URL for " + sourceIdentifier + " not registered"));
         }
+
+        return Futures.immediateCheckedFuture(ret);
     }
 }