You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by an...@apache.org on 2020/11/18 13:52:50 UTC

[jena] branch master updated: JENA-1998: Imports closure machinery

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

andy pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/jena.git


The following commit(s) were added to refs/heads/master by this push:
     new 7eb3ba9  JENA-1998: Imports closure machinery
     new 0c93f37  Merge pull request #866 from afs/shacl-imports
7eb3ba9 is described below

commit 7eb3ba91784ad0a6ad625d7dba52f857c293a6f0
Author: Andy Seaborne <an...@apache.org>
AuthorDate: Sun Nov 15 19:58:27 2020 +0000

    JENA-1998: Imports closure machinery
---
 .../main/java/org/apache/jena/shacl/Imports.java   | 152 +++++++++++++++++++++
 .../main/java/org/apache/jena/shacl/Shapes.java    |  30 ++--
 .../org/apache/jena/shacl/parser/ShapesParser.java |  19 ++-
 jena-shacl/src/test/files/imports/graph1.ttl       |  12 ++
 jena-shacl/src/test/files/imports/graph2.ttl       |  13 ++
 jena-shacl/src/test/files/imports/graph3.ttl       |  16 +++
 jena-shacl/src/test/files/imports/graph4.ttl       |   9 ++
 jena-shacl/src/test/files/imports/graph5.ttl       |   9 ++
 .../test/java/org/apache/jena/shacl/TC_SHACL.java  |   2 +
 .../org/apache/jena/shacl/tests/TestImports.java   | 115 ++++++++++++++++
 10 files changed, 352 insertions(+), 25 deletions(-)

diff --git a/jena-shacl/src/main/java/org/apache/jena/shacl/Imports.java b/jena-shacl/src/main/java/org/apache/jena/shacl/Imports.java
new file mode 100644
index 0000000..222221a
--- /dev/null
+++ b/jena-shacl/src/main/java/org/apache/jena/shacl/Imports.java
@@ -0,0 +1,152 @@
+/*
+ * 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
+ *
+ *     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.apache.jena.shacl;
+
+import static org.apache.jena.atlas.iterator.Iter.iter;
+import static org.apache.jena.sparql.graph.NodeConst.nodeOwlImports;
+import static org.apache.jena.sparql.graph.NodeConst.nodeOwlOntology;
+import static org.apache.jena.sparql.graph.NodeConst.nodeRDFType;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.jena.atlas.lib.Pair;
+import org.apache.jena.graph.Graph;
+import org.apache.jena.graph.GraphUtil;
+import org.apache.jena.graph.Node;
+import org.apache.jena.riot.RDFDataMgr;
+import org.apache.jena.riot.other.G;
+import org.apache.jena.riot.system.IRIResolver;
+import org.apache.jena.sparql.graph.GraphFactory;
+
+/**
+ * Import processing.
+ * <p>
+ * Imports are triggered by a base (a single triple "? rdf:type owl:Ontology")
+ * and imports (triples "base owl:Imports URI").
+ * <p>
+ * If there are other "? owl:imports ?" triples, they are ignored.
+ */
+public class Imports {
+    private Imports() {}
+
+    /**
+     * Load a graph and process owl:imports to create a new, single graph.
+     */
+    public static Graph loadWithImports(String url) {
+        url = IRIResolver.resolveString(url);
+        Graph graph = RDFDataMgr.loadGraph(url);
+        return withImportsWorker(url, graph);
+    }
+
+    /**
+     * Process and return the owl:imports closure of a graph. The graph is included
+     * in the results. Note that without knowing the URI, the start graph may be read
+     * again if it is named as an import.
+     */
+    public static Graph withImports(Graph graph) {
+        return withImportsWorker(null, graph);
+    }
+
+    /**
+     * Process and return the owl:imports closure of a graph.
+     * The graph is included in the results.
+     */
+    public static Graph withImports(String url, Graph graph) {
+        url = IRIResolver.resolveString(url);
+        return withImportsWorker(url, graph);
+    }
+
+    private static Graph withImportsWorker(String url, Graph graph) {
+        // Partial check for any imports. Are there any imports triples?
+        boolean hasImports = G.contains(graph, null, nodeOwlImports, null);
+        if ( ! hasImports )
+            return graph;
+        // Probably some work to do.
+        // This is "import self", and start the "visited".
+        Graph acc = GraphFactory.createDefaultGraph();
+        GraphUtil.addInto(acc, graph);
+        Set<String> visited = new HashSet<>();
+        if ( url != null )
+            visited.add(url);
+        processImports(visited, graph, acc);
+        return acc;
+    }
+
+    /** Carefully traverse the imports, loading graphs. */
+    private static void processImports(Set<String> visited, Graph graph, Graph acc) {
+        List<Node> imports = imports(graph);
+        for ( Node imported : imports ) {
+            if ( ! imported.isURI() )
+                // Ignore non-URIs.
+                continue;
+            String uri = imported.getURI();
+            if ( ! visited.contains(uri) ) {
+                visited.add(uri);
+                // Read into a temporary graph to isolate errors.
+                try {
+                    Graph g2 = RDFDataMgr.loadGraph(uri);
+                    GraphUtil.addInto(acc, g2);
+                    processImports(visited, g2, acc);
+                } catch (RuntimeException ex) {}
+            }
+        }
+    }
+
+    /** Return the imports for a graph */
+    public static List<Node> imports(Graph graph) {
+        Pair<Node,List<Node>> pair = baseAndImports(graph);
+        return pair.getRight();
+    }
+
+    /**
+     * Locate the base (a single triple ? rdf:type owl:Ontology)
+     * and imports (triples "base owl:Imports URI").
+     * Returns a Pair of (null,EmptyList) for no base.
+     */
+    public static Pair<Node,List<Node>> baseAndImports(Graph graph) {
+        Node base = G.getZeroOrOnePO(graph, nodeRDFType, nodeOwlOntology);
+        if ( base == null )
+            return Pair.create(null, Collections.emptyList());
+        List<Node> imports = allImports(base, graph);
+        return Pair.create(base, imports);
+    }
+
+    /**
+     * Locate the base (a single triple ? rdf:type owl:Ontology).
+     * If none or more than one matching triple, then return null.
+     */
+    public static Node base(Graph graph) {
+        // Filter for URI?
+        return G.getZeroOrOnePO(graph, nodeRDFType, nodeOwlOntology);
+    }
+
+    /**
+     * Locate any imports (triples "base owl:Imports URI").
+     * Base may be a wildcard indicating "any owl:imports".
+     */
+    public static List<Node> allImports(Node base, Graph graph) {
+        List<Node> imports = iter(G.listSP(graph, base, nodeOwlImports)).filter(Node::isURI).collect(Collectors.toList());
+        return imports;
+    }
+}
+
diff --git a/jena-shacl/src/main/java/org/apache/jena/shacl/Shapes.java b/jena-shacl/src/main/java/org/apache/jena/shacl/Shapes.java
index 12401a0..8e10e70 100644
--- a/jena-shacl/src/main/java/org/apache/jena/shacl/Shapes.java
+++ b/jena-shacl/src/main/java/org/apache/jena/shacl/Shapes.java
@@ -18,19 +18,14 @@
 
 package org.apache.jena.shacl;
 
-import static org.apache.jena.sparql.graph.NodeConst.nodeOwlImports;
-import static org.apache.jena.sparql.graph.NodeConst.nodeOwlOntology;
-import static org.apache.jena.sparql.graph.NodeConst.nodeRDFType;
-
 import java.util.*;
 
 import org.apache.jena.atlas.iterator.Iter;
+import org.apache.jena.atlas.lib.Pair;
 import org.apache.jena.graph.Graph;
 import org.apache.jena.graph.Node;
 import org.apache.jena.rdf.model.Model;
 import org.apache.jena.riot.RDFDataMgr;
-import org.apache.jena.riot.other.G;
-import org.apache.jena.riot.other.RDFDataException;
 import org.apache.jena.shacl.engine.Targets;
 import org.apache.jena.shacl.parser.Shape;
 import org.apache.jena.shacl.parser.ShapesParser;
@@ -70,6 +65,16 @@ public class Shapes implements Iterable<Shape> {
         return parse(g);
     }
 
+    /** Load the file, parse the graph and return the shapes. */
+    public static Shapes parse(String fileOrURL, boolean withImports) {
+        Graph g = withImports
+            ? Imports.loadWithImports(fileOrURL)
+            : RDFDataMgr.loadGraph(fileOrURL);
+        return parse(g);
+    }
+
+
+    
     /** Parse the graph and return the shapes connected to the targets. */
     public static Shapes parse(Graph graph) {
         return parseAll(graph);
@@ -122,7 +127,7 @@ public class Shapes implements Iterable<Shape> {
         return new Shapes(shapesGraph, shapesMap, targets, rootShapes, declShapes);
     }
 
-    public Shapes(Graph shapesGraph, Map<Node, Shape> shapesMap, Targets targets,
+    private Shapes(Graph shapesGraph, Map<Node, Shape> shapesMap, Targets targets,
                   Collection<Shape> rootShapes, Collection<Shape> declShapes) {
         this.shapesGraph = shapesGraph;
         this.targets = targets;
@@ -130,15 +135,10 @@ public class Shapes implements Iterable<Shape> {
         this.rootShapes = rootShapes;
         this.declShapes = declShapes;
 
-        Node _shapesBase = null;
-        List<Node> _imports = null;
         // Extract base and imports.
-        try {
-            _shapesBase = G.getOnePO(shapesGraph, nodeRDFType, nodeOwlOntology);
-            _imports = G.listSP(shapesGraph, _shapesBase, nodeOwlImports);
-        } catch (RDFDataException ex) {}
-        this.shapesBase = _shapesBase;
-        this.imports = _imports;
+        Pair<Node,List<Node>> pair = Imports.baseAndImports(shapesGraph);
+        this.shapesBase = pair.getLeft();
+        this.imports = pair.getRight();
     }
 
     public boolean isEmpty() {
diff --git a/jena-shacl/src/main/java/org/apache/jena/shacl/parser/ShapesParser.java b/jena-shacl/src/main/java/org/apache/jena/shacl/parser/ShapesParser.java
index b4daec9..bb2e331 100644
--- a/jena-shacl/src/main/java/org/apache/jena/shacl/parser/ShapesParser.java
+++ b/jena-shacl/src/main/java/org/apache/jena/shacl/parser/ShapesParser.java
@@ -70,13 +70,13 @@ public class ShapesParser {
      * Applications should call functions in {@link Shapes} rather than call the parser directly.
      */
     public static Collection<Shape> parseShapes(Graph shapesGraph, Targets targets, Map<Node, Shape> shapesMap) {
-        // Cycle detection. 
+        // Cycle detection.
         Set<Node> cycles = new HashSet<>();
         return parseShapes(shapesGraph, targets, shapesMap, cycles);
     }
-    
+
     /*package*/ static Collection<Shape> parseShapes(Graph shapesGraph, Targets targets, Map<Node, Shape> shapesMap, Set<Node> cycles) {
-        
+
         Targets rootShapes = targets;
 
         if ( DEBUG )
@@ -173,21 +173,20 @@ public class ShapesParser {
 
     // ---- Main parser worker.
     /**
-     *  Parse one shape updating the record of shapes already parsed.
+     *  Parse one shape, updating the record of shapes already parsed.
      *
      * @param shapesMap
      * @param shapesGraph
      * @param shNode
      * @return Shape
      */
-    
+
     public static Shape parseShape(Map<Node, Shape> shapesMap, Graph shapesGraph, Node shNode) {
         Set<Node> traversed = new HashSet<>();
         Shape shape = parseShapeStep(traversed, shapesMap, shapesGraph, shNode);
         return shape;
     }
 
-
 //    /** Parse a specific shape from the Shapes graph */
 //    private static Shape parseShape(Graph shapesGraph, Node shNode) {
 //        // Avoid recursion.
@@ -230,18 +229,18 @@ public class ShapesParser {
     | sh:path         |
     -------------------
      */
-    
+
     /** Do nothing placeholder shape. */
-    static Shape unshape(Graph shapesGraph, Node shapeNode) { return 
+    static Shape unshape(Graph shapesGraph, Node shapeNode) { return
             new NodeShape(shapesGraph, shapeNode, false, Severity.Violation,
                           Collections.emptySet(), Collections.emptySet(),
                           Collections.singleton(new JLogConstraint("Cycle")),
                           Collections.emptySet());
     }
-    
+
     /** parse a shape during a parsing process */
     /*package*/ static Shape parseShapeStep(Set<Node> traversed, Map<Node, Shape> parsed, Graph shapesGraph, Node shapeNode) {
-        try { 
+        try {
             // Called by Constraints
             if ( parsed.containsKey(shapeNode) )
                 return parsed.get(shapeNode);
diff --git a/jena-shacl/src/test/files/imports/graph1.ttl b/jena-shacl/src/test/files/imports/graph1.ttl
new file mode 100644
index 0000000..1331124
--- /dev/null
+++ b/jena-shacl/src/test/files/imports/graph1.ttl
@@ -0,0 +1,12 @@
+PREFIX : <http://example/>
+
+PREFIX rdf:     <http://www.w3.org/1999/02/22-rdf-syntax-ns#> 
+PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>
+PREFIX sh:      <http://www.w3.org/ns/shacl#>
+PREFIX xsd:     <http://www.w3.org/2001/XMLSchema#>
+PREFIX owl:     <http://www.w3.org/2002/07/owl#>
+
+:graph1 rdf:type owl:Ontology ;
+        owl:imports <graph2.ttl> , <graph3.ttl> .
+
+:graph1 :p [] .
diff --git a/jena-shacl/src/test/files/imports/graph2.ttl b/jena-shacl/src/test/files/imports/graph2.ttl
new file mode 100644
index 0000000..2e71c06
--- /dev/null
+++ b/jena-shacl/src/test/files/imports/graph2.ttl
@@ -0,0 +1,13 @@
+PREFIX : <http://example/>
+
+PREFIX rdf:     <http://www.w3.org/1999/02/22-rdf-syntax-ns#> 
+PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>
+PREFIX sh:      <http://www.w3.org/ns/shacl#>
+PREFIX xsd:     <http://www.w3.org/2001/XMLSchema#>
+PREFIX owl:     <http://www.w3.org/2002/07/owl#>
+
+:graph2 rdf:type owl:Ontology .
+
+:graph2 owl:imports <graph4.ttl> .
+
+:graph2 :p [] .
diff --git a/jena-shacl/src/test/files/imports/graph3.ttl b/jena-shacl/src/test/files/imports/graph3.ttl
new file mode 100644
index 0000000..7ab1c13
--- /dev/null
+++ b/jena-shacl/src/test/files/imports/graph3.ttl
@@ -0,0 +1,16 @@
+PREFIX : <http://example/>
+
+PREFIX rdf:     <http://www.w3.org/1999/02/22-rdf-syntax-ns#> 
+PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>
+PREFIX sh:      <http://www.w3.org/ns/shacl#>
+PREFIX xsd:     <http://www.w3.org/2001/XMLSchema#>
+PREFIX owl:     <http://www.w3.org/2002/07/owl#>
+
+:graph3 rdf:type owl:Ontology .
+
+# Complicated - if starting at graph1, this is a cycle.
+:graph3 owl:imports <graph1.ttl> .
+
+:graph3 owl:imports <graph4.ttl> , <graph5.ttl> .
+
+:graph3 :p [] .
diff --git a/jena-shacl/src/test/files/imports/graph4.ttl b/jena-shacl/src/test/files/imports/graph4.ttl
new file mode 100644
index 0000000..8e68743
--- /dev/null
+++ b/jena-shacl/src/test/files/imports/graph4.ttl
@@ -0,0 +1,9 @@
+PREFIX : <http://example/>
+
+PREFIX rdf:     <http://www.w3.org/1999/02/22-rdf-syntax-ns#> 
+PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>
+PREFIX sh:      <http://www.w3.org/ns/shacl#>
+PREFIX xsd:     <http://www.w3.org/2001/XMLSchema#>
+PREFIX owl:     <http://www.w3.org/2002/07/owl#>
+
+:graph4 :p [] .
diff --git a/jena-shacl/src/test/files/imports/graph5.ttl b/jena-shacl/src/test/files/imports/graph5.ttl
new file mode 100644
index 0000000..2f2c7c4
--- /dev/null
+++ b/jena-shacl/src/test/files/imports/graph5.ttl
@@ -0,0 +1,9 @@
+PREFIX : <http://example/>
+
+PREFIX rdf:     <http://www.w3.org/1999/02/22-rdf-syntax-ns#> 
+PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>
+PREFIX sh:      <http://www.w3.org/ns/shacl#>
+PREFIX xsd:     <http://www.w3.org/2001/XMLSchema#>
+PREFIX owl:     <http://www.w3.org/2002/07/owl#>
+
+:graph5 :p [] .
diff --git a/jena-shacl/src/test/java/org/apache/jena/shacl/TC_SHACL.java b/jena-shacl/src/test/java/org/apache/jena/shacl/TC_SHACL.java
index 9fa247b..3ced5f8 100644
--- a/jena-shacl/src/test/java/org/apache/jena/shacl/TC_SHACL.java
+++ b/jena-shacl/src/test/java/org/apache/jena/shacl/TC_SHACL.java
@@ -19,6 +19,7 @@
 package org.apache.jena.shacl;
 
 import org.apache.jena.shacl.compact.TS_Compact;
+import org.apache.jena.shacl.tests.TestImports;
 import org.apache.jena.shacl.tests.TestValidationReport;
 import org.apache.jena.shacl.tests.jena_shacl.TS_JenaShacl;
 import org.apache.jena.shacl.tests.std.TS_StdSHACL;
@@ -31,6 +32,7 @@ import org.junit.runners.Suite;
     , TS_StdSHACL.class
     , TS_JenaShacl.class
     , TS_Compact.class
+    , TestImports.class
 } )
 
 public class TC_SHACL { }
diff --git a/jena-shacl/src/test/java/org/apache/jena/shacl/tests/TestImports.java b/jena-shacl/src/test/java/org/apache/jena/shacl/tests/TestImports.java
new file mode 100644
index 0000000..0d39d16
--- /dev/null
+++ b/jena-shacl/src/test/java/org/apache/jena/shacl/tests/TestImports.java
@@ -0,0 +1,115 @@
+/*
+ * 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
+ *
+ *     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.apache.jena.shacl.tests;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.jena.atlas.lib.Pair;
+import org.apache.jena.graph.Graph;
+import org.apache.jena.graph.Node;
+import org.apache.jena.graph.NodeFactory;
+import org.apache.jena.riot.RDFDataMgr;
+import org.apache.jena.riot.other.G;
+import org.apache.jena.riot.system.IRIResolver;
+import org.apache.jena.shacl.Imports;
+import org.junit.Test;
+
+public class TestImports {
+    // Work in absolute URIs.
+    private static String FILES = IRIResolver.resolveString("src/test/files/imports");
+    private static Node g1 = NodeFactory.createURI("http://example/graph1");
+    private static Node g2 = NodeFactory.createURI("http://example/graph2");
+    private static Node g3 = NodeFactory.createURI("http://example/graph3");
+    private static Node g4 = NodeFactory.createURI("http://example/graph4");
+    private static Node g5 = NodeFactory.createURI("http://example/graph5");
+
+    private static Node u1 = NodeFactory.createURI(FILES+"/graph1.ttl");
+    private static Node u2 = NodeFactory.createURI(FILES+"/graph2.ttl");
+    private static Node u3 = NodeFactory.createURI(FILES+"/graph3.ttl");
+    private static Node u4 = NodeFactory.createURI(FILES+"/graph4.ttl");
+    private static Node u5 = NodeFactory.createURI(FILES+"/graph5.ttl");
+
+    private static Node predicate = NodeFactory.createURI("http://example/p");
+
+    @Test public void testImports1() {
+        Graph graph = RDFDataMgr.loadGraph(FILES+"/graph1.ttl");
+        Node base = Imports.base(graph);
+        assertEquals(g1, base);
+    }
+
+    @Test public void testImports2() {
+        Graph graph = RDFDataMgr.loadGraph(FILES+"/graph1.ttl");
+        List<Node> imports = Imports.imports(graph);
+        assertEquals(2, imports.size());
+        assertTrue(imports.contains(u2));
+        assertTrue(imports.contains(u3));
+    }
+
+    @Test public void testImports3() {
+        Graph graph = RDFDataMgr.loadGraph(FILES+"/graph1.ttl");
+
+        Pair<Node, List<Node>> pair = Imports.baseAndImports(graph);
+        Node base = pair.getLeft();
+        List<Node> imports = pair.getRight();
+
+        assertEquals(g1, base);
+        assertEquals(2, imports.size());
+        assertTrue(imports.contains(u2));
+        assertTrue(imports.contains(u3));
+    }
+
+    @Test public void testImportsLoading1() {
+        Graph graph = Imports.loadWithImports(FILES+"/graph1.ttl");
+        // Used blank nodes to detect loaded once or multiple times.
+        //RDFDataMgr.write(System.out, graph, Lang.TTL);
+        assertTrue(G.containsOne(graph, g1, predicate, null));
+        assertTrue(G.containsOne(graph, g2, predicate, null));
+        assertTrue(G.containsOne(graph, g3, predicate, null));
+        assertTrue(G.containsOne(graph, g4, predicate, null));
+        assertTrue(G.containsOne(graph, g5, predicate, null));
+    }
+
+    @Test public void testImportsLoading2() {
+        Graph graph1 = RDFDataMgr.loadGraph(FILES+"/graph1.ttl");
+        Graph graph = Imports.withImports(FILES+"/graph1.ttl",graph1);
+        assertTrue(G.containsOne(graph, g1, predicate, null));
+        assertTrue(G.containsOne(graph, g2, predicate, null));
+        assertTrue(G.containsOne(graph, g3, predicate, null));
+        assertTrue(G.containsOne(graph, g4, predicate, null));
+        assertTrue(G.containsOne(graph, g5, predicate, null));
+    }
+
+    @Test public void testImportsLoading3() {
+        Graph graph1 = RDFDataMgr.loadGraph(FILES+"/graph1.ttl");
+        Graph graph = Imports.withImports(graph1);
+        // Will be read again due to not knowing it URI.
+        // Skip test.
+        // assertTrue(G.containsOne(graph, g1, p, null));
+        assertTrue(G.containsOne(graph, g2, predicate, null));
+        assertTrue(G.containsOne(graph, g3, predicate, null));
+        assertTrue(G.containsOne(graph, g4, predicate, null));
+        assertTrue(G.containsOne(graph, g5, predicate, null));
+    }
+
+
+}
+