You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@jena.apache.org by GitBox <gi...@apache.org> on 2021/04/22 10:19:18 UTC

[GitHub] [jena] rvesse commented on a change in pull request #984: JENA-2088: Quad execution. Rework the basic graph pattern execution. Code clean up.

rvesse commented on a change in pull request #984:
URL: https://github.com/apache/jena/pull/984#discussion_r618251844



##########
File path: jena-arq/src/main/java/org/apache/jena/sparql/engine/main/solver/PatternMatchData.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.sparql.engine.main.solver;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Predicate;
+
+import org.apache.jena.graph.Graph;
+import org.apache.jena.graph.Node;
+import org.apache.jena.graph.Triple;
+import org.apache.jena.sparql.core.BasicPattern;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.Quad;
+import org.apache.jena.sparql.engine.ExecutionContext;
+import org.apache.jena.sparql.engine.QueryIterator;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.engine.iterator.Abortable;
+import org.apache.jena.sparql.engine.iterator.QueryIterAbortable;
+
+/**
+ * Match a graph node + basic graph pattern.
+ */
+public class PatternMatchData {
+
+    /**
+     * Non-reordering execution of a triple pattern (basic graph pattern),
+     * given an iterator of bindings as input.
+     */
+    public static QueryIterator execute(Graph graph, BasicPattern pattern,
+                                        QueryIterator input, Predicate<Triple> filter,

Review comment:
       `filter` argument never used?

##########
File path: jena-arq/src/main/java/org/apache/jena/sparql/engine/main/solver/PatternMatchData.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.sparql.engine.main.solver;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Predicate;
+
+import org.apache.jena.graph.Graph;
+import org.apache.jena.graph.Node;
+import org.apache.jena.graph.Triple;
+import org.apache.jena.sparql.core.BasicPattern;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.Quad;
+import org.apache.jena.sparql.engine.ExecutionContext;
+import org.apache.jena.sparql.engine.QueryIterator;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.engine.iterator.Abortable;
+import org.apache.jena.sparql.engine.iterator.QueryIterAbortable;
+
+/**
+ * Match a graph node + basic graph pattern.
+ */
+public class PatternMatchData {
+
+    /**
+     * Non-reordering execution of a triple pattern (basic graph pattern),
+     * given an iterator of bindings as input.
+     */
+    public static QueryIterator execute(Graph graph, BasicPattern pattern,
+                                        QueryIterator input, Predicate<Triple> filter,
+                                        ExecutionContext execCxt)
+    {
+        List<Triple> triples = pattern.getList();
+
+        Iterator<Binding> chain = input;
+        List<Abortable> killList = new ArrayList<>();
+
+        for ( Triple triple : triples ) {
+            // Plain, no RDF-star
+            //chain = StageMatchTriple.accessTriple(chain, graph, triple, filter, execCxt);
+
+            // [Match] Missing filter.
+            chain = SolverRX3.rdfStarTriple(chain, triple, execCxt);
+            chain = SolverLib.makeAbortable(chain, killList);
+        }
+
+        // "input" will be closed by QueryIterAbortable but is otherwise unused.
+        // "killList" will be aborted on timeout.
+        return new QueryIterAbortable(chain, killList, input, execCxt);
+    }
+
+    /** Non-reordering execution of a quad pattern, a graph name and a basic graph pattern,
+     *  given an iterator of bindings as input.
+     *  <p>
+     *  GraphNode is Node.ANY for execution over the union of named graphs.<br/>
+     *  GraphNode is null for execution over the real default graph.
+     */
+    public static QueryIterator execute(DatasetGraph dsg, Node graphNode, BasicPattern pattern,
+                                        QueryIterator input, Predicate<Quad> filter,
+                                        ExecutionContext execCxt)
+    {
+        // Translate:
+        //   graphNode may be Node.ANY, meaning we should make triples unique.
+        //   graphNode may be null, meaning default graph
+        if ( Quad.isUnionGraph(graphNode) )
+            graphNode = Node.ANY;
+        if ( Quad.isDefaultGraph(graphNode) )
+            graphNode = null;
+
+        List<Triple> triples = pattern.getList();
+        boolean isDefaultGraph = (graphNode == null);
+        boolean anyGraph = isDefaultGraph ? false : (Node.ANY.equals(graphNode));
+
+        Iterator<Binding> chain = input;
+        List<Abortable> killList = new ArrayList<>();
+
+        for ( Triple triple : triples ) {
+            // Plain - no RDF-star.
+            //chain = StageMatchData.accessQuad(chain, graphNode, triple, filter, anyGraph, execCxt);

Review comment:
       Again how is RDF-star disabled?

##########
File path: jena-arq/src/main/java/org/apache/jena/sparql/engine/main/solver/StageMatchData.java
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.sparql.engine.main.solver;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import org.apache.jena.atlas.iterator.Iter;
+import org.apache.jena.graph.Node;
+import org.apache.jena.graph.Triple;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.Quad;
+import org.apache.jena.sparql.core.Var;
+import org.apache.jena.sparql.engine.ExecutionContext;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.engine.binding.BindingBuilder;
+import org.apache.jena.sparql.engine.binding.BindingFactory;
+
+/**
+ * This is the data access step of quads and datasets.
+ * <p>
+ * Plain matching - see {@link SolverRX4#rdfStarQuad} for matching with variables
+ * in RDF-star embedded triples and
+ */
+public class StageMatchData {
+
+    // Positions in Tuple4/Quad
+    private static int QG = 0 ;
+    private static int QS = 1 ;
+    private static int QP = 2 ;
+    private static int QO = 3 ;
+
+    private static Function<Quad, Quad> quadsToUnion =
+            quad -> Quad.create(Quad.unionGraph, quad.getSubject(), quad.getPredicate(), quad.getObject());
+
+    /*
+     * Entry point from PattenMatchData.
+     *   graphNode may be Node.ANY, meaning union graph and should make triples unique.
+     *   graphNode may be null, meaning default graph
+     */
+    static Iterator<Binding> accessQuad(Iterator<Binding> input, Node graphName, Triple pattern, Predicate<Quad> filter, boolean anyGraph, ExecutionContext execCxt) {
+        return Iter.flatMap(input, binding -> {
+            return accessQuad(binding, graphName, pattern, filter, anyGraph, execCxt);
+        });
+    }
+
+    static Iterator<Binding> accessQuad(Binding binding, Node graphName, Triple pattern, Predicate<Quad> filter, boolean anyGraph, ExecutionContext execCxt) {
+        // Assumes if anyGraph, then graphName ==ANY.

Review comment:
       Since this is `package` scoped code I guess calling code will always maintain this assumption?

##########
File path: jena-tdb/src/main/java/org/apache/jena/tdb/solver/StageMatchTuple.java
##########
@@ -90,24 +90,25 @@
             iterMatches = Iter.distinctAdjacent(iterMatches);
         }
 
-        // Map Tuple<NodeId> to BindingNodeId
-        Function<Tuple<NodeId>, BindingNodeId> binder = tuple -> {
-            BindingNodeId output = new BindingNodeId(input);
-            for ( int i = 0 ; i < var.length ; i++ ) {
-                Var v = var[i];
-                if ( v == null )
-                    continue;
-                NodeId id = tuple.get(i);
-                if ( reject(output, v, id) )
-                    return null;
-                output.put(v, id);
-            }
-            return output;
-        };
-
+        Function<Tuple<NodeId>, BindingNodeId> binder = tuple -> tupleToBinding(input, tuple, vars);
         return Iter.iter(iterMatches).map(binder).removeNulls();
     }
 
+    private static BindingNodeId tupleToBinding(BindingNodeId input, Tuple<NodeId> tuple, Var[] var) {
+        // Reuseable BindingNodeId builder?
+        BindingNodeId output = new BindingNodeId(input);
+        for ( int i = 0 ; i < var.length ; i++ ) {
+            Var v = var[i];
+            if ( v == null )
+                continue;
+            NodeId id = tuple.get(i);
+            if ( ! compatiable(output, v, id) )

Review comment:
       Again typo

##########
File path: jena-arq/src/main/java/org/apache/jena/sparql/engine/main/solver/PatternMatchData.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.sparql.engine.main.solver;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Predicate;
+
+import org.apache.jena.graph.Graph;
+import org.apache.jena.graph.Node;
+import org.apache.jena.graph.Triple;
+import org.apache.jena.sparql.core.BasicPattern;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.Quad;
+import org.apache.jena.sparql.engine.ExecutionContext;
+import org.apache.jena.sparql.engine.QueryIterator;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.engine.iterator.Abortable;
+import org.apache.jena.sparql.engine.iterator.QueryIterAbortable;
+
+/**
+ * Match a graph node + basic graph pattern.
+ */
+public class PatternMatchData {
+
+    /**
+     * Non-reordering execution of a triple pattern (basic graph pattern),
+     * given an iterator of bindings as input.
+     */
+    public static QueryIterator execute(Graph graph, BasicPattern pattern,
+                                        QueryIterator input, Predicate<Triple> filter,
+                                        ExecutionContext execCxt)
+    {
+        List<Triple> triples = pattern.getList();
+
+        Iterator<Binding> chain = input;
+        List<Abortable> killList = new ArrayList<>();
+
+        for ( Triple triple : triples ) {
+            // Plain, no RDF-star
+            //chain = StageMatchTriple.accessTriple(chain, graph, triple, filter, execCxt);
+
+            // [Match] Missing filter.
+            chain = SolverRX3.rdfStarTriple(chain, triple, execCxt);
+            chain = SolverLib.makeAbortable(chain, killList);
+        }
+
+        // "input" will be closed by QueryIterAbortable but is otherwise unused.
+        // "killList" will be aborted on timeout.
+        return new QueryIterAbortable(chain, killList, input, execCxt);
+    }
+
+    /** Non-reordering execution of a quad pattern, a graph name and a basic graph pattern,
+     *  given an iterator of bindings as input.
+     *  <p>
+     *  GraphNode is Node.ANY for execution over the union of named graphs.<br/>
+     *  GraphNode is null for execution over the real default graph.
+     */
+    public static QueryIterator execute(DatasetGraph dsg, Node graphNode, BasicPattern pattern,
+                                        QueryIterator input, Predicate<Quad> filter,

Review comment:
       Again `filter` is unused?

##########
File path: jena-integration-tests/src/test/java/org/apache/jena/test/TestSettings.java
##########
@@ -30,7 +30,8 @@
 
     @Test
     public void setting_RDFstar_fastpath() {
-        assertTrue(org.apache.jena.sparql.engine.iterator.RX.DATAPATH);
+        assertTrue(org.apache.jena.sparql.engine.main.solver.SolverRX3.DATAPATH);
+        assertTrue(org.apache.jena.sparql.engine.main.solver.SolverRX4.DATAPATH);

Review comment:
       ???

##########
File path: jena-arq/src/main/java/org/apache/jena/sparql/engine/main/solver/SolverRX4.java
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.sparql.engine.main.solver;
+
+import static org.apache.jena.sparql.engine.main.solver.SolverLib.nodeTopLevel;
+import static org.apache.jena.sparql.engine.main.solver.SolverLib.sameTermAs;
+import static org.apache.jena.sparql.engine.main.solver.SolverLib.tripleHasEmbTripleWithVars;
+
+import java.util.Iterator;
+
+import org.apache.jena.atlas.iterator.Iter;
+import org.apache.jena.graph.Graph;
+import org.apache.jena.graph.Node;
+import org.apache.jena.graph.Triple;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.Quad;
+import org.apache.jena.sparql.core.Var;
+import org.apache.jena.sparql.engine.ExecutionContext;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.engine.binding.BindingBuilder;
+
+/** RDF-star - quad form. Solve patterns when there is an embedded triple with variables. */
+public class SolverRX4 {
+
+    /**
+     * This constant is not public API. It is exposed only so integration testing can
+     * check the value for a release build.
+     */
+    public static final boolean DATAPATH = true;
+
+    public static Iterator<Binding> rdfStarQuad(Iterator<Binding> chain, Quad pattern, ExecutionContext execCxt) {
+        return rdfStarQuad(chain, pattern.getGraph(), pattern.asTriple(), execCxt) ;
+    }
+
+    public static Iterator<Binding> rdfStarQuad(Iterator<Binding> chain, Node graphName, Triple pattern, ExecutionContext execCxt) {
+        // Should all work without this trap for plain RDF.
+        if ( DATAPATH ) {
+            if ( ! tripleHasEmbTripleWithVars(pattern) &&
+                    (graphName == null || ! graphName.isNodeTriple() ) )
+                // No variables inside <<>>
+                return matchDataQuad(chain, graphName, pattern, execCxt);
+        }
+        return rdfStarQuadSub(chain, graphName, pattern, execCxt);
+    }
+
+    private static Iterator<Binding> rdfStarQuadSub(Iterator<Binding> chain, Node graphName, Triple pattern, ExecutionContext execCxt) {
+        Iterator<Binding> matches = Iter.flatMap(chain, binding->rdfStarQuadSub(binding, graphName, pattern, execCxt));
+        return matches;
+    }
+
+    private static Iterator<Binding> rdfStarQuadSub(Binding binding, Node graphName, Triple tPattern, ExecutionContext execCxt) {
+        Node g = (graphName == null) ? Quad.defaultGraphIRI : nodeTopLevel(graphName);
+        Node s = nodeTopLevel(tPattern.getSubject());
+        Node p = nodeTopLevel(tPattern.getPredicate());
+        Node o = nodeTopLevel(tPattern.getObject());
+        DatasetGraph dsg = execCxt.getDataset();
+        Iterator<Quad> dataIter = dsg.find(g, s, p, o);
+        Iterator<Binding> matched = Iter.map(dataIter, qData->matchQuad(binding, qData, g, tPattern));
+        return Iter.removeNulls(matched);
+    }
+
+    private static Iterator<Binding> matchDataQuad(Iterator<Binding> chain, Node graphName, Triple pattern, ExecutionContext execCxt) {
+        Iterator<Binding> matches = StageMatchData.accessQuad(chain, graphName, pattern, /*filtr*/null, /*anyGraph*/false, execCxt);

Review comment:
       This is the type of thing where having named parameters would make the code clearer.
   
   Alternatively maybe define constants `NO_FILTER` and `SPECIFIC_GRAPH` to make intent clearer?

##########
File path: jena-db/jena-tdb2/src/main/java/org/apache/jena/tdb2/solver/StageMatchTuple.java
##########
@@ -90,24 +90,26 @@
             iterMatches = Iter.distinctAdjacent(iterMatches);
         }
 
-        // Map Tuple<NodeId> to BindingNodeId
-        Function<Tuple<NodeId>, BindingNodeId> binder = tuple -> {
-            BindingNodeId output = new BindingNodeId(input);
-            for ( int i = 0 ; i < var.length ; i++ ) {
-                Var v = var[i];
-                if ( v == null )
-                    continue;
-                NodeId id = tuple.get(i);
-                if ( reject(output, v, id) )
-                    return null;
-                output.put(v, id);
-            }
-            return output;
-        };
 
+        Function<Tuple<NodeId>, BindingNodeId> binder = tuple -> tupleToBinding(input, tuple, vars);
         return Iter.iter(iterMatches).map(binder).removeNulls();
     }
 
+    private static BindingNodeId tupleToBinding(BindingNodeId input, Tuple<NodeId> tuple, Var[] var) {
+        // Reuseable BindingNodeId builder?
+        BindingNodeId output = new BindingNodeId(input);
+        for ( int i = 0 ; i < var.length ; i++ ) {
+            Var v = var[i];
+            if ( v == null )
+                continue;
+            NodeId id = tuple.get(i);
+            if ( ! compatiable(output, v, id) )

Review comment:
       Typo - `compatiable` -> `compatible`

##########
File path: jena-arq/src/main/java/org/apache/jena/sparql/engine/main/solver/PatternMatchData.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.sparql.engine.main.solver;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Predicate;
+
+import org.apache.jena.graph.Graph;
+import org.apache.jena.graph.Node;
+import org.apache.jena.graph.Triple;
+import org.apache.jena.sparql.core.BasicPattern;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.Quad;
+import org.apache.jena.sparql.engine.ExecutionContext;
+import org.apache.jena.sparql.engine.QueryIterator;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.engine.iterator.Abortable;
+import org.apache.jena.sparql.engine.iterator.QueryIterAbortable;
+
+/**
+ * Match a graph node + basic graph pattern.
+ */
+public class PatternMatchData {
+
+    /**
+     * Non-reordering execution of a triple pattern (basic graph pattern),
+     * given an iterator of bindings as input.
+     */
+    public static QueryIterator execute(Graph graph, BasicPattern pattern,
+                                        QueryIterator input, Predicate<Triple> filter,
+                                        ExecutionContext execCxt)
+    {
+        List<Triple> triples = pattern.getList();
+
+        Iterator<Binding> chain = input;
+        List<Abortable> killList = new ArrayList<>();
+
+        for ( Triple triple : triples ) {
+            // Plain, no RDF-star
+            //chain = StageMatchTriple.accessTriple(chain, graph, triple, filter, execCxt);

Review comment:
       Not sure I follow how RDF-start is disabled with these changes?

##########
File path: jena-arq/src/main/java/org/apache/jena/sparql/expr/nodevalue/NodeFunctions.java
##########
@@ -139,31 +140,24 @@ public static NodeValue sameTerm(NodeValue nv1, NodeValue nv2) {
         return NodeValue.booleanReturn(sameTerm(nv1.asNode(), nv2.asNode())) ;
     }
 
-    public static boolean sameTerm(Node n1, Node n2) {
-        if ( n1.equals(n2) )
+    public static boolean sameTerm(Node node1, Node node2) {
+        if ( node1.equals(node2) )
             return true ;
-        if ( n1.isLiteral() && n2.isLiteral() ) {
-            // But language tags are case insensitive.
-            String lang1 = n1.getLiteralLanguage() ;
-            String lang2 = n2.getLiteralLanguage() ;
-
-            if ( !lang1.equals("") && lang1.equalsIgnoreCase(lang2) ) {
-                // Two language tags, equal by case insensitivity.
-                boolean b = n1.getLiteralLexicalForm().equals(n2.getLiteralLexicalForm()) ;
-                if ( b )
-                    return true ;
-            }
+        if ( Util.isLangString(node1) && Util.isLangString(node2) ) {
+            String lex1 = node1.getLiteralLexicalForm();
+            String lex2 = node2.getLiteralLexicalForm();
+            if ( !lex1.equals(lex2) )
+                return false;
+            return node1.getLiteralLanguage().equalsIgnoreCase(node2.getLiteralLanguage());

Review comment:
       I guess this is accounting for messy data with `"foo"@en-us` and `"foo"@en-US` since specs do say that language tags should be case insensitive

##########
File path: jena-tdb/src/main/java/org/apache/jena/tdb/solver/SolverRX.java
##########
@@ -35,13 +36,13 @@
 import org.apache.jena.sparql.engine.ExecutionContext;
 import org.apache.jena.sparql.engine.binding.Binding;
 import org.apache.jena.sparql.engine.binding.BindingFactory;
-import org.apache.jena.sparql.engine.iterator.RX;
+import org.apache.jena.sparql.engine.main.solver.SolverRX4;
 import org.apache.jena.tdb.lib.TupleLib;
 import org.apache.jena.tdb.store.NodeId;
 import org.apache.jena.tdb.store.nodetable.NodeTable;
 import org.apache.jena.tdb.store.nodetupletable.NodeTupleTable;
 
-/** RDF-start processing for matching a triple/quad pattern. */
+/** RDF-start processing for matching an individual triple/quad pattern. */

Review comment:
       Typo - `RDF-start` -> `RDF-star`

##########
File path: jena-db/jena-tdb2/src/main/java/org/apache/jena/tdb2/solver/SolverRX.java
##########
@@ -94,7 +94,7 @@
         Iterator<Quad> dsgIter = accessData(patternTuple, nodeTupleTable, anyGraph, filter, execCxt);
 
         Binding input = bnid.isEmpty() ? BindingFactory.empty() : new BindingTDB(bnid, nodeTable);
-        Iterator<Binding> matched = Iter.iter(dsgIter).map(dQuad->RX.matchQuad(input, dQuad, tGraphNode, tPattern)).removeNulls();
+        Iterator<Binding> matched = Iter.iter(dsgIter).map(dQuad->SolverRX4.matchQuad(input, dQuad, tGraphNode, tPattern)).removeNulls();

Review comment:
       Does using the general purpose solver mean `Node` get materialized sooner or is the TDB `DatasetGraph` going to return special `Node` instances backed by internal Node IDs?

##########
File path: jena-arq/src/main/java/org/apache/jena/sparql/engine/main/solver/SolverLib.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.sparql.engine.main.solver;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.jena.graph.Node;
+import org.apache.jena.graph.Triple;
+import org.apache.jena.sparql.core.Quad;
+import org.apache.jena.sparql.core.Var;
+import org.apache.jena.sparql.engine.iterator.Abortable;
+import org.apache.jena.sparql.engine.iterator.IterAbortable;
+import org.apache.jena.sparql.expr.nodevalue.NodeFunctions;
+
+public class SolverLib {
+
+    /**
+     * Create an abortable iterator, storing it in the killList.
+     * Just return the input iterator if kilList is null.
+     */
+    public static <T> Iterator<T> makeAbortable(Iterator<T> iter, List<Abortable> killList) {
+        if ( killList == null )
+            return iter;
+        IterAbortable<T> k = new IterAbortable<>(iter);
+        killList.add(k);
+        return k;
+    }
+
+    /**
+     * Test whether a triple has an triple term as one of its components.
+     */
+    public static boolean tripleHasNodeTriple(Triple triple) {
+        return triple.getSubject().isNodeTriple()
+               /*|| triple.getPredicate().isNodeTriple()*/

Review comment:
       RDF-star does not allow triple predicates currently?

##########
File path: jena-integration-tests/src/test/java/org/apache/jena/test/TestSettings.java
##########
@@ -30,7 +30,8 @@
 
     @Test
     public void setting_RDFstar_fastpath() {
-        assertTrue(org.apache.jena.sparql.engine.iterator.RX.DATAPATH);
+        assertTrue(org.apache.jena.sparql.engine.main.solver.SolverRX3.DATAPATH);
+        assertTrue(org.apache.jena.sparql.engine.main.solver.SolverRX4.DATAPATH);

Review comment:
       What is this testing?

##########
File path: jena-arq/src/main/java/org/apache/jena/sparql/expr/nodevalue/NodeFunctions.java
##########
@@ -172,14 +166,20 @@ public static boolean rdfTermEquals(Node n1, Node n2) {
             // Two literals, may be sameTerm by language tag case insensitivity.
             String lang1 = n1.getLiteralLanguage() ;
             String lang2 = n2.getLiteralLanguage() ;
-
-            if ( !lang1.equals("") && lang1.equalsIgnoreCase(lang2) ) {
-                // Two language tags, equal by case insensitivity.
-                boolean b = n1.getLiteralLexicalForm().equals(n2.getLiteralLexicalForm()) ;
-                if ( b )
-                    return true ;
+            if ( ! lang1.equals("") && ! lang2.equals("") ) {

Review comment:
       Can `lang1`/`lang2` ever be `null`?  Would `StringUtils.isNotBlank()` be safer (assuming we have `commons-lang` as a dependency here)?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@jena.apache.org
For additional commands, e-mail: pr-help@jena.apache.org