Skip to content

Commit

Permalink
Merge pull request #444 from silk-framework/feature/AutoSuggestionOnV…
Browse files Browse the repository at this point in the history
…aluePathFields-CMEM-3052

Feature/auto suggestion on value path fields cmem 3052
  • Loading branch information
robertisele authored May 5, 2021
2 parents ebb09b6 + 5a03a3b commit 6e811b7
Show file tree
Hide file tree
Showing 78 changed files with 2,856 additions and 342 deletions.
2 changes: 2 additions & 0 deletions silk-core/src/main/scala/org/silkframework/config/Task.scala
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,8 @@ trait Task[+TaskType <: TaskSpec] {
metaData.formattedLabel(id, maxLength)
}

def fullTaskLabel: String = taskLabel(Int.MaxValue)

override def equals(obj: scala.Any): Boolean = obj match {
case task: Task[_] =>
id == task.id &&
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ trait DataSource {
protected def genericEntityIRI(identifier: Identifier): String = DataSource.generateEntityUri(underlyingTask.id, identifier)
}

object DataSource{
object DataSource {

//the URN_NID prefix (see rfc 8141) for for generic dataset and entity naming
val URN_NID_PREFIX: String = "urn:instance:"
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
package org.silkframework.dataset

import org.silkframework.dataset.DataSourceCharacteristics.SupportedPathExpressions

/** Characteristics of a data source.
*
* @param supportedPathExpressions The characteristics of the supported path expressions.
*/
case class DataSourceCharacteristics(supportedPathExpressions: SupportedPathExpressions = SupportedPathExpressions())

object DataSourceCharacteristics {

/** Sources that only support plain attributes (i.e., forward paths of length 1 without any filters) */
def attributesOnly: DataSourceCharacteristics = DataSourceCharacteristics()

/** The kind of path expressions supported by a data source.
*
* @param multiHopPaths If enabled it is possible to define multi-hop paths (e.g. in RDF, JSON, XML). Else only single-hop
* path are supported.
* @param backwardPaths If the data source supports backward paths, i.e. reversing the direction of a property (e.g. in
* RDF, JSON (parent), XML (parent)).
* @param languageFilter If the data source supports language filters, i.e. is able to filter by different language versions
* of property values (only supported in RDF).
* @param propertyFilter If the data source supports (single-hop forward) property filters.
* @param specialPaths The data source specific paths that are supported by a data source, e.g. row ID in CSV.
*/
case class SupportedPathExpressions(multiHopPaths: Boolean = false,
backwardPaths: Boolean = false,
languageFilter: Boolean = false,
propertyFilter: Boolean = false,
specialPaths: Seq[SpecialPathInfo] = Seq.empty)

/**
* Information about data source specific special paths that have a special semantic.
*
* @param value The path value.
* @param description Description of the semantics of the special path.
*/
case class SpecialPathInfo(value: String, description: Option[String])

}

Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,10 @@ trait Dataset extends AnyPlugin with DatasetAccess {

/** Related links for this dataset. */
def datasetLinks: Seq[TaskLink] = Seq.empty

/** Characteristics of the data source. */
def characteristics: DataSourceCharacteristics

}

trait DatasetPluginAutoConfigurable[T <: Dataset] {
Expand All @@ -28,6 +32,6 @@ trait DatasetPluginAutoConfigurable[T <: Dataset] {
}

/**
* Creates new dataset olugin instances.
* Creates new dataset plugin instances.
*/
object Dataset extends PluginFactory[Dataset]
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ trait DatasetAccess extends DatasetReadAccess with DatasetWriteAccess
trait DatasetReadAccess {

/**
* Returns a data source for reading entities from the data set.
* Creates a new data source for reading entities from the data set.
*/
def source(implicit userContext: UserContext): DataSource

Expand All @@ -25,12 +25,12 @@ trait DatasetReadAccess {
trait DatasetWriteAccess {

/**
* Returns a link sink for writing entity links to the data set.
* Creates a new link sink for writing entity links to the data set.
*/
def linkSink(implicit userContext: UserContext): LinkSink

/**
* Returns a entity sink for writing entities to the data set.
* Creates a new entity sink for writing entities to the data set.
*/
def entitySink(implicit userContext: UserContext): EntitySink

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,8 @@ case class DatasetSpec[+DatasetType <: Dataset](plugin: DatasetType, uriAttribut
safeAccess(DatasetSpec.LinkSinkWrapper(plugin.linkSink, this), SafeModeSink)
}

def characteristics: DataSourceCharacteristics = plugin.characteristics

// True if access should be prevented regarding the dataset and safe-mode config
private def preventAccessInSafeMode(implicit userContext: UserContext): Boolean = {
ProductionConfig.inSafeMode && !plugin.isFileResourceBased && !userContext.executionContext.insideWorkflow
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,4 +69,6 @@ object EmptyDataset extends Dataset with Serializable {
*/
override def clear()(implicit userContext: UserContext): Unit = {}
}

override def characteristics: DataSourceCharacteristics = DataSourceCharacteristics.attributesOnly
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,5 +29,7 @@ final class VariableDataset extends Dataset {
*/
override def linkSink(implicit userContext: UserContext): LinkSink = error()

override def characteristics: DataSourceCharacteristics = DataSourceCharacteristics.attributesOnly

private def error() = throw new RuntimeException("A Variable Dataset cannot be accessed! Only use it in workflows that replace all variable datasets before execution.")
}
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package org.silkframework.dataset.rdf

import org.silkframework.dataset.Dataset
import org.silkframework.dataset.DataSourceCharacteristics.SupportedPathExpressions
import org.silkframework.dataset.{DataSourceCharacteristics, Dataset}

trait RdfDataset extends Dataset {

Expand All @@ -11,4 +12,16 @@ trait RdfDataset extends Dataset {
*/
def graphOpt: Option[String] = None

/** Shared characteristics of all RDF Datasets */
override final val characteristics: DataSourceCharacteristics = {
DataSourceCharacteristics(
SupportedPathExpressions(
multiHopPaths = true,
backwardPaths = true,
propertyFilter = true,
languageFilter = true
)
)
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import org.silkframework.config.Prefixes
import org.silkframework.runtime.validation.ValidationException
import org.silkframework.util.Uri

import scala.util.matching.Regex
import scala.util.parsing.combinator.RegexParsers
import scala.util.parsing.input.CharSequenceReader

Expand All @@ -30,12 +31,7 @@ private[entity] class PathParser(prefixes: Prefixes) extends RegexParsers {
if(pathStr.isEmpty) {
UntypedPath(Nil)
} else {
// Complete path if a simplified syntax is used
val completePath = pathStr.head match {
case '?' => pathStr // Path includes a variable
case '/' | '\\' => "?a" + pathStr // Variable has been left out
case _ => "?a/" + pathStr // Variable and leading '/' have been left out
}
val completePath = normalized(pathStr)
// Parse path
parseAll(path, new CharSequenceReader(completePath)) match {
case Success(parsedPath, _) => parsedPath
Expand All @@ -44,7 +40,68 @@ private[entity] class PathParser(prefixes: Prefixes) extends RegexParsers {
}
}

private def path = variable ~ rep(forwardOperator | backwardOperator | filterOperator) ^^ {
/**
* Returns the part of the path that could be parsed until a parse error and the error if one occurred.
* @param pathStr The input path string.
*/
def parseUntilError(pathStr: String): PartialParseResult = {
if(pathStr.isEmpty) {
return PartialParseResult(UntypedPath(List.empty), None)
}
val completePath = normalized(pathStr)
// Added characters because of normalization. Need to be removed when reporting the actual error offset.
val addedOffset = completePath.length - pathStr.length
val inputSequence = new CharSequenceReader(completePath)
var partialPathOps: Vector[PathOperator] = Vector.empty
var partialParseError: Option[PartialParseError] = None
val variableResult = parse(variable, inputSequence) // Ignore variable
var parseOffset = variableResult.next.offset
def originalParseOffset = math.max(0, parseOffset - addedOffset)
while(partialParseError.isEmpty && parseOffset < completePath.length) {
try {
parse(ops, inputSequence.drop(parseOffset)) match {
case Success(pathOperator, next) => {
partialPathOps :+= pathOperator
parseOffset = next.offset
}
case error: NoSuccess =>
// Subtract 1 because next is positioned after the character that lead to the parse error.
val originalErrorOffset = math.max(error.next.offset - addedOffset - 1, 0)
partialParseError = Some(PartialParseError(
originalErrorOffset,
error.msg,
pathStr.substring(originalParseOffset, math.min(originalErrorOffset + 1, pathStr.length)), // + 1 since we want to have the character where it failed
originalParseOffset
))
}
} catch {
case validationException: ValidationException =>
// Can happen e.g. when a qualified name used an invalid/unknown prefix name
partialParseError = Some(PartialParseError(
originalParseOffset,
validationException.getMessage,
"",
originalParseOffset
))
}
}
PartialParseResult(UntypedPath(partialPathOps.toList), partialParseError)
}

// Normalizes the path syntax in case a simplified syntax has been used
private def normalized(pathStr: String): String = {
pathStr.headOption match {
case Some('?') => pathStr // Path includes a variable
case Some('/') | Some('\\') => "?a" + pathStr // Variable has been left out
case _ => "?a/" + pathStr // Variable and leading '/' have been left out
}
}

private def ops = forwardOperator | backwardOperator | filterOperator ^^ {
case operator => operator
}

private def path = variable ~ rep(ops) ^^ {
case variable ~ operators => UntypedPath(operators)
}

Expand Down Expand Up @@ -73,14 +130,36 @@ private[entity] class PathParser(prefixes: Prefixes) extends RegexParsers {
}

// An identifier that is either a URI enclosed in angle brackets (e.g., <URI>) or a plain identifier (e.g., name or prefix:name)
private def identifier = """<[^>]+>|[^\\/\[\]<>=!" ]+""".r
private def identifier: Regex = PathParser.Regexes.identifier

// A language tag according to the Sparql spec
private def languageTag = """[a-zA-Z]+('-'[a-zA-Z0-9]+)*""".r

// A value that is either an identifier or a literal value enclosed in quotes (e.g., "literal").
private def value = identifier | "\"[^\"]+\"".r
private def value = identifier | "\"[^\"]*\"".r

// A comparison operator
private def compOperator = ">" | "<" | ">=" | "<=" | "=" | "!="
}

object PathParser {
object RegexParts {
val uriExpression = "<[^>]+>"
val plainIdentifier = """[^\\/\[\]<>=!" ]+"""
}
object Regexes {
import RegexParts._
val identifier: Regex = (s"$uriExpression|$plainIdentifier").r
}
}

/**
* A partial path parse result.
*
* @param partialPath The (valid) partial path that has been parsed until the parse error.
* @param error An optional parse error when not all of the input string could be parsed.
*/
case class PartialParseResult(partialPath: UntypedPath, error: Option[PartialParseError])

/** Offset and error message of the parse error. The offset defines the position before the character that lead to the parse error. */
case class PartialParseError(offset: Int, message: String, inputLeadingToError: String, nextParseOffset: Int)
Original file line number Diff line number Diff line change
Expand Up @@ -119,4 +119,8 @@ object UntypedPath {
path
}
}

def partialParse(pathStr: String)(implicit prefixes: Prefixes = Prefixes.empty): PartialParseResult = {
new PathParser(prefixes).parseUntilError(pathStr)
}
}
Original file line number Diff line number Diff line change
@@ -1,14 +1,13 @@
package org.silkframework.execution.local

import java.util.concurrent.ConcurrentHashMap
import java.util.logging.{Level, Logger}

import org.silkframework.dataset.Dataset
import org.silkframework.execution.ExecutionType
import org.silkframework.execution.local.LocalExecution.LocalInternalDataset
import org.silkframework.plugins.dataset.{InternalDataset, InternalDatasetTrait}
import org.silkframework.util.Identifier

import java.util.concurrent.ConcurrentHashMap
import java.util.logging.{Level, Logger}
import scala.collection.JavaConverters._
import scala.collection.mutable

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,10 +15,9 @@
package org.silkframework.plugins.dataset

import java.io.File

import org.silkframework.cache.FileEntityCache
import org.silkframework.config.{PlainTask, RuntimeConfig, Task}
import org.silkframework.dataset.{DataSource, Dataset, DatasetSpec}
import org.silkframework.dataset.{DataSource, DataSourceCharacteristics, Dataset, DatasetSpec}
import org.silkframework.entity._
import org.silkframework.entity.paths.TypedPath
import org.silkframework.execution.EntityHolder
Expand All @@ -38,6 +37,8 @@ case class CacheDataset(dir: String) extends Dataset {

override def linkSink(implicit userContext: UserContext) = ???

override def characteristics: DataSourceCharacteristics = DataSourceCharacteristics()

object CacheSource extends DataSource {
override def retrieve(entityDesc: EntitySchema, limit: Option[Int])
(implicit userContext: UserContext): EntityHolder = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@ case class MockDataset(@Param(label = "person name", value = "The full name of a
override def linkSink(implicit userContext: UserContext): LinkSink = DummyLinkSink(writeLinkFn, clearFn)

override def entitySink(implicit userContext: UserContext): EntitySink = DummyEntitySink(writeEntityFn, clearFn)

override def characteristics: DataSourceCharacteristics = DataSourceCharacteristics.attributesOnly
}

case class DummyDataSource(retrieveFn: (EntitySchema, Option[Int]) => Traversable[Entity],
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
package org.silkframework.entity.paths

import org.scalatest.{FlatSpec, MustMatchers}
import org.silkframework.config.Prefixes
import org.silkframework.util.Uri

class PathParserTest extends FlatSpec with MustMatchers {
behavior of "path parser"

val parser = new PathParser(Prefixes.default)
private val SPACE = " "

it should "parse the full path and not return any errors for valid paths" in {
testValidPath("/a/b[d = 5]\\c")
testValidPath("""\<urn:some:test>[<urn:prop:check> != "check value"]/abc""")
testValidPath("""abc[@lang ='en']""").operators.drop(1).head mustBe LanguageFilter("=", "en")
testValidPath("""abc[ @lang = 'en']""").operators.drop(1).head mustBe LanguageFilter("=", "en")
}

it should "parse invalid paths and return the parsed part and the position where it failed" in {
testInvalidPath(inputString = "/a/b/c/not valid/d/e", expectedParsedString = "/a/b/c/not", expectedErrorOffset = 10, expectedInputOnError = SPACE, expectedNextParseOffset = "/a/b/c/not".length)
testInvalidPath(inputString = s"$SPACE/alreadyInvalid/a", expectedParsedString = "", expectedErrorOffset = 0, expectedInputOnError = SPACE, expectedNextParseOffset = 0)
testInvalidPath(inputString = """\<urn:test:1>/a[b :+ 1]/c""", expectedParsedString = "\\<urn:test:1>/a", expectedErrorOffset = 17, expectedInputOnError = "[b ", expectedNextParseOffset = "\\<urn:test:1>/a".length)
testInvalidPath(inputString = """/a\b/c/""", expectedParsedString = """/a\b/c""",expectedErrorOffset = 6, expectedInputOnError = "/", expectedNextParseOffset = 6)
testInvalidPath(inputString = """invalidNs:broken""", expectedParsedString = "",expectedErrorOffset = 0, expectedInputOnError = "", expectedNextParseOffset = 0)
testInvalidPath(inputString = """<'""", expectedParsedString = "",expectedErrorOffset = 0, expectedInputOnError = "<", expectedNextParseOffset = 0)
}

it should "partially parse filter expressions correctly" in {
parser.parseUntilError("""a/b[@lang = "en"]""").error must not be defined
val result = parser.parseUntilError("""a/b[@lang = "en"]/error now""")
val error = result.error
error mustBe defined
error.get.offset mustBe """a/b[@lang = "en"]/error""".length
}

private def testValidPath(inputString: String): UntypedPath = {
val result = parser.parseUntilError(inputString)
result mustBe PartialParseResult(UntypedPath.parse(inputString), None)
result.partialPath
}

private def testInvalidPath(inputString: String,
expectedParsedString: String,
expectedErrorOffset: Int,
expectedInputOnError: String,
expectedNextParseOffset: Int): Unit = {
val result = parser.parseUntilError(inputString)
result.copy(error = result.error.map(e => e.copy(message = ""))) mustBe PartialParseResult(
UntypedPath.parse(expectedParsedString),
Some(PartialParseError(expectedErrorOffset, "", expectedInputOnError, expectedNextParseOffset))
)
}
}
Loading

0 comments on commit 6e811b7

Please sign in to comment.