You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nlpcraft.apache.org by se...@apache.org on 2022/08/24 19:39:15 UTC

[incubator-nlpcraft] 02/02: IDL fragments related fixes.

This is an automated email from the ASF dual-hosted git repository.

sergeykamov pushed a commit to branch NLPCRAFT-500
in repository https://gitbox.apache.org/repos/asf/incubator-nlpcraft.git

commit a4d318ed25a47e7447de420d32db5cbd46f35926
Author: Sergey Kamov <sk...@gmail.com>
AuthorDate: Wed Aug 24 22:30:56 2022 +0300

    IDL fragments related fixes.
---
 .../nlpcraft/internal/impl/NCModelScanner.scala    |  36 +++++---
 .../internal/intent/compiler/NCIDLCompiler.scala   |  73 ++++++++++-----
 .../internal/intent/compiler/NCIDLGlobal.scala     |  83 -----------------
 .../intent/compiler/NCIDLCompilerSpec.scala        |  14 ++-
 .../intent/compiler/NCIDLFragmentsSpec.scala       | 101 +++++++++++++++++++++
 .../intent/compiler/functions/NCIDLFunctions.scala |   5 +-
 6 files changed, 184 insertions(+), 128 deletions(-)

diff --git a/nlpcraft/src/main/scala/org/apache/nlpcraft/internal/impl/NCModelScanner.scala b/nlpcraft/src/main/scala/org/apache/nlpcraft/internal/impl/NCModelScanner.scala
index 563045e2..f957c198 100644
--- a/nlpcraft/src/main/scala/org/apache/nlpcraft/internal/impl/NCModelScanner.scala
+++ b/nlpcraft/src/main/scala/org/apache/nlpcraft/internal/impl/NCModelScanner.scala
@@ -412,6 +412,8 @@ object NCModelScanner extends LazyLogging:
     def scan(mdl: NCModel): Seq[NCModelIntent] =
         require(mdl != null)
 
+        var compiler = new NCIDLCompiler(mdl.getConfig)
+
         val cfg = mdl.getConfig
         lazy val z = s"mdlId=${cfg.getId}"
         val intentsMtds = mutable.HashMap.empty[Method, IntentHolder]
@@ -437,11 +439,17 @@ object NCModelScanner extends LazyLogging:
             val errAnns = mutable.ArrayBuffer.empty[NCIntent]
             val intents = mutable.ArrayBuffer.empty[NCIDLIntent]
 
-            def addIntents(ann: NCIntent) = intents ++= NCIDLCompiler.compile(ann.value, cfg, origin)
+            def addIntents(ann: NCIntent) = intents ++= compiler.compile(ann.value, origin)
 
             // 1. First pass.
-            for (ann <- anns) try addIntents(ann)
-            catch case _: NCException => errAnns += ann
+            for (ann <- anns)
+                val copy = compiler.clone()
+                try
+                    addIntents(ann)
+                catch
+                    case _: NCException =>
+                        compiler = copy
+                        errAnns += ann
 
             // 2. Second pass.
             for (ann <- errAnns) addIntents(ann)
@@ -464,18 +472,24 @@ object NCModelScanner extends LazyLogging:
         def scan(obj: AnyRef): Unit =
             objs += obj
             processClassAnnotations(obj.getClass)
-            val methods = getAllMethods(obj)
-
-            // // Collects intents for each method.
-            for (mtd <- methods)
-                val anns = mtd.getAnnotationsByType(CLS_INTENT)
-                val intents = addIntent2Phases(anns, method2Str(mtd))
-
-                for (intent <- intents) addIntent(intent, mtd, obj)
 
             // Scans annotated fields.
             for (f <- getAllFields(obj) if f.isAnnotationPresent(CLS_INTENT_OBJ)) scan(getFieldObject(cfg, f, obj))
 
+            val methods = getAllMethods(obj)
+
+            // // Collects intents for each method.
+            for (mtd <- methods)
+                val copy = compiler.clone()
+
+                try
+                    for (
+                        ann <- mtd.getAnnotationsByType(CLS_INTENT);
+                        intent <- compiler.compile(ann.value, method2Str(mtd))
+                    )
+                        addDecl(intent)
+                        addIntent(intent, mtd, obj)
+                finally compiler = copy
         scan(mdl)
 
         // Second phase. For model and all its references scans each method and finds intents references (NCIntentRef)
diff --git a/nlpcraft/src/main/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLCompiler.scala b/nlpcraft/src/main/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLCompiler.scala
index 9f8a1ee9..fb82548e 100644
--- a/nlpcraft/src/main/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLCompiler.scala
+++ b/nlpcraft/src/main/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLCompiler.scala
@@ -32,12 +32,39 @@ import java.io.*
 import java.net.*
 import java.util.Optional
 import java.util.regex.*
+import scala.collection.concurrent.TrieMap
 import scala.collection.mutable
 import scala.jdk.CollectionConverters.*
 
-object NCIDLCompiler extends LazyLogging:
+class NCIDLCompiler(cfg: NCModelConfig) extends LazyLogging with scala.collection.mutable.Cloneable[NCIDLCompiler]:
     // Compiler caches.
-    private val cache = new mutable.HashMap[String, Set[NCIDLIntent]]
+    private val cacheIntents = new mutable.HashMap[String, Set[NCIDLIntent]]
+    private val fragCache = TrieMap.empty[String /* Model ID. */ , mutable.Map[String, NCIDLFragment]]
+    private val importCache = mutable.HashSet.empty[String]
+
+    private def getFragment(mdlId: String, fragId: String): Option[NCIDLFragment] = fragCache.get(mdlId).flatMap(_.get(fragId))
+
+    private def addFragment(mdlId: String, frag: NCIDLFragment): Unit =
+        fragCache.getOrElse(mdlId, {
+            val m = mutable.HashMap.empty[String, NCIDLFragment]
+
+            fragCache += mdlId -> m
+
+            m
+        }) += (frag.id -> frag)
+
+    private def addImport(imp: String): Unit = importCache.synchronized {
+        importCache += imp
+    }
+
+    /**
+      *
+      * @param imp
+      * @return
+      */
+    private def hasImport(imp: String): Boolean = importCache.synchronized {
+        importCache.contains(imp)
+    }
 
     /**
       *
@@ -181,12 +208,12 @@ object NCIDLCompiler extends LazyLogging:
 
         override def exitFragId(ctx: IDP.FragIdContext): Unit =
             fragId = ctx.id().getText
-            if NCIDLGlobal.getFragment(mdlCfg.getId, fragId).isDefined then SE(s"Duplicate fragment ID: $fragId")(ctx.id())
+            if getFragment(mdlCfg.getId, fragId).isDefined then SE(s"Duplicate fragment ID: $fragId")(ctx.id())
 
         override def exitFragRef(ctx: IDP.FragRefContext): Unit =
             val id = ctx.id().getText
 
-            NCIDLGlobal.getFragment(mdlCfg.getId, id) match
+            getFragment(mdlCfg.getId, id) match
                 case Some(frag) =>
                     val meta = if fragMeta == null then Map.empty[String, Any] else fragMeta
                     for (fragTerm <- frag.terms)
@@ -256,7 +283,7 @@ object NCIDLCompiler extends LazyLogging:
             }
 
         override def exitFrag(ctx: IDP.FragContext): Unit =
-            NCIDLGlobal.addFragment(mdlCfg.getId, NCIDLFragment(fragId, terms.toList))
+            addFragment(mdlCfg.getId, NCIDLFragment(fragId, terms.toList))
             terms.clear()
             fragId = null
 
@@ -290,9 +317,9 @@ object NCIDLCompiler extends LazyLogging:
         override def exitImprt(ctx: IDP.ImprtContext): Unit =
                 val x = NCUtils.trimQuotes(ctx.qstring().getText)
 
-                if NCIDLGlobal.hasImport(x) then logger.warn(s"Ignoring already processed IDL import '$x' in: $origin")
+                if hasImport(x) then logger.warn(s"Ignoring already processed IDL import '$x' in: $origin")
                 else
-                    NCIDLGlobal.addImport(x)
+                   addImport(x)
 
                     var imports: Set[NCIDLIntent] = null
                     val file = new File(x)
@@ -300,20 +327,20 @@ object NCIDLCompiler extends LazyLogging:
                     // First, try absolute path.
                     if file.exists() then
                         val idl = NCUtils.readFile(file).mkString("\n")
-                        imports = NCIDLCompiler.compile(idl, mdlCfg, x)
+                        imports = compile(idl, x)
 
                     // Second, try as a classloader resource.
                     if imports == null then
                         val in = mdlCfg.getClass.getClassLoader.getResourceAsStream(x)
                         if (in != null)
                             val idl = NCUtils.readStream(in).mkString("\n")
-                            imports = NCIDLCompiler.compile(idl, mdlCfg, x)
+                            imports = compile(idl, x)
 
                     // Finally, try as URL resource.
                     if imports == null then
                         try
                             val idl = NCUtils.readStream(new URL(x).openStream()).mkString("\n")
-                            imports = NCIDLCompiler.compile(idl, mdlCfg, x )
+                            imports = compile(idl,  x)
                         catch case _: Exception => throw newRuntimeError(s"Invalid or unknown import location: $x")(ctx.qstring())
 
                     require(imports != null)
@@ -436,22 +463,17 @@ object NCIDLCompiler extends LazyLogging:
     /**
       *
       * @param idl
-      * @param mdlCfg
       * @param srcName
       * @return
       */
-    private def parseIntents(
-        idl: String,
-        mdlCfg: NCModelConfig,
-        srcName: String
-    ): Set[NCIDLIntent] =
+    private def parseIntents(idl: String, srcName: String): Set[NCIDLIntent] =
         require(idl != null)
-        require(mdlCfg != null)
+        require(cfg != null)
         require(srcName != null)
 
         val x = idl.strip()
-        val intents: Set[NCIDLIntent] = cache.getOrElseUpdate(x, {
-            val (fsm, parser) = antlr4Armature(x, mdlCfg, srcName)
+        val intents: Set[NCIDLIntent] = cacheIntents.getOrElseUpdate(x, {
+            val (fsm, parser) = antlr4Armature(x, cfg, srcName)
 
             // Parse the input IDL and walk built AST.
             (new ParseTreeWalker).walk(fsm, parser.idl())
@@ -487,10 +509,17 @@ object NCIDLCompiler extends LazyLogging:
       * map keyed by model ID. Only intents are returned, if any.
       *
       * @param idl Intent IDL to compile.
-      * @param mdlCfg Model configuration.
       * @param origin Optional source name.
       * @return
       */
     @throws[NCException]
-    def compile(idl: String, mdlCfg: NCModelConfig, origin: String): Set[NCIDLIntent] =
-        parseIntents(idl, mdlCfg, origin)
+    def compile(idl: String, origin: String): Set[NCIDLIntent] = parseIntents(idl,  origin)
+
+    override def clone(): NCIDLCompiler =
+        val copy = new NCIDLCompiler(cfg)
+        this.cacheIntents ++= cacheIntents.clone()
+        this.importCache ++= importCache.clone()
+        this.fragCache ++= fragCache.clone()
+        copy
+
+
diff --git a/nlpcraft/src/main/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLGlobal.scala b/nlpcraft/src/main/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLGlobal.scala
deleted file mode 100644
index 125dfb4b..00000000
--- a/nlpcraft/src/main/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLGlobal.scala
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.nlpcraft.internal.intent.compiler
-
-import scala.collection.concurrent.TrieMap
-import scala.collection.mutable
-
-/**
-  * Global IDL compiler state.
-  */
-object NCIDLGlobal:
-    private final val fragCache = TrieMap.empty[String /* Model ID. */ , mutable.Map[String, NCIDLFragment]]
-    private final val importCache = mutable.HashSet.empty[String]
-
-    /**
-      *
-      */
-    def clearAllCaches(): Unit =
-        fragCache.clear()
-        clearImportCache()
-
-    /**
-      *
-      */
-    private def clearImportCache(): Unit = importCache.synchronized { importCache.clear() }
-
-    /**
-      *
-      * @param mdlId
-      */
-    def clearCache(mdlId: String): Unit = fragCache += mdlId -> mutable.HashMap.empty[String, NCIDLFragment]
-
-    /**
-      *
-      * @param imp
-      */
-    def addImport(imp: String): Unit = importCache.synchronized { importCache += imp }
-
-    /**
-      *
-      * @param imp
-      * @return
-      */
-    def hasImport(imp: String): Boolean = importCache.synchronized { importCache.contains(imp) }
-
-    /**
-      *
-      * @param mdlId
-      * @param frag
-      */
-    def addFragment(mdlId: String, frag: NCIDLFragment): Unit =
-        fragCache.getOrElse(mdlId, {
-            val m = mutable.HashMap.empty[String, NCIDLFragment]
-
-            fragCache += mdlId -> m
-
-            m
-        }) += (frag.id -> frag)
-
-    /**
-      *
-      * @param mdlId
-      * @param fragId
-      * @return
-      */
-    def getFragment(mdlId: String, fragId: String): Option[NCIDLFragment] =
-        fragCache.get(mdlId).flatMap(_.get(fragId))
-
diff --git a/nlpcraft/src/test/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLCompilerSpec.scala b/nlpcraft/src/test/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLCompilerSpec.scala
index b5d490fc..2b4bff1b 100644
--- a/nlpcraft/src/test/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLCompilerSpec.scala
+++ b/nlpcraft/src/test/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLCompilerSpec.scala
@@ -27,13 +27,15 @@ import org.junit.jupiter.api.Test
 class NCIDLCompilerSpec:
     private final val MODEL_ID = "test.mdl.id"
 
+    private final val compiler = new NCIDLCompiler(CFG)
+
     /**
      *
      * @param idl
      */
     private def checkCompileOk(idl: String): Unit =
         try
-            NCIDLCompiler.compile(idl, CFG, MODEL_ID)
+            compiler.compile(idl,  MODEL_ID)
             assert(true)
         catch case e: Exception => assert(assertion = false, e)
 
@@ -43,7 +45,7 @@ class NCIDLCompilerSpec:
      */
     private def checkCompileError(txt: String): Unit =
         try
-            NCIDLCompiler.compile(txt, CFG, MODEL_ID)
+            compiler.compile(txt, MODEL_ID)
             assert(false)
         catch
             case e: NCException =>
@@ -53,8 +55,6 @@ class NCIDLCompilerSpec:
     @Test
     @throws[NCException]
     def testInlineCompileOk(): Unit =
-        NCIDLGlobal.clearCache(MODEL_ID)
-    
         checkCompileOk(
             """
               |import('org/apache/nlpcraft/internal/intent/compiler/test_ok.idl')
@@ -120,8 +120,6 @@ class NCIDLCompilerSpec:
     @Test
     @throws[NCException]
     def testInlineCompileFail(): Unit =
-        NCIDLGlobal.clearCache(MODEL_ID)
-
         checkCompileError(
             """
               |intent=i1
@@ -292,13 +290,13 @@ class NCIDLCompilerSpec:
         )
 
     @Test
-    def testImport(): Unit = require(NCIDLCompiler.compile("import('scan/idl.idl')", CFG, "test-origin").size == 1)
+    def testImport(): Unit = require(compiler.compile("import('scan/idl.idl')", "test-origin").size == 1)
 
     @Test
     def testEmpty(): Unit =
         def test0(idl: String): Unit =
             try
-                NCIDLCompiler.compile(idl, CFG, "test-origin")
+                compiler.compile(idl, "test-origin")
                 require(true)
             catch
                 case e: NCException => println(s"Unexpected error: $e")
diff --git a/nlpcraft/src/test/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLFragmentsSpec.scala b/nlpcraft/src/test/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLFragmentsSpec.scala
new file mode 100644
index 00000000..164fa271
--- /dev/null
+++ b/nlpcraft/src/test/scala/org/apache/nlpcraft/internal/intent/compiler/NCIDLFragmentsSpec.scala
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nlpcraft.internal.intent.compiler
+
+import org.apache.nlpcraft.annotations.NCIntent
+import org.apache.nlpcraft.*
+import org.apache.nlpcraft.internal.impl.NCModelScanner
+import org.apache.nlpcraft.nlp.util.*
+import org.junit.jupiter.api.Test
+
+import scala.util.Using
+
+class NCIDLFragmentsSpec:
+    private val PL = mkEnPipeline
+
+    private def mkCfg(id: String): NCModelConfig = NCModelConfig(id, "test", "1.0", desc = "Test", orig = "Test")
+
+    // Normal models.
+
+    // Fragment. One annotations order.
+    @NCIntent("fragment=f term(city)~{# == 'opennlp:location'}")
+    @NCIntent("intent=intent2 term~{# == 'x:time'} fragment(f)")
+    class M1 extends NCModelAdapter(mkCfg("m1"), PL)
+
+    // Fragment. Another annotations order.
+    @NCIntent("intent=intent2 term~{# == 'x:time'} fragment(f)")
+    @NCIntent("fragment=f term(city)~{# == 'opennlp:location'}")
+    class M2 extends NCModelAdapter(mkCfg("m2"), PL)
+
+    // Fragment. Reference from method to class.
+    @NCIntent("fragment=f term(city)~{# == 'opennlp:location'}")
+    class M3 extends NCModelAdapter(mkCfg("m3"), PL):
+        @NCIntent("intent=intent2 term~{# == 'x:time'} fragment(f)")
+        private def m(ctx: NCContext, im: NCIntentMatch): NCResult = null
+
+    // Fragment. Reference from method (inside a).
+    class M4 extends NCModelAdapter(mkCfg("m4"), PL) :
+        @NCIntent("fragment=f term(city)~{# == 'opennlp:location'} intent=intent2 term~{# == 'x:time'} fragment(f)")
+        private def m(ctx: NCContext, im: NCIntentMatch): NCResult = null
+
+    // Bad models.
+
+    // Invalid fragment.
+    @NCIntent("intent=intent2 term~{# == 'x:time'} fragment(f)")
+    class E1 extends NCModelAdapter(mkCfg("e1"), PL)
+
+    class E2 extends NCModelAdapter(mkCfg("e2"), PL):
+        @NCIntent("fragment=f term(city)~{# == 'opennlp:location'} intent=intent1 term~{# == 'x:time'} fragment(f)")
+        private def m1(ctx: NCContext, im: NCIntentMatch): NCResult = null
+
+        @NCIntent("intent=intent2 term~{# == 'x:time'} fragment(f)")
+        private def m2(ctx: NCContext, im: NCIntentMatch): NCResult = null
+
+    class E3 extends NCModelAdapter(mkCfg("e3"), PL):
+        @NCIntent("fragment=f term(city)~{# == 'opennlp:location'} intent=intent1 term~{# == 'x:time'} fragment(f)")
+        private def m2(ctx: NCContext, im: NCIntentMatch): NCResult = null
+
+        @NCIntent("intent=intent2 term~{# == 'x:time'} fragment(f)")
+        private def m1(ctx: NCContext, im: NCIntentMatch): NCResult = null
+
+    private def testOk(mdls: NCModel*): Unit =
+        for (mdl <- mdls)
+            NCModelScanner.scan(mdl)
+            println(s"Model valid: '${mdl.getConfig.getId}'")
+
+    private def testError(mdls: NCModel*): Unit =
+        for (mdl <- mdls)
+            try
+                NCModelScanner.scan(mdl)
+                require(false, s"Model shouldn't be scanned: ${mdl.getConfig.getId}")
+            catch case e: NCException => println(s"Model '${mdl.getConfig.getId}' expected error: '${e.getMessage}'")
+
+    @Test
+    def test(): Unit =
+        testOk(
+            new M1(),
+            new M2(),
+            new M3(),
+            new M4()
+        )
+
+        testError(
+            new E1(),
+            new E2(),
+            new E3()
+        )
\ No newline at end of file
diff --git a/nlpcraft/src/test/scala/org/apache/nlpcraft/internal/intent/compiler/functions/NCIDLFunctions.scala b/nlpcraft/src/test/scala/org/apache/nlpcraft/internal/intent/compiler/functions/NCIDLFunctions.scala
index f9fb7655..bc0a5b61 100644
--- a/nlpcraft/src/test/scala/org/apache/nlpcraft/internal/intent/compiler/functions/NCIDLFunctions.scala
+++ b/nlpcraft/src/test/scala/org/apache/nlpcraft/internal/intent/compiler/functions/NCIDLFunctions.scala
@@ -51,7 +51,7 @@ private[functions] object NCIDLFunctions:
         entitiesUsed: Option[Int] = None
     ):
         lazy val term: NCIDLTerm =
-            val intents = NCIDLCompiler.compile(s"intent=i term(t)={$truth}", idlCtx.mdlCfg, MODEL_ID)
+            val intents = new NCIDLCompiler(idlCtx.mdlCfg).compile(s"intent=i term(t)={$truth}", MODEL_ID)
 
             require(intents.size == 1)
             require(intents.head.terms.sizeIs == 1)
@@ -164,9 +164,6 @@ import org.apache.nlpcraft.internal.intent.compiler.functions.NCIDLFunctions.*
   * Tests for IDL functions.
   */
 private[functions] trait NCIDLFunctions:
-    @BeforeEach
-    def before(): Unit = NCIDLGlobal.clearCache(MODEL_ID)
-
     /**
       *
       * @param funcs