You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by ca...@apache.org on 2017/07/20 21:16:55 UTC

incubator-rya git commit: RYA-291 Added owl:hasValue inference. Closes #174.

Repository: incubator-rya
Updated Branches:
  refs/heads/master abe7d71b1 -> 2b73c3062


RYA-291 Added owl:hasValue inference. Closes #174.

Given a type associated with a hasValue property restriction: 1) expand queries
for members of the type to also check for anything with the value; and 2) expand
queries for values of that property to check for instances of the type.


Project: http://git-wip-us.apache.org/repos/asf/incubator-rya/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-rya/commit/2b73c306
Tree: http://git-wip-us.apache.org/repos/asf/incubator-rya/tree/2b73c306
Diff: http://git-wip-us.apache.org/repos/asf/incubator-rya/diff/2b73c306

Branch: refs/heads/master
Commit: 2b73c30628013ccd785007c7928492c2ac8cfead
Parents: abe7d71
Author: Jesse Hatfield <je...@parsons.com>
Authored: Mon Jun 26 14:06:06 2017 -0400
Committer: Caleb Meier <ca...@parsons.com>
Committed: Thu Jul 20 12:30:15 2017 -0700

----------------------------------------------------------------------
 .../RdfCloudTripleStoreConnection.java          |   2 +
 .../inference/AbstractInferVisitor.java         |   6 +-
 .../inference/HasValueVisitor.java              | 141 +++++++++++++
 .../inference/InferenceEngine.java              | 119 ++++++++++-
 .../inference/HasValueVisitorTest.java          | 197 +++++++++++++++++++
 .../inference/InferenceEngineTest.java          | 167 ++++++++++++++++
 .../rdftriplestore/inference/InferenceIT.java   | 181 +++++++++++++++++
 7 files changed, 806 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2b73c306/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java b/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
index ea8db77..44b0bd4 100644
--- a/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
+++ b/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
@@ -49,6 +49,7 @@ import org.apache.rya.rdftriplestore.evaluation.QueryJoinSelectOptimizer;
 import org.apache.rya.rdftriplestore.evaluation.RdfCloudTripleStoreEvaluationStatistics;
 import org.apache.rya.rdftriplestore.evaluation.RdfCloudTripleStoreSelectivityEvaluationStatistics;
 import org.apache.rya.rdftriplestore.evaluation.SeparateFilterJoinsVisitor;
+import org.apache.rya.rdftriplestore.inference.HasValueVisitor;
 import org.apache.rya.rdftriplestore.inference.InferenceEngine;
 import org.apache.rya.rdftriplestore.inference.InverseOfVisitor;
 import org.apache.rya.rdftriplestore.inference.PropertyChainVisitor;
@@ -345,6 +346,7 @@ public class RdfCloudTripleStoreConnection extends SailConnectionBase {
                     && this.inferenceEngine != null
                     ) {
                 try {
+                    tupleExpr.visit(new HasValueVisitor(queryConf, inferenceEngine));
                     tupleExpr.visit(new PropertyChainVisitor(queryConf, inferenceEngine));
                     tupleExpr.visit(new TransitivePropertyVisitor(queryConf, inferenceEngine));
                     tupleExpr.visit(new SymmetricPropertyVisitor(queryConf, inferenceEngine));

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2b73c306/sail/src/main/java/org/apache/rya/rdftriplestore/inference/AbstractInferVisitor.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/org/apache/rya/rdftriplestore/inference/AbstractInferVisitor.java b/sail/src/main/java/org/apache/rya/rdftriplestore/inference/AbstractInferVisitor.java
index 5480d95..b1c7eb5 100644
--- a/sail/src/main/java/org/apache/rya/rdftriplestore/inference/AbstractInferVisitor.java
+++ b/sail/src/main/java/org/apache/rya/rdftriplestore/inference/AbstractInferVisitor.java
@@ -19,13 +19,9 @@ package org.apache.rya.rdftriplestore.inference;
  * under the License.
  */
 
-
-
 import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 import org.apache.rya.rdftriplestore.utils.FixedStatementPattern;
 import org.apache.rya.rdftriplestore.utils.TransitivePropertySP;
-import org.apache.rya.rdftriplestore.utils.FixedStatementPattern;
-import org.apache.rya.rdftriplestore.utils.TransitivePropertySP;
 import org.openrdf.query.algebra.Join;
 import org.openrdf.query.algebra.StatementPattern;
 import org.openrdf.query.algebra.Union;
@@ -39,7 +35,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
  * Date: Mar 14, 2012
  * Time: 5:33:01 PM
  */
-public class AbstractInferVisitor extends QueryModelVisitorBase {
+public class AbstractInferVisitor extends QueryModelVisitorBase<Exception> {
 
     static Var EXPANDED = new Var("infer-expanded");
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2b73c306/sail/src/main/java/org/apache/rya/rdftriplestore/inference/HasValueVisitor.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/org/apache/rya/rdftriplestore/inference/HasValueVisitor.java b/sail/src/main/java/org/apache/rya/rdftriplestore/inference/HasValueVisitor.java
new file mode 100644
index 0000000..a7e6bc2
--- /dev/null
+++ b/sail/src/main/java/org/apache/rya/rdftriplestore/inference/HasValueVisitor.java
@@ -0,0 +1,141 @@
+package org.apache.rya.rdftriplestore.inference;
+/*
+ * 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.
+ */
+
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
+import org.apache.rya.api.utils.NullableStatementImpl;
+import org.apache.rya.rdftriplestore.utils.FixedStatementPattern;
+import org.openrdf.model.Resource;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.vocabulary.OWL;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.TupleExpr;
+import org.openrdf.query.algebra.Var;
+
+/**
+ * Expands the query tree to account for any relevant has-value class
+ * expressions in the ontology known to the {@link InferenceEngine}.
+ *
+ * Only operates on {@link StatementPattern} nodes, and only those including a
+ * defined type or defined predicate which is relevant to a has-value
+ * expression in the ontology. When applicable, replaces the node with one or
+ * more nested {@link InferUnion}s, one of whose leaves is the original
+ * StatementPattern.
+ *
+ * A has-value class expression references a specific predicate and a specific
+ * value (object or literal), and represents the set of all individuals having
+ * the specified value for the specified predicate. This has two implications
+ * for inference: 1) If an individual has the specified value for the specified
+ * predicate, then it implicitly belongs to the has-value class expression; and
+ * 2) If an individual belongs to the has-value class expression, then it
+ * implicitly has the specified value for the specified predicate.
+ *
+ * To handle the first case, the visitor expands statement patterns of the form
+ * "?individual rdf:type :class" if the class or any of its subclasses are known
+ * to be has-value class expressions. (Does not apply if the class term
+ * is a variable.) The resulting query tree will match individuals who
+ * implicitly belong to the class expression because they have the specified
+ * value for the specified property, as well as any individuals explicitly
+ * stated to belong to the class.
+ *
+ * To handle the second case, the visitor expands statement patterns of the form
+ * "?individual :predicate ?value" if the predicate is known to be referenced by
+ * any has-value expression. (Does not apply if the predicate term is a
+ * variable.) The resulting query tree will match individuals and values that
+ * can be derived from the individual's membership in a has-value class
+ * expression (which itself may be explicit or derived from membership in a
+ * subclass of the has-value class expression), as well as any individuals and
+ * values explicitly stated.
+ */
+public class HasValueVisitor extends AbstractInferVisitor {
+    /**
+     * Creates a new {@link HasValueVisitor}, which is enabled by default.
+     * @param conf The {@link RdfCloudTripleStoreConfiguration}.
+     * @param inferenceEngine The InferenceEngine containing the relevant ontology.
+     */
+    public HasValueVisitor(RdfCloudTripleStoreConfiguration conf, InferenceEngine inferenceEngine) {
+        super(conf, inferenceEngine);
+        include = true;
+    }
+
+    /**
+     * Checks whether facts matching the StatementPattern could be derived using
+     * has-value inference, and if so, replaces the StatementPattern node with a
+     * union of itself and any such possible derivations.
+     */
+    @Override
+    protected void meetSP(StatementPattern node) throws Exception {
+        final Var subjVar = node.getSubjectVar();
+        final Var predVar = node.getPredicateVar();
+        final Var objVar = node.getObjectVar();
+        // We can reason over two types of statement patterns:
+        // { ?var rdf:type :Restriction } and { ?var :property ?value }
+        // Both require defined predicate
+        if (predVar != null && predVar.getValue() != null) {
+            final URI predURI = (URI) predVar.getValue();
+            if (RDF.TYPE.equals(predURI) && objVar != null && objVar.getValue() != null
+                    && objVar.getValue() instanceof Resource) {
+                // If the predicate is rdf:type and the type is specified, check whether it can be
+                // inferred using any hasValue restriction(s)
+                final Resource objType = (Resource) objVar.getValue();
+                final Map<URI, Set<Value>> sufficientValues = inferenceEngine.getHasValueByType(objType);
+                if (sufficientValues.size() > 0) {
+                    final Var valueVar = new Var("v-" + UUID.randomUUID());
+                    TupleExpr currentNode = node.clone();
+                    for (URI property : sufficientValues.keySet()) {
+                        final Var propVar = new Var(property.toString(), property);
+                        final TupleExpr valueSP = new DoNotExpandSP(subjVar, propVar, valueVar);
+                        final FixedStatementPattern relevantValues = new FixedStatementPattern(objVar, propVar, valueVar);
+                        for (Value value : sufficientValues.get(property)) {
+                            relevantValues.statements.add(new NullableStatementImpl(objType, property, value));
+                        }
+                        currentNode = new InferUnion(currentNode, new InferJoin(relevantValues, valueSP));
+                    }
+                    node.replaceWith(currentNode);
+                }
+            }
+            else {
+                // If the predicate has some hasValue restriction associated with it, then finding
+                // that the object belongs to the appropriate type implies a value.
+                final Map<Resource, Set<Value>> impliedValues = inferenceEngine.getHasValueByProperty(predURI);
+                if (impliedValues.size() > 0) {
+                    final Var rdfTypeVar = new Var(RDF.TYPE.stringValue(), RDF.TYPE);
+                    final Var typeVar = new Var("t-" + UUID.randomUUID());
+                    final Var hasValueVar = new Var(OWL.HASVALUE.stringValue(), OWL.HASVALUE);
+                    final TupleExpr typeSP = new DoNotExpandSP(subjVar, rdfTypeVar, typeVar);
+                    final FixedStatementPattern typeToValue = new FixedStatementPattern(typeVar, hasValueVar, objVar);
+                    final TupleExpr directValueSP = node.clone();
+                    for (Resource type : impliedValues.keySet()) {
+                        // { ?var rdf:type :type } implies { ?var :property :val } for certain (:type, :val) pairs
+                        for (Value impliedValue : impliedValues.get(type)) {
+                            typeToValue.statements.add(new NullableStatementImpl(type, OWL.HASVALUE, impliedValue));
+                        }
+                    }
+                    node.replaceWith(new InferUnion(new InferJoin(typeToValue, typeSP), directValueSP));
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2b73c306/sail/src/main/java/org/apache/rya/rdftriplestore/inference/InferenceEngine.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/org/apache/rya/rdftriplestore/inference/InferenceEngine.java b/sail/src/main/java/org/apache/rya/rdftriplestore/inference/InferenceEngine.java
index 3ffa211..a87dd8d 100644
--- a/sail/src/main/java/org/apache/rya/rdftriplestore/inference/InferenceEngine.java
+++ b/sail/src/main/java/org/apache/rya/rdftriplestore/inference/InferenceEngine.java
@@ -71,6 +71,8 @@ public class InferenceEngine {
     private Set<URI> symmetricPropertySet;
     private Map<URI, URI> inverseOfMap;
     private Set<URI> transitivePropertySet;
+    private Map<Resource, Map<URI, Value>> hasValueByType;
+    private Map<URI, Map<Resource, Value>> hasValueByProperty;
 
     private RyaDAO ryaDAO;
     private RdfCloudTripleStoreConfiguration conf;
@@ -360,12 +362,60 @@ public class InferenceEngine {
             	  }
             	}
             }
-            
+
+            refreshPropertyRestrictions();
+
         } catch (QueryEvaluationException e) {
             throw new InferenceEngineException(e);
         }
     }
-    
+
+    private void refreshPropertyRestrictions() throws QueryEvaluationException {
+        // Get a set of all property restrictions of any type
+        CloseableIteration<Statement, QueryEvaluationException> iter = RyaDAOHelper.query(ryaDAO, null, OWL.ONPROPERTY, null, conf);
+        Map<Resource, URI> restrictions = new HashMap<>();
+        try {
+            while (iter.hasNext()) {
+                Statement st = iter.next();
+                restrictions.put(st.getSubject(), (URI) st.getObject());
+            }
+        } finally {
+            if (iter != null) {
+                iter.close();
+            }
+        }
+        // Query for the hasValue restrictions and add them to the schema
+        refreshHasValueRestrictions(restrictions);
+    }
+
+    private void refreshHasValueRestrictions(Map<Resource, URI> restrictions) throws QueryEvaluationException {
+        hasValueByType = new HashMap<>();
+        hasValueByProperty = new HashMap<>();
+        CloseableIteration<Statement, QueryEvaluationException> iter = RyaDAOHelper.query(ryaDAO, null, OWL.HASVALUE, null, conf);
+        try {
+            while (iter.hasNext()) {
+                Statement st = iter.next();
+                Resource restrictionClass = st.getSubject();
+                if (restrictions.containsKey(restrictionClass)) {
+                    URI property = restrictions.get(restrictionClass);
+                    Value value = st.getObject();
+                    if (!hasValueByType.containsKey(restrictionClass)) {
+                        hasValueByType.put(restrictionClass, new HashMap<>());
+                    }
+                    if (!hasValueByProperty.containsKey(property)) {
+                        hasValueByProperty.put(property, new HashMap<>());
+                    }
+                    hasValueByType.get(restrictionClass).put(property, value);
+                    hasValueByProperty.get(property).put(restrictionClass, value);
+                }
+            }
+        } finally {
+            if (iter != null) {
+                iter.close();
+            }
+        }
+    }
+
     private static Vertex getVertex(Graph graph, Object id) {
         Iterator<Vertex> it = graph.vertices(id.toString());
         if (it.hasNext()) {
@@ -583,4 +633,69 @@ public class InferenceEngine {
     public void setSchedule(boolean schedule) {
         this.schedule = schedule;
     }
+
+    /**
+     * For a given type, return any properties and values such that owl:hasValue restrictions on
+     * those properties could imply this type. No matter how many restrictions are returned, each
+     * one is considered individually sufficient: if a resource has the property and the value, then
+     * it belongs to the provided type. Takes type hierarchy into account, so the value may imply a
+     * subtype which in turn implies the provided type.
+     * @param type The type (URI or bnode) to check against the known restrictions
+     * @return For each relevant property, a set of values such that whenever a resource has that
+     *      value for that property, it is implied to belong to the type.
+     */
+    public Map<URI, Set<Value>> getHasValueByType(Resource type) {
+        Map<URI, Set<Value>> implications = new HashMap<>();
+        if (hasValueByType != null) {
+            Set<Resource> types = new HashSet<>();
+            types.add(type);
+            if (type instanceof URI) {
+                types.addAll(findParents(subClassOfGraph, (URI) type));
+            }
+            for (Resource relevantType : types) {
+                if (hasValueByType.containsKey(relevantType)) {
+                    for (Map.Entry<URI, Value> propertyToValue : hasValueByType.get(relevantType).entrySet()) {
+                        if (!implications.containsKey(propertyToValue.getKey())) {
+                            implications.put(propertyToValue.getKey(), new HashSet<>());
+                        }
+                        implications.get(propertyToValue.getKey()).add(propertyToValue.getValue());
+                    }
+                }
+            }
+        }
+        return implications;
+    }
+
+    /**
+     * For a given property, return any types and values such that some owl:hasValue restriction
+     * states that members of the type are implied to have the associated specific value(s) for
+     * this property. Takes class hierarchy into account, which means one type may imply multiple
+     * values by way of supertypes with their own restrictions. Does not consider the property
+     * hierarchy, so only restrictions that directly reference the given property (using
+     * owl:onProperty) are relevant.
+     * @param property The property whose owl:hasValue restrictions to return
+     * @return A mapping from type (URIs or bnodes) to the set of any values that belonging to that
+     *      type implies.
+     */
+    public Map<Resource, Set<Value>> getHasValueByProperty(URI property) {
+        Map<Resource, Set<Value>> implications = new HashMap<>();
+        if (hasValueByProperty != null && hasValueByProperty.containsKey(property)) {
+            for (Map.Entry<Resource, Value> typeToValue : hasValueByProperty.get(property).entrySet()) {
+                Resource type = typeToValue.getKey();
+                if (!implications.containsKey(type)) {
+                    implications.put(type, new HashSet<>());
+                }
+                implications.get(type).add(typeToValue.getValue());
+                if (type instanceof URI) {
+                    for (URI subtype : findParents(subClassOfGraph, (URI) type)) {
+                        if (!implications.containsKey(subtype)) {
+                            implications.put(subtype, new HashSet<>());
+                        }
+                        implications.get(subtype).add(typeToValue.getValue());
+                    }
+                }
+            }
+        }
+        return implications;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2b73c306/sail/src/test/java/org/apache/rya/rdftriplestore/inference/HasValueVisitorTest.java
----------------------------------------------------------------------
diff --git a/sail/src/test/java/org/apache/rya/rdftriplestore/inference/HasValueVisitorTest.java b/sail/src/test/java/org/apache/rya/rdftriplestore/inference/HasValueVisitorTest.java
new file mode 100644
index 0000000..c5f2a90
--- /dev/null
+++ b/sail/src/test/java/org/apache/rya/rdftriplestore/inference/HasValueVisitorTest.java
@@ -0,0 +1,197 @@
+package org.apache.rya.rdftriplestore.inference;
+/*
+ * 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.
+ */
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.rya.accumulo.AccumuloRdfConfiguration;
+import org.apache.rya.rdftriplestore.inference.HasValueVisitor;
+import org.apache.rya.rdftriplestore.inference.InferenceEngine;
+import org.apache.rya.rdftriplestore.utils.FixedStatementPattern;
+import org.junit.Assert;
+import org.junit.Test;
+import org.openrdf.model.Resource;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.query.algebra.Join;
+import org.openrdf.query.algebra.Projection;
+import org.openrdf.query.algebra.ProjectionElem;
+import org.openrdf.query.algebra.ProjectionElemList;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.TupleExpr;
+import org.openrdf.query.algebra.Union;
+import org.openrdf.query.algebra.Var;
+
+public class HasValueVisitorTest {
+    private final AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
+    private final ValueFactory vf = new ValueFactoryImpl();
+
+    private final URI chordate = vf.createURI("urn:Chordate");
+    private final URI vertebrate = vf.createURI("urn:Vertebrate");
+    private final URI mammal = vf.createURI("urn:Mammal");
+    private final URI tunicate = vf.createURI("urn:Tunicate");
+    private final URI hasCharacteristic = vf.createURI("urn:anatomicalCharacteristic");
+    private final URI notochord = vf.createURI("urn:notochord");
+    private final URI skull = vf.createURI("urn:skull");
+    private final URI belongsTo = vf.createURI("urn:belongsToTaxon");
+    private final URI chordata = vf.createURI("urn:Chordata");
+
+    @Test
+    public void testRewriteTypePattern() throws Exception {
+        // Configure a mock instance engine with an ontology:
+        final InferenceEngine inferenceEngine = mock(InferenceEngine.class);
+        Map<URI, Set<Value>> vertebrateValues = new HashMap<>();
+        vertebrateValues.put(hasCharacteristic, new HashSet<>());
+        vertebrateValues.put(belongsTo, new HashSet<>());
+        vertebrateValues.get(hasCharacteristic).add(notochord);
+        vertebrateValues.get(hasCharacteristic).add(skull);
+        vertebrateValues.get(belongsTo).add(chordata);
+        when(inferenceEngine.getHasValueByType(vertebrate)).thenReturn(vertebrateValues);
+        // Query for a specific type and rewrite using the visitor:
+        final Projection query = new Projection(
+                new StatementPattern(new Var("s"), new Var("p", RDF.TYPE), new Var("o", vertebrate)),
+                new ProjectionElemList(new ProjectionElem("s", "subject")));
+        query.visit(new HasValueVisitor(conf, inferenceEngine));
+        // Expected structure: two nested unions whose members are (in some order) the original
+        // statement pattern and two joins, one for each unique property involved in a relevant
+        // restriction. Each join should be between a StatementPattern for the property and a
+        // FixedStatementPattern providing the value(s).
+        // Collect the arguments to the unions, ignoring nesting order:
+        Assert.assertTrue(query.getArg() instanceof Union);
+        final Union union1 = (Union) query.getArg();
+        final Set<TupleExpr> unionArgs = new HashSet<>();
+        if (union1.getLeftArg() instanceof Union) {
+            unionArgs.add(((Union) union1.getLeftArg()).getLeftArg());
+            unionArgs.add(((Union) union1.getLeftArg()).getRightArg());
+            unionArgs.add(union1.getRightArg());
+        }
+        else {
+            Assert.assertTrue(union1.getRightArg() instanceof Union);
+            unionArgs.add(union1.getLeftArg());
+            unionArgs.add(((Union) union1.getRightArg()).getLeftArg());
+            unionArgs.add(((Union) union1.getRightArg()).getRightArg());
+        }
+        // There should be one StatementPattern and two joins with structure Join(FSP, SP):
+        final StatementPattern directSP = new StatementPattern(new Var("s"), new Var("p", RDF.TYPE), new Var("o", vertebrate));
+        StatementPattern actualSP = null;
+        FixedStatementPattern hasCharacteristicFSP = null;
+        FixedStatementPattern belongsToFSP = null;
+        for (TupleExpr arg : unionArgs) {
+            if (arg instanceof StatementPattern) {
+                actualSP = (StatementPattern) arg;
+            }
+            else {
+                Assert.assertTrue(arg instanceof Join);
+                final Join join = (Join) arg;
+                Assert.assertTrue(join.getLeftArg() instanceof FixedStatementPattern);
+                Assert.assertTrue(join.getRightArg() instanceof StatementPattern);
+                final FixedStatementPattern fsp = (FixedStatementPattern) join.getLeftArg();
+                final StatementPattern sp = (StatementPattern) join.getRightArg();
+                // Should join FSP([unused], property, ?value) with SP(subject, property, ?value)
+                Assert.assertEquals(directSP.getSubjectVar(), sp.getSubjectVar());
+                Assert.assertEquals(fsp.getPredicateVar(), sp.getPredicateVar());
+                Assert.assertEquals(fsp.getObjectVar(), sp.getObjectVar());
+                if (hasCharacteristic.equals(fsp.getPredicateVar().getValue())) {
+                    hasCharacteristicFSP = fsp;
+                }
+                else if (belongsTo.equals(fsp.getPredicateVar().getValue())) {
+                    belongsToFSP = fsp;
+                }
+                else {
+                    Assert.fail("Unexpected property variable in rewritten query: " + fsp.getPredicateVar());
+                }
+            }
+        }
+        Assert.assertEquals(directSP, actualSP);
+        Assert.assertNotNull(hasCharacteristicFSP);
+        Assert.assertNotNull(belongsToFSP);
+        // Verify the expected FSPs for the appropriate properties:
+        Assert.assertEquals(2, hasCharacteristicFSP.statements.size());
+        Assert.assertTrue(hasCharacteristicFSP.statements.contains(vf.createStatement(vertebrate, hasCharacteristic, skull)));
+        Assert.assertTrue(hasCharacteristicFSP.statements.contains(vf.createStatement(vertebrate, hasCharacteristic, notochord)));
+        Assert.assertEquals(1, belongsToFSP.statements.size());
+        Assert.assertTrue(belongsToFSP.statements.contains(vf.createStatement(vertebrate, belongsTo, chordata)));
+    }
+
+    @Test
+    public void testRewriteValuePattern() throws Exception {
+        // Configure a mock inference engine with an ontology:
+        final InferenceEngine inferenceEngine = mock(InferenceEngine.class);
+        Map<Resource, Set<Value>> typeToCharacteristic = new HashMap<>();
+        Set<Value> chordateCharacteristics = new HashSet<>();
+        Set<Value> vertebrateCharacteristics = new HashSet<>();
+        chordateCharacteristics.add(notochord);
+        vertebrateCharacteristics.addAll(chordateCharacteristics);
+        vertebrateCharacteristics.add(skull);
+        typeToCharacteristic.put(chordate, chordateCharacteristics);
+        typeToCharacteristic.put(tunicate, chordateCharacteristics);
+        typeToCharacteristic.put(vertebrate, vertebrateCharacteristics);
+        typeToCharacteristic.put(mammal, vertebrateCharacteristics);
+        when(inferenceEngine.getHasValueByProperty(hasCharacteristic)).thenReturn(typeToCharacteristic);
+        // Query for a specific type and rewrite using the visitor:
+        final Projection query = new Projection(
+                new StatementPattern(new Var("s"), new Var("p", hasCharacteristic), new Var("o")),
+                new ProjectionElemList(new ProjectionElem("s", "subject"), new ProjectionElem("o", "characteristic")));
+        query.visit(new HasValueVisitor(conf, inferenceEngine));
+        // Expected structure: Union(Join(FSP, SP), [original SP])
+        Assert.assertTrue(query.getArg() instanceof Union);
+        final Union union = (Union) query.getArg();
+        final StatementPattern originalSP = new StatementPattern(new Var("s"), new Var("p", hasCharacteristic), new Var("o"));
+        Join join;
+        if (union.getLeftArg() instanceof Join) {
+            join = (Join) union.getLeftArg();
+            Assert.assertEquals(originalSP, union.getRightArg());
+        }
+        else {
+            Assert.assertTrue(union.getRightArg() instanceof Join);
+            join = (Join) union.getRightArg();
+            Assert.assertEquals(originalSP, union.getLeftArg());
+        }
+        Assert.assertTrue(join.getLeftArg() instanceof FixedStatementPattern);
+        Assert.assertTrue(join.getRightArg() instanceof StatementPattern);
+        final FixedStatementPattern fsp = (FixedStatementPattern) join.getLeftArg();
+        final StatementPattern sp = (StatementPattern) join.getRightArg();
+        // Verify join: FSP{ ?t _ ?originalObjectVar } JOIN { ?originalSubjectVar rdf:type ?t }
+        Assert.assertEquals(originalSP.getSubjectVar(), sp.getSubjectVar());
+        Assert.assertEquals(RDF.TYPE, sp.getPredicateVar().getValue());
+        Assert.assertEquals(fsp.getSubjectVar(), sp.getObjectVar());
+        Assert.assertEquals(originalSP.getObjectVar(), fsp.getObjectVar());
+        // Verify FSP: should provide (type, value) pairs
+        final Set<Statement> expectedStatements = new HashSet<>();
+        final URI fspPred = (URI) fsp.getPredicateVar().getValue();
+        expectedStatements.add(vf.createStatement(chordate, fspPred, notochord));
+        expectedStatements.add(vf.createStatement(tunicate, fspPred, notochord));
+        expectedStatements.add(vf.createStatement(vertebrate, fspPred, notochord));
+        expectedStatements.add(vf.createStatement(mammal, fspPred, notochord));
+        expectedStatements.add(vf.createStatement(vertebrate, fspPred, skull));
+        expectedStatements.add(vf.createStatement(mammal, fspPred, skull));
+        final Set<Statement> actualStatements = new HashSet<>(fsp.statements);
+        Assert.assertEquals(expectedStatements, actualStatements);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2b73c306/sail/src/test/java/org/apache/rya/rdftriplestore/inference/InferenceEngineTest.java
----------------------------------------------------------------------
diff --git a/sail/src/test/java/org/apache/rya/rdftriplestore/inference/InferenceEngineTest.java b/sail/src/test/java/org/apache/rya/rdftriplestore/inference/InferenceEngineTest.java
new file mode 100644
index 0000000..f60a1e2
--- /dev/null
+++ b/sail/src/test/java/org/apache/rya/rdftriplestore/inference/InferenceEngineTest.java
@@ -0,0 +1,167 @@
+package org.apache.rya.rdftriplestore.inference;
+/*
+ * 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.
+ */
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.rya.accumulo.AccumuloRdfConfiguration;
+import org.apache.rya.accumulo.AccumuloRyaDAO;
+import org.apache.rya.rdftriplestore.RdfCloudTripleStore;
+import org.junit.After;
+import org.junit.Assert;
+
+import org.junit.Test;
+import org.openrdf.model.Resource;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.QueryLanguage;
+import org.openrdf.repository.sail.SailRepository;
+import org.openrdf.repository.sail.SailRepositoryConnection;
+
+import junit.framework.TestCase;
+
+public class InferenceEngineTest extends TestCase {
+    private Connector connector;
+    private AccumuloRyaDAO dao;
+    private ValueFactory vf = new ValueFactoryImpl();
+    private AccumuloRdfConfiguration conf;
+    private RdfCloudTripleStore store;
+    private InferenceEngine inferenceEngine;
+    private SailRepository repository;
+    private SailRepositoryConnection conn;
+
+    @Override
+    public void setUp() throws Exception {
+        super.setUp();
+        dao = new AccumuloRyaDAO();
+        connector = new MockInstance().getConnector("", new PasswordToken(""));
+        dao.setConnector(connector);
+        conf = new AccumuloRdfConfiguration();
+        dao.setConf(conf);
+        dao.init();
+        store = new RdfCloudTripleStore();
+        store.setConf(conf);
+        store.setRyaDAO(dao);
+        inferenceEngine = new InferenceEngine();
+        inferenceEngine.setRyaDAO(dao);
+        store.setInferenceEngine(inferenceEngine);
+        inferenceEngine.refreshGraph();
+        store.initialize();
+        repository = new SailRepository(store);
+        conn = repository.getConnection();
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        conn.close();
+        repository.shutDown();
+        store.shutDown();
+        dao.purge(conf);
+        dao.destroy();
+    }
+
+    @Test
+    public void testHasValueGivenProperty() throws Exception {
+        String insert = "INSERT DATA { GRAPH <http://updated/test> {\n"
+                + "  <urn:Biped> owl:onProperty <urn:walksUsingLegs>  . \n"
+                + "  <urn:Biped> owl:hasValue \"2\" . \n"
+                + "  <urn:Mammal> owl:onProperty <urn:taxon> . \n"
+                + "  <urn:Mammal> owl:hasValue <urn:Mammalia> . \n"
+                + "  <urn:Vertebrate> owl:onProperty <urn:taxon> . \n"
+                + "  <urn:Vertebrate> owl:hasValue <urn:Vertebrata> . \n"
+                + "  <urn:Tunicate> owl:onProperty <urn:taxon> . \n"
+                + "  <urn:Tunicate> owl:hasValue <urn:Tunicata> . \n"
+                + "  <urn:Mammal> rdfs:subClassOf <urn:Vertebrate> . \n"
+                + "  <urn:Vertebrate> rdfs:subClassOf <urn:Animal> . \n"
+                + "  <urn:Tunicate> rdfs:subClassOf <urn:Animal> . \n"
+                + "  <urn:Biped> rdfs:subClassOf <urn:Animal> . \n"
+                + "}}";
+        conn.prepareUpdate(QueryLanguage.SPARQL, insert).execute();
+        inferenceEngine.refreshGraph();
+        final Map<Resource, Set<Value>> typeToValueImplications = new HashMap<>();
+        final Set<Value> vertebrateTaxa = new HashSet<>();
+        final Set<Value> tunicateTaxa = new HashSet<>();
+        vertebrateTaxa.add(vf.createURI("urn:Vertebrata"));
+        tunicateTaxa.add(vf.createURI("urn:Tunicata"));
+        final Set<Value> mammalTaxa = new HashSet<>(vertebrateTaxa);
+        mammalTaxa.add(vf.createURI("urn:Mammalia"));
+        typeToValueImplications.put(vf.createURI("urn:Vertebrate"), vertebrateTaxa);
+        typeToValueImplications.put(vf.createURI("urn:Tunicate"), tunicateTaxa);
+        typeToValueImplications.put(vf.createURI("urn:Mammal"), mammalTaxa);
+        Assert.assertEquals(typeToValueImplications, inferenceEngine.getHasValueByProperty(vf.createURI("urn:taxon")));
+    }
+
+    public void testHasValueGivenType() throws Exception {
+        String insert = "INSERT DATA { GRAPH <http://updated/test> {\n"
+                + "  <urn:Biped> owl:onProperty <urn:walksUsingLegs>  . \n"
+                + "  <urn:Biped> owl:hasValue \"2\" . \n"
+                + "  <urn:Mammal> owl:onProperty <urn:taxon> . \n"
+                + "  <urn:Mammal> owl:hasValue <urn:Mammalia> . \n"
+                + "  <urn:Vertebrate> owl:onProperty <urn:taxon> . \n"
+                + "  <urn:Vertebrate> owl:hasValue <urn:Vertebrata> . \n"
+                + "  <urn:Tunicate> owl:onProperty <urn:taxon> . \n"
+                + "  <urn:Tunicate> owl:hasValue <urn:Tunicata> . \n"
+                + "  <urn:Plant> owl:onProperty <urn:taxon>  . \n"
+                + "  <urn:Plant> owl:hasValue <urn:Plantae> . \n"
+                + "  <urn:Mammal> rdfs:subClassOf <urn:Vertebrate> . \n"
+                + "  <urn:Vertebrate> rdfs:subClassOf <urn:Animal> . \n"
+                + "  <urn:Tunicate> rdfs:subClassOf <urn:Animal> . \n"
+                + "  <urn:Biped> rdfs:subClassOf <urn:Animal> . \n"
+                + "}}";
+        conn.prepareUpdate(QueryLanguage.SPARQL, insert).execute();
+        inferenceEngine.refreshGraph();
+        final URI legs = vf.createURI("urn:walksUsingLegs");
+        final URI taxon = vf.createURI("urn:taxon");
+        // Verify direct restrictions:
+        final Map<URI, Set<Value>> valuesImplyingBiped = new HashMap<>();
+        valuesImplyingBiped.put(legs, new HashSet<>());
+        valuesImplyingBiped.get(legs).add(vf.createLiteral("2"));
+        Assert.assertEquals(valuesImplyingBiped, inferenceEngine.getHasValueByType(vf.createURI("urn:Biped")));
+        final Map<URI, Set<Value>> valuesImplyingMammal = new HashMap<>();
+        valuesImplyingMammal.put(taxon, new HashSet<>());
+        valuesImplyingMammal.get(taxon).add(vf.createURI("urn:Mammalia"));
+        Assert.assertEquals(valuesImplyingMammal, inferenceEngine.getHasValueByType(vf.createURI("urn:Mammal")));
+        final Map<URI, Set<Value>> valuesImplyingTunicate = new HashMap<>();
+        valuesImplyingTunicate.put(taxon, new HashSet<>());
+        valuesImplyingTunicate.get(taxon).add(vf.createURI("urn:Tunicata"));
+        Assert.assertEquals(valuesImplyingTunicate, inferenceEngine.getHasValueByType(vf.createURI("urn:Tunicate")));
+        final Map<URI, Set<Value>> valuesImplyingPlant = new HashMap<>();
+        valuesImplyingPlant.put(taxon, new HashSet<>());
+        valuesImplyingPlant.get(taxon).add(vf.createURI("urn:Plantae"));
+        Assert.assertEquals(valuesImplyingPlant, inferenceEngine.getHasValueByType(vf.createURI("urn:Plant")));
+        // Verify indirect restrictions given a supertype, including multiple properties where relevant:
+        final Map<URI, Set<Value>> valuesImplyingVertebrate = new HashMap<>();
+        valuesImplyingVertebrate.put(taxon, new HashSet<>(valuesImplyingMammal.get(taxon)));
+        valuesImplyingVertebrate.get(taxon).add(vf.createURI("urn:Vertebrata"));
+        Assert.assertEquals(valuesImplyingVertebrate, inferenceEngine.getHasValueByType(vf.createURI("urn:Vertebrate")));
+        final Map<URI, Set<Value>> valuesImplyingAnimal = new HashMap<>();
+        valuesImplyingAnimal.put(legs, valuesImplyingBiped.get(legs));
+        valuesImplyingAnimal.put(taxon, new HashSet<>(valuesImplyingVertebrate.get(taxon)));
+        valuesImplyingAnimal.get(taxon).addAll(valuesImplyingTunicate.get(taxon));
+        Assert.assertEquals(valuesImplyingAnimal, inferenceEngine.getHasValueByType(vf.createURI("urn:Animal")));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2b73c306/sail/src/test/java/org/apache/rya/rdftriplestore/inference/InferenceIT.java
----------------------------------------------------------------------
diff --git a/sail/src/test/java/org/apache/rya/rdftriplestore/inference/InferenceIT.java b/sail/src/test/java/org/apache/rya/rdftriplestore/inference/InferenceIT.java
new file mode 100644
index 0000000..c950a3e
--- /dev/null
+++ b/sail/src/test/java/org/apache/rya/rdftriplestore/inference/InferenceIT.java
@@ -0,0 +1,181 @@
+package org.apache.rya.rdftriplestore.inference;
+/*
+ * 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.
+ */
+
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.rya.accumulo.AccumuloRdfConfiguration;
+import org.apache.rya.accumulo.AccumuloRyaDAO;
+import org.apache.rya.rdftriplestore.RdfCloudTripleStore;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.QueryLanguage;
+import org.openrdf.query.QueryResultHandlerException;
+import org.openrdf.query.TupleQueryResultHandler;
+import org.openrdf.query.TupleQueryResultHandlerException;
+import org.openrdf.query.impl.ListBindingSet;
+import org.openrdf.repository.sail.SailRepository;
+import org.openrdf.repository.sail.SailRepositoryConnection;
+
+import junit.framework.TestCase;
+
+public class InferenceIT extends TestCase {
+    private Connector connector;
+    private AccumuloRyaDAO dao;
+    private ValueFactory vf = new ValueFactoryImpl();
+    private AccumuloRdfConfiguration conf;
+    private RdfCloudTripleStore store;
+    private InferenceEngine inferenceEngine;
+    private SailRepository repository;
+    private SailRepositoryConnection conn;
+    private TupleQueryResultHandler resultHandler;
+    private List<BindingSet> solutions;
+
+    @Override
+    public void setUp() throws Exception {
+        super.setUp();
+        dao = new AccumuloRyaDAO();
+        connector = new MockInstance().getConnector("", new PasswordToken(""));
+        dao.setConnector(connector);
+        conf = new AccumuloRdfConfiguration();
+        conf.setInfer(true);
+        dao.setConf(conf);
+        dao.init();
+        store = new RdfCloudTripleStore();
+        store.setConf(conf);
+        store.setRyaDAO(dao);
+        inferenceEngine = new InferenceEngine();
+        inferenceEngine.setRyaDAO(dao);
+        store.setInferenceEngine(inferenceEngine);
+        inferenceEngine.refreshGraph();
+        store.initialize();
+        repository = new SailRepository(store);
+        conn = repository.getConnection();
+        solutions = new LinkedList<>();
+        resultHandler = new TupleQueryResultHandler() {
+            @Override
+            public void endQueryResult() throws TupleQueryResultHandlerException { }
+            @Override
+            public void handleBoolean(boolean arg0) throws QueryResultHandlerException { }
+            @Override
+            public void handleLinks(List<String> arg0) throws QueryResultHandlerException { }
+            @Override
+            public void handleSolution(BindingSet arg0) throws TupleQueryResultHandlerException {
+                solutions.add(arg0);
+            }
+            @Override
+            public void startQueryResult(List<String> arg0) throws TupleQueryResultHandlerException { }
+        };
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        conn.close();
+        repository.shutDown();
+        store.shutDown();
+        dao.purge(conf);
+        dao.destroy();
+    }
+
+    @Test
+    public void testHasValueTypeQuery() throws Exception {
+        final String ontology = "INSERT DATA { GRAPH <http://updated/test> {\n"
+                + "  <urn:Biped> owl:onProperty <urn:walksOnLegs>  ; owl:hasValue \"2\"^^<xsd:integer> . \n"
+                + "  <urn:Quadruped> owl:onProperty <urn:walksOnLegs>  ; owl:hasValue \"4\"^^<xsd:int> . \n"
+                + "  <urn:Person> owl:onProperty <urn:taxon>  ; owl:hasValue <urn:Hominidae> . \n"
+                + "  <urn:Person> rdfs:subClassOf <urn:Biped> . \n"
+                + "  <urn:Horse> rdfs:subClassOf <urn:Quadruped> . \n"
+                + "  <urn:Biped> rdfs:subClassOf <urn:Animal> . \n"
+                + "}}";
+        final String instances = "INSERT DATA { GRAPH <http://updated/test> {\n"
+                + "  <urn:Alice> a <urn:Person>  . \n"
+                + "  <urn:Bob> <urn:walksOnLegs> \"2\"^^<xsd:integer> . \n"
+                + "  <urn:Carol> <urn:walksOnLegs> \"2\" . \n"
+                + "  <urn:Dan> <urn:taxon> <urn:Hominidae> . \n"
+                + "  <urn:Laika> a <urn:Quadruped> . \n"
+                + "  <urn:Lucy> a <urn:Biped> . \n"
+                + "  <urn:Hank> <urn:walksOnLegs> \"4\"^^<xsd:int> . \n"
+                + "  <urn:Hans> a <urn:Horse> . \n"
+                + "}}";
+        final String query = "SELECT ?x { GRAPH <http://updated/test> { ?x a <urn:Biped> } } \n";
+        conn.prepareUpdate(QueryLanguage.SPARQL, ontology).execute();
+        inferenceEngine.refreshGraph();
+        conn.prepareUpdate(QueryLanguage.SPARQL, instances).execute();
+        conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate(resultHandler);
+        Set<Value> expected = new HashSet<>();
+        expected.add(vf.createURI("urn:Alice"));
+        expected.add(vf.createURI("urn:Bob"));
+        expected.add(vf.createURI("urn:Carol"));
+        expected.add(vf.createURI("urn:Dan"));
+        expected.add(vf.createURI("urn:Lucy"));
+        Set<Value> returned = new HashSet<>();
+        for (BindingSet bs : solutions) {
+            returned.add(bs.getBinding("x").getValue());
+        }
+        Assert.assertEquals(expected, returned);
+        Assert.assertEquals(5, solutions.size());
+    }
+
+    @Test
+    public void testHasValueValueQuery() throws Exception {
+        final String ontology = "INSERT DATA { GRAPH <http://updated/test> {\n"
+                + "  <urn:Hominid> owl:onProperty <urn:taxon> ; owl:hasValue <urn:Hominidae> . \n"
+                + "  <urn:Carnivoran> owl:onProperty <urn:taxon>  ; owl:hasValue <urn:Carnivora> . \n"
+                + "  <urn:Mammal> owl:onProperty <urn:taxon>  ; owl:hasValue <urn:Mammalia> . \n"
+                + "  <urn:Tunicate> owl:onProperty <urn:taxon>  ; owl:hasValue <urn:Tunicata> . \n"
+                + "  <urn:Person> rdfs:subClassOf <urn:Hominid> . \n"
+                + "  <urn:Hominid> rdfs:subClassOf <urn:Mammal> . \n"
+                + "  <urn:Cat> rdfs:subClassOf <urn:Carnivoran> . \n"
+                + "  <urn:Carnivoran> rdfs:subClassOf <urn:Mammal> . \n"
+                + "}}";
+        final String instances = "INSERT DATA { GRAPH <http://updated/test> {\n"
+                + "  <urn:Alice> a <urn:Person> . \n"
+                + "  <urn:Bigfoot> a <urn:Mammal> . \n"
+                + "  <urn:Carol> <urn:taxon> <urn:Hominidae> . \n"
+                + "  <urn:Hank> a <urn:Cat> . \n"
+                + "}}";
+        final String query = "SELECT ?individual ?taxon { GRAPH <http://updated/test> { ?individual <urn:taxon> ?taxon } } \n";
+        conn.prepareUpdate(QueryLanguage.SPARQL, ontology).execute();
+        conn.prepareUpdate(QueryLanguage.SPARQL, instances).execute();
+        inferenceEngine.refreshGraph();
+        conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate(resultHandler);
+        Set<BindingSet> expected = new HashSet<BindingSet>();
+        List<String> varNames = new LinkedList<>();
+        varNames.add("individual");
+        varNames.add("taxon");
+        expected.add(new ListBindingSet(varNames, vf.createURI("urn:Alice"), vf.createURI("urn:Hominidae")));
+        expected.add(new ListBindingSet(varNames, vf.createURI("urn:Alice"), vf.createURI("urn:Mammalia")));
+        expected.add(new ListBindingSet(varNames, vf.createURI("urn:Bigfoot"), vf.createURI("urn:Mammalia")));
+        expected.add(new ListBindingSet(varNames, vf.createURI("urn:Carol"), vf.createURI("urn:Hominidae")));
+        expected.add(new ListBindingSet(varNames, vf.createURI("urn:Hank"), vf.createURI("urn:Carnivora")));
+        expected.add(new ListBindingSet(varNames, vf.createURI("urn:Hank"), vf.createURI("urn:Mammalia")));
+        Assert.assertEquals(expected, new HashSet<>(solutions));
+    }
+}