diff --git a/api-check-ignore.xml b/api-check-ignore.xml
index f62f6eef..16a4f491 100644
--- a/api-check-ignore.xml
+++ b/api-check-ignore.xml
@@ -43,4 +43,12 @@
7002
*
+
+ org/camunda/dmn/DmnEngine$Configuration
+ *Configuration*
+
+ 7004
+ 3
+ 4
+
diff --git a/src/main/scala/org/camunda/dmn/DmnEngine.scala b/src/main/scala/org/camunda/dmn/DmnEngine.scala
index 038c9f9f..7ec2b5e6 100644
--- a/src/main/scala/org/camunda/dmn/DmnEngine.scala
+++ b/src/main/scala/org/camunda/dmn/DmnEngine.scala
@@ -15,17 +15,20 @@
*/
package org.camunda.dmn
+import org.camunda.bpm.model.dmn.DmnModelInstance
+
import java.io.InputStream
import java.util.ServiceLoader
import org.camunda.dmn.Audit._
import org.camunda.dmn.evaluation._
+import org.camunda.dmn.model.xml.instance.DmnModelInstanceProvider
import org.camunda.dmn.parser._
import org.camunda.feel.{FeelEngine, FeelEngineClock}
import org.camunda.feel.context.{CustomFunctionProvider, FunctionProvider}
import org.camunda.feel.syntaxtree.{Val, ValError, ValNull}
import org.camunda.feel.valuemapper.{CustomValueMapper, ValueMapper}
-import scala.collection.JavaConverters._
+import scala.jdk.CollectionConverters._
import scala.collection.mutable.{ListBuffer => mutableList}
import scala.reflect.{ClassTag, classTag}
@@ -79,13 +82,15 @@ object DmnEngine {
case class Configuration(escapeNamesWithSpaces: Boolean = false,
escapeNamesWithDashes: Boolean = false,
- lazyEvaluation: Boolean = false)
+ lazyEvaluation: Boolean = false,
+ modelInstanceProvider: Option[DmnModelInstanceProvider] = None)
class Builder {
private var escapeNamesWithSpaces_ = false
private var escapeNamesWithDashes_ = false
private var lazyEvaluation_ = false
+ private var modelInstanceProvider_ : Option[DmnModelInstanceProvider] = None
private var auditLogListeners_ = List[AuditLogListener]().toBuffer
private var clock: FeelEngineClock = FeelEngineClock.SystemClock
@@ -104,6 +109,11 @@ object DmnEngine {
this
}
+ def modelInstanceProvider(provider: DmnModelInstanceProvider): Builder = {
+ modelInstanceProvider_ = Some(provider)
+ this
+ }
+
def addAuditListener(listener: AuditLogListener): Builder = {
auditLogListeners_ += listener
this
@@ -119,7 +129,8 @@ object DmnEngine {
configuration = DmnEngine.Configuration(
escapeNamesWithSpaces = escapeNamesWithSpaces_,
escapeNamesWithDashes = escapeNamesWithDashes_,
- lazyEvaluation = lazyEvaluation_),
+ lazyEvaluation = lazyEvaluation_,
+ modelInstanceProvider = modelInstanceProvider_),
auditLogListeners = auditLogListeners_.toList,
clock = clock
)
@@ -137,6 +148,7 @@ class DmnEngine(configuration: DmnEngine.Configuration =
private val valueMapper = loadValueMapper()
private val functionProvider = loadFunctionProvider()
+ private val loadedModels: Map[String, DmnModelInstance] = Map.empty
logger.info(
s"DMN-Engine created. [" +
diff --git a/src/main/scala/org/camunda/dmn/model/xml/instance/DmnModelInstanceProvider.scala b/src/main/scala/org/camunda/dmn/model/xml/instance/DmnModelInstanceProvider.scala
new file mode 100644
index 00000000..07b13af0
--- /dev/null
+++ b/src/main/scala/org/camunda/dmn/model/xml/instance/DmnModelInstanceProvider.scala
@@ -0,0 +1,48 @@
+/*
+ * Copyright © 2022 Camunda Services GmbH (info@camunda.com)
+ *
+ * Licensed 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
+ *
+ * http://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.camunda.dmn.model.xml.instance
+
+import jdk.dynalink.Namespace
+import org.camunda.bpm.model.dmn.DmnModelInstance
+import org.camunda.bpm.model.xml.ModelBuilder
+import org.camunda.bpm.model.xml.impl.ModelImpl
+import org.camunda.bpm.model.xml.instance.DomDocument
+
+import java.io.InputStream
+
+/**
+ * Provides DmnModelInstances via an InputStream or through some other locator
+ */
+trait DmnModelInstanceProvider {
+
+ /**
+ * Attempts to find and load a dmn model for the given namespace, optionally
+ * using the locator if necessary
+ *
+ * @param namespace - the namespace of the dmn model searched for
+ * @param locator - a hint for the provider about where the model might be found
+ * @return - a DmnModelInstance if one was located for the given namespace, or None if the model could not be found
+ */
+ def loadModel[T](namespace: String, locator: Option[T]): Option[DmnModelInstance]
+
+ /**
+ * Instantiates a DmnModelInstance from the given Input Stream
+ *
+ * @param is - the input stream from which a dmn model can be read
+ * @return
+ */
+ def readModelFromStream(is: InputStream): DmnModelInstance
+}
diff --git a/src/main/scala/org/camunda/dmn/model/xml/instance/ImportAwareDmnModelInstanceImpl.scala b/src/main/scala/org/camunda/dmn/model/xml/instance/ImportAwareDmnModelInstanceImpl.scala
new file mode 100644
index 00000000..3eced7f3
--- /dev/null
+++ b/src/main/scala/org/camunda/dmn/model/xml/instance/ImportAwareDmnModelInstanceImpl.scala
@@ -0,0 +1,141 @@
+/*
+ * Copyright © 2022 Camunda Services GmbH (info@camunda.com)
+ *
+ * Licensed 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
+ *
+ * http://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.camunda.dmn.model.xml.instance
+
+import org.camunda.bpm.model.dmn.impl.DmnModelInstanceImpl
+import org.camunda.bpm.model.dmn.impl.DmnModelConstants
+import org.camunda.bpm.model.xml.impl.ModelImpl
+import org.camunda.bpm.model.xml.{ModelBuilder, ModelException}
+import org.camunda.bpm.model.xml.instance.DomDocument
+import org.camunda.bpm.model.xml.instance.ModelElementInstance
+import org.camunda.bpm.model.dmn.instance.{DrgElement, NamedElement, Variable}
+import org.camunda.bpm.model.xml.`type`.ModelElementType
+import org.slf4j.LoggerFactory
+
+import java.util
+import scala.jdk.CollectionConverters._
+import scala.util.{Failure, Success, Try}
+
+object ImportAwareDmnModelInstanceImpl {
+
+ val dmnNameSpaces: List[String] = List(
+ DmnModelConstants.DMN12_NS,
+ DmnModelConstants.DMN12_NS,
+ DmnModelConstants.DMN13_NS,
+ DmnModelConstants.DMN11_ALTERNATIVE_NS,
+ DmnModelConstants.DMN13_ALTERNATIVE_NS
+ ).map(_.toLowerCase())
+
+ /**
+ * Allows an element's name to be qualified by the (import) name of the model that defines the element
+ *
+ * @param modelElement
+ */
+ implicit class ModelQualifiedElementName(modelElement: NamedElement) {
+ def qualifiedName: String = {
+ ((modelElement.getModelInstance, modelElement) match {
+ case (m: ImportAwareDmnModelInstanceImpl, e: DrgElement) => m.importedModelName
+ case (m: ImportAwareDmnModelInstanceImpl, v: Variable) => m.importedModelName
+ case (_, _) => None
+ }) match {
+ case Some(qualifier) => s"$qualifier.${modelElement.getName}"
+ case None => modelElement.getName
+ }
+ }
+ }
+}
+
+/**
+ * A dmn model instance that is able to resolve elements existing in the DRG that were imported from other dmn models.
+ */
+
+class ImportAwareDmnModelInstanceImpl (model: ModelImpl,
+ modelBuilder: ModelBuilder,
+ document: DomDocument,
+ private val dmnModelInstanceProvider: DmnModelInstanceProvider)
+ extends DmnModelInstanceImpl(model, modelBuilder, document) {
+ import ImportAwareDmnModelInstanceImpl._
+
+ private var importedModelName: Option[String] = None
+
+ def withImportedModelName(modelName: Option[String]): ImportAwareDmnModelInstanceImpl = {
+ val copy = clone()
+ copy.importedModelName = modelName;
+ copy
+ }
+
+ /**
+ * The collection of imported models to be loaded once when required
+ */
+ private val importedModels = loadImports
+
+ private def loadImports = getDefinitions.getImports.asScala
+ .filter(id => dmnNameSpaces.contains(id.getImportType.toLowerCase()))
+ .flatMap(id =>
+ Try(dmnModelInstanceProvider.loadModel(id.getNamespace, Option(id.getLocationUri))
+ .map {
+ case iam: ImportAwareDmnModelInstanceImpl =>
+ iam.withImportedModelName(Option(id.getAttributeValue("name")))
+ case other => other
+ }) match {
+ case Success(m) => m
+ case Failure(exception) =>
+ val errorMsg = s"Unable to load imported model at location ${id.getLocationUri} " +
+ s"for name ${id.getAttributeValue("name")}"
+ LoggerFactory.getLogger(getClass).error(errorMsg, exception)
+ throw new ModelException(errorMsg, exception)
+ }).toSeq
+
+ /**
+ * Gets a model element from the model or an imported model whose id matches the given id
+ *
+ * @param id
+ * @tparam T
+ * @return
+ */
+ override def getModelElementById[T <: ModelElementInstance](id: String): T = {
+ var element = super.getModelElementById[T](id)
+ if (element == null) {
+ importedModels.map(_.getModelElementById[T](id)).find(e => e != null) match {
+ case Some(value) => element = value
+ case None => //nothing to do
+ }
+ }
+ element
+ }
+
+ /**
+ * Gets all elements of the model and its imported models
+ *
+ * @param referencingClass - the type of element to be retrieved
+ * @tparam T
+ * @return
+ */
+ override def getModelElementsByType[T <: ModelElementInstance](referencingClass: Class[T]): util.Collection[T] = {
+ (super.getModelElementsByType[T](referencingClass).asScala ++
+ importedModels.flatMap(_.getModelElementsByType[T](referencingClass).asScala)).asJavaCollection
+ }
+
+ override def getModelElementsByType(`type`: ModelElementType): util.Collection[ModelElementInstance] = {
+ (super.getModelElementsByType(`type`).asScala ++
+ importedModels.flatMap(_.getModelElementsByType(`type`).asScala)).asJavaCollection
+ }
+ override def clone(): ImportAwareDmnModelInstanceImpl = {
+ val superClone = super.clone()
+ new ImportAwareDmnModelInstanceImpl(model, modelBuilder, superClone.getDocument, dmnModelInstanceProvider)
+ }
+
+}
diff --git a/src/main/scala/org/camunda/dmn/model/xml/instance/StatefulDmnModelInstanceProvider.scala b/src/main/scala/org/camunda/dmn/model/xml/instance/StatefulDmnModelInstanceProvider.scala
new file mode 100644
index 00000000..05019771
--- /dev/null
+++ b/src/main/scala/org/camunda/dmn/model/xml/instance/StatefulDmnModelInstanceProvider.scala
@@ -0,0 +1,64 @@
+/*
+ * Copyright © 2022 Camunda Services GmbH (info@camunda.com)
+ *
+ * Licensed 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
+ *
+ * http://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.camunda.dmn.model.xml.instance
+
+import org.camunda.bpm.model.dmn.impl.{DmnModelInstanceImpl, DmnParser}
+import org.camunda.bpm.model.dmn.{Dmn, DmnModelInstance}
+import org.camunda.bpm.model.xml.impl.ModelImpl
+import org.camunda.bpm.model.xml.instance.DomDocument
+
+import java.io.InputStream
+
+/**
+ * A DmnModelInstanceProvider that maintains a list of all models that it has already been used to load,
+ * which can then be retrieved by namespace
+ */
+class StatefulDmnModelInstanceProvider extends DmnParser with DmnModelInstanceProvider {
+
+ val loadedModels: scala.collection.mutable.Map[String, DmnModelInstance] = scala.collection.mutable.Map.empty
+
+ /**
+ * Retrieves the dmn model
+ * @param namespace - the namespace of the dmn model searched for
+ * @param locator - a hint for the provider about where the model might be found. In this case, returns none if locator is not an InputStream
+ * @return - a DmnModelInstance if one was located for the given namespace, or None if the model could not be found
+ */
+ override def loadModel[T](namespace: String, locator: Option[T] = None): Option[DmnModelInstance] = {
+ if (!loadedModels.contains(namespace)) {
+ None
+ } else {
+ loadedModels.get(namespace)
+ }
+ }
+
+ override def readModelFromStream(is: InputStream): DmnModelInstance = {
+ Option(parseModelFromStream(is))
+ .map(modelInstance => {
+ loadedModels.put(
+ modelInstance.getDefinitions.getNamespace,
+ modelInstance)
+ modelInstance
+ }).orNull
+ }
+
+ override def createModelInstance(document: DomDocument): DmnModelInstanceImpl = {
+ new ImportAwareDmnModelInstanceImpl(
+ Dmn.INSTANCE.getDmnModel.asInstanceOf[ModelImpl],
+ Dmn.INSTANCE.getDmnModelBuilder,
+ document,
+ this)
+ }
+}
diff --git a/src/main/scala/org/camunda/dmn/model/xml/instance/URILocatorDmnModelInstanceProvider.scala b/src/main/scala/org/camunda/dmn/model/xml/instance/URILocatorDmnModelInstanceProvider.scala
new file mode 100644
index 00000000..03048e02
--- /dev/null
+++ b/src/main/scala/org/camunda/dmn/model/xml/instance/URILocatorDmnModelInstanceProvider.scala
@@ -0,0 +1,61 @@
+/*
+ * Copyright © 2022 Camunda Services GmbH (info@camunda.com)
+ *
+ * Licensed 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
+ *
+ * http://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.camunda.dmn.model.xml.instance
+
+import org.camunda.bpm.model.dmn.{Dmn, DmnModelInstance}
+import org.camunda.bpm.model.xml.impl.ModelImpl
+
+import java.io.InputStream
+import java.net.URI
+import scala.reflect.ClassTag.Nothing
+
+object URILocatorDmnModelInstanceProvider {
+ /**
+ * Attempts to open an input stream on a resource at the given uri. If the uri is absolute, then an attempt is made
+ * to open the stream at that location. If the URI is relative, then the resource is assumed to be a classpath
+ * resource, and an attempt will be made to locate it on the classpath.
+ *
+ * @param uri - A java.net.URI suitable for locating a resource
+ */
+ implicit class URItoStream(val uri: URI) {
+ def inputStream: InputStream = if (uri.isAbsolute) {
+ uri.toURL.openStream()
+ } else {
+ ClassLoader.getSystemResourceAsStream(s"${uri.getPath}")
+ }
+ }
+}
+
+class URILocatorDmnModelInstanceProvider extends StatefulDmnModelInstanceProvider {
+ import org.camunda.dmn.model.xml.instance.URILocatorDmnModelInstanceProvider._
+
+ /**
+ * Loads a model from a URI given by the locator, or returns a model already loaded
+ *
+ * @param namespace - the namespace of the dmn model searched for
+ * @param locator - An absolute URI or a path to a location on the classpath where the model can be found
+ * @tparam T
+ * @return - a DmnModelInstance if one was located for the given namespace, or None if the model could not be found
+ */
+ override def loadModel[T](namespace: String, locator: Option[T]): Option[DmnModelInstance] = {
+ super.loadModel(namespace, locator)
+ .orElse(locator.map {
+ case s: String => readModelFromStream(URI.create(s).inputStream)
+ case uri: URI => readModelFromStream(uri.inputStream)
+ case _ => null
+ })
+ }
+}
diff --git a/src/main/scala/org/camunda/dmn/parser/DmnParser.scala b/src/main/scala/org/camunda/dmn/parser/DmnParser.scala
index 514772b5..894776c4 100644
--- a/src/main/scala/org/camunda/dmn/parser/DmnParser.scala
+++ b/src/main/scala/org/camunda/dmn/parser/DmnParser.scala
@@ -19,8 +19,9 @@ import java.io.InputStream
import org.camunda.dmn.logger
import org.camunda.bpm.model.dmn._
import org.camunda.bpm.model.dmn.impl.DmnModelConstants
-import org.camunda.bpm.model.dmn.instance.{BusinessKnowledgeModel, Column, Context, Decision, DecisionTable, DrgElement, Expression, FunctionDefinition, InformationItem, Invocation, ItemDefinition, LiteralExpression, Relation, UnaryTests, List => DmnList}
+import org.camunda.bpm.model.dmn.instance.{BusinessKnowledgeModel, Column, Context, Decision, DecisionTable, DrgElement, Expression, FunctionDefinition, InformationItem, Invocation, ItemDefinition, LiteralExpression, NamedElement, Relation, UnaryTests, List => DmnList}
import org.camunda.dmn.DmnEngine.{Configuration, Failure}
+import org.camunda.dmn.model.xml.instance.ImportAwareDmnModelInstanceImpl.ModelQualifiedElementName
import org.camunda.feel
import scala.annotation.tailrec
@@ -72,7 +73,7 @@ class DmnParser(
def parse(stream: InputStream): Either[Failure, ParsedDmn] = {
- Try(Dmn.readModelFromStream(stream)) match {
+ Try(readModelFromStream(stream)) match {
case scala.util.Success(model) => {
parseModel(model).left.map(failures =>
Failure(failures.map(_.message).mkString("\n")))
@@ -81,6 +82,13 @@ class DmnParser(
}
}
+ private def readModelFromStream(stream: InputStream): DmnModelInstance = {
+ configuration.modelInstanceProvider match {
+ case Some(provider) => provider.readModelFromStream(stream)
+ case _ => Dmn.readModelFromStream(stream)
+ }
+ }
+
private def parseModel(
model: DmnModelInstance): Either[Iterable[Failure], ParsedDmn] = {
@@ -190,7 +198,7 @@ class DmnParser(
val requiredBkms = knowledgeRequirements
.map(r => r.getRequiredKnowledge)
.map(k =>
- ctx.bkms.getOrElseUpdate(k.getName, parseBusinessKnowledgeModel(k)))
+ ctx.bkms.getOrElseUpdate(k.qualifiedName, parseBusinessKnowledgeModel(k)))
val logic: ParsedDecisionLogic = decision.getExpression match {
case dt: DecisionTable => parseDecisionTable(dt)
@@ -208,12 +216,12 @@ class DmnParser(
val variable = Option(decision.getVariable)
val resultType = variable.flatMap(v => Option(v.getTypeRef))
val resultName = variable
- .map(_.getName)
+ .map(_.qualifiedName)
.orElse(Option(decision.getId))
- .getOrElse(decision.getName)
+ .getOrElse(decision.qualifiedName)
ParsedDecision(decision.getId,
- decision.getName,
+ decision.qualifiedName,
logic,
resultName,
resultType,
@@ -230,7 +238,7 @@ class DmnParser(
val requiredBkms = knowledgeRequirements
.map(r => r.getRequiredKnowledge)
.map(k =>
- ctx.bkms.getOrElseUpdate(k.getName, parseBusinessKnowledgeModel(k)))
+ ctx.bkms.getOrElseUpdate(k.qualifiedName, parseBusinessKnowledgeModel(k)))
Option(bkm.getEncapsulatedLogic)
.map { encapsulatedLogic =>
@@ -248,10 +256,10 @@ class DmnParser(
}
val parameters = encapsulatedLogic.getFormalParameters.asScala
- .map(f => f.getName -> f.getTypeRef)
+ .map(f => f.qualifiedName -> f.getTypeRef)
ParsedBusinessKnowledgeModel(bkm.getId,
- bkm.getName,
+ bkm.qualifiedName,
logic,
parameters,
requiredBkms)
@@ -260,7 +268,7 @@ class DmnParser(
.getOrElse {
ParsedBusinessKnowledgeModel(bkm.getId,
- bkm.getName,
+ bkm.qualifiedName,
EmptyLogic,
Iterable.empty,
requiredBkms)
@@ -339,13 +347,13 @@ class DmnParser(
// TODO verify that every entry has a variable name
if (Option(lastEntry.getVariable).isDefined) {
val contextEntries = entries.map(e =>
- e.getVariable.getName -> parseAnyExpression(e.getExpression))
+ e.getVariable.qualifiedName -> parseAnyExpression(e.getExpression))
ParsedContext(contextEntries, None)
} else {
val contextEntries = entries
.take(entries.size - 1)
- .map(e => e.getVariable.getName -> parseAnyExpression(e.getExpression))
+ .map(e => e.getVariable.qualifiedName -> parseAnyExpression(e.getExpression))
val aggregationEntry = parseAnyExpression(lastEntry.getExpression)
ParsedContext(contextEntries, Some(aggregationEntry))
@@ -365,7 +373,7 @@ class DmnParser(
ctx: ParsingContext): ParsedRelation = {
val rows = relation.getRows.asScala
val columns = relation.getColumns.asScala
- val columNames = columns.map(_.getName)
+ val columNames = columns.map(_.qualifiedName)
rows
.filterNot(row => row.getExpressions.size == columns.size)
@@ -394,7 +402,7 @@ class DmnParser(
expression match {
case lt: LiteralExpression => {
val expr = parseFeelExpression(lt)
- val parametersWithTypes = parameters.map(p => p.getName -> p.getTypeRef)
+ val parametersWithTypes = parameters.map(p => p.qualifiedName -> p.getTypeRef)
ParsedFunctionDefinition(expr, parametersWithTypes)
}
@@ -414,7 +422,7 @@ class DmnParser(
.map(b =>
b.getExpression match {
case lt: LiteralExpression =>
- Some(b.getParameter.getName -> parseFeelExpression(lt))
+ Some(b.getParameter.qualifiedName -> parseFeelExpression(lt))
case other => {
ctx.failures += Failure(
s"expected binding with literal expression but found '$other'")
@@ -561,28 +569,27 @@ class DmnParser(
(expression /: namesWithSpaces)(
(e, name) =>
- e.replaceAll("""([(,.]|\s|^)(""" + name + """)([(),.]|\s|$)""",
+ e.replaceAll("""([(),.+-/=:]|\[|\]|\*|\s|^)(""" + name + """)([(),.+-/=:]|\[|\]|\*|\s|$)""",
"$1`$2`$3"))
}
private def getNamesToEscape(model: DmnModelInstance): Iterable[String] = {
- val names = Seq(classOf[InformationItem], classOf[ItemDefinition]).flatMap(elementType =>
- model.getModelElementsByType(elementType).asScala)
- .filterNot(classOf[Column].isInstance(_))
- .map(_.getName)
+ val names =
+ Seq(model.getModel.getType(classOf[InformationItem]), model.getModel.getType(classOf[ItemDefinition])).
+ flatMap(elementType => model.getModelElementsByType(elementType).asScala)
+ .filterNot(classOf[Column].isInstance(_))
+ .flatMap {
+ case named: NamedElement => Some(named.qualifiedName)
+ case _ => None
+ }
val nameFilter: (String => Boolean) = {
- if (configuration.escapeNamesWithSpaces && configuration.escapeNamesWithDashes) {
- name =>
- name.contains(" ") || name.contains("-")
- } else if (configuration.escapeNamesWithSpaces) { name =>
- name.contains(" ")
- } else if (configuration.escapeNamesWithDashes) { name =>
- name.contains("-")
- } else { name =>
- false
- }
+ name => List(
+ configuration.escapeNamesWithSpaces && name.contains(" "),
+ configuration.escapeNamesWithDashes && name.contains("-"),
+ configuration.modelInstanceProvider.isDefined && name.contains("."))
+ .reduce((a,b)=>a||b)
}
val namesToEscape = names.filter(nameFilter)
diff --git a/src/test/resources/imports/imported_model.dmn b/src/test/resources/imports/imported_model.dmn
new file mode 100644
index 00000000..6c03a750
--- /dev/null
+++ b/src/test/resources/imports/imported_model.dmn
@@ -0,0 +1,39 @@
+
+
+
+
+
+
+
+
+
+
+ dividend / divisor
+
+
+
+
+
+
+
+
+ 497
+
+
+ 50
+ 497
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/src/test/resources/imports/importing_model.dmn b/src/test/resources/imports/importing_model.dmn
new file mode 100644
index 00000000..0d5f303f
--- /dev/null
+++ b/src/test/resources/imports/importing_model.dmn
@@ -0,0 +1,88 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ imports_for_division.quotient(dividend, divisor)
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ 300
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/src/test/resources/imports/importing_model_not_found.dmn b/src/test/resources/imports/importing_model_not_found.dmn
new file mode 100644
index 00000000..060cd79d
--- /dev/null
+++ b/src/test/resources/imports/importing_model_not_found.dmn
@@ -0,0 +1,88 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ imports_for_division.quotient(dividend, divisor)
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ 300
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/src/test/resources/imports/tck-0086-import.dmn b/src/test/resources/imports/tck-0086-import.dmn
new file mode 100644
index 00000000..f3fc176e
--- /dev/null
+++ b/src/test/resources/imports/tck-0086-import.dmn
@@ -0,0 +1,101 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ myimport.Say Hello(A Person)
+
+
+
+
+
+
+
+ A Person.age
+
+
+
+
+
+ <=30
+
+
+ normal greeting
+
+
+
+
+ >30
+
+
+ "Respectfully, "+normal greeting
+
+
+
+
+
+
+ override greeting
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/src/test/resources/imports/tck-0086-imported-model.dmn b/src/test/resources/imports/tck-0086-imported-model.dmn
new file mode 100644
index 00000000..6b15db7c
--- /dev/null
+++ b/src/test/resources/imports/tck-0086-imported-model.dmn
@@ -0,0 +1,41 @@
+
+
+
+
+ string
+
+
+ number
+
+
+
+
+
+
+
+ "Hello " + Person.name + "!"
+
+
+
+
+
+
+
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/src/test/scala/org/camunda/dmn/ImportTest.scala b/src/test/scala/org/camunda/dmn/ImportTest.scala
new file mode 100644
index 00000000..5d8e2538
--- /dev/null
+++ b/src/test/scala/org/camunda/dmn/ImportTest.scala
@@ -0,0 +1,86 @@
+/*
+ * Copyright © 2022 Camunda Services GmbH (info@camunda.com)
+ *
+ * Licensed 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
+ *
+ * http://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.camunda.dmn
+
+import org.camunda.dmn.DmnEngine._
+import org.camunda.dmn.model.xml.instance.{StatefulDmnModelInstanceProvider, URILocatorDmnModelInstanceProvider}
+import org.camunda.dmn.parser.ParsedDmn
+import org.scalatest.flatspec.AnyFlatSpec
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.matchers.should.Matchers.a
+
+import scala.util.{Success, Try}
+
+class ImportTest extends AnyFlatSpec with Matchers with DecisionTest {
+
+ private val engineWithImportsFromURI = new DmnEngine.Builder()
+ .modelInstanceProvider(new URILocatorDmnModelInstanceProvider())
+ .build
+
+ private val failResult = engineWithImportsFromURI.parse(
+ getClass.getResourceAsStream("/imports/importing_model_not_found.dmn")
+ )
+
+ private val parseResult = engineWithImportsFromURI.parse(
+ getClass.getResourceAsStream("/imports/importing_model.dmn")
+ )
+
+ "A model importing a dmn model" should "successfully parse" in {
+ parseResult.isLeft should be(false)
+ }
+
+
+ it should "successfully evaluate an imported BKM" in {
+ val parsedDmn = new ParsedResult(parseResult).dmn
+ val result = engineWithImportsFromURI.evalByName(parsedDmn,
+ "Compute Quotient",
+ Map("dividend" -> 6, "divisor" -> 2))
+ result.isRight should be(true)
+ result.map(_.value should be(3))
+
+ }
+
+ it should "fail with an appropriate message when unable to import" in {
+ failResult.isLeft should be(true)
+ failResult.swap.toOption.map(_.message should include("Unable to load imported model"))
+ }
+
+ it should "successfully import from pre-loaded models" in {
+ val engine = new DmnEngine.Builder()
+ .modelInstanceProvider(new StatefulDmnModelInstanceProvider())
+ .escapeNamesWithSpaces(true)
+ .build
+
+ engine.parse(
+ getClass.getResourceAsStream("/imports/tck-0086-imported-model.dmn")
+ ).isLeft should be(false)
+ val parseRslt = engine.parse(
+ getClass.getResourceAsStream("/imports/tck-0086-import.dmn")
+ )
+ parseRslt.isLeft should be(false)
+
+ val parsedDmn = new ParsedResult(parseRslt).dmn
+
+ val result = engine.evalByName(parsedDmn,
+ "A Decision Ctx with DT",
+ Map("A Person" -> Map("name" -> "John Doe", "age" -> 21))
+ )
+
+ result.isRight should be(true)
+ result.map(_.value should be("Hello John Doe!"))
+
+ }
+}