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/08/25 20:38:00 UTC

incubator-rya git commit: RYA-296 add HasSelf inference rule. Closes #209.

Repository: incubator-rya
Updated Branches:
  refs/heads/master 051472660 -> d47190b1a


RYA-296 add HasSelf inference rule.  Closes #209.

Added has self along with tests


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

Branch: refs/heads/master
Commit: d47190b1ab429f1a200ac0d9a0ae07b451db5027
Parents: 0514726
Author: isper3at <sm...@gmail.com>
Authored: Tue Aug 8 12:03:57 2017 -0400
Committer: Caleb Meier <ca...@parsons.com>
Committed: Fri Aug 25 13:36:56 2017 -0700

----------------------------------------------------------------------
 .../api/RdfCloudTripleStoreConfiguration.java   |  10 ++
 .../RdfCloudTripleStoreConnection.java          |   2 +
 .../inference/HasSelfVisitor.java               | 106 +++++++++++++++
 .../inference/InferenceEngine.java              | 126 ++++++++++++++++--
 .../inference/HasSelfVisitorTest.java           | 133 +++++++++++++++++++
 .../inference/InferenceEngineTest.java          |  15 +++
 .../rdftriplestore/inference/InferenceIT.java   |  49 +++++++
 7 files changed, 430 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/d47190b1/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfiguration.java b/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfiguration.java
index 6bebb86..ed120ef 100644
--- a/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfiguration.java
+++ b/common/rya.api/src/main/java/org/apache/rya/api/RdfCloudTripleStoreConfiguration.java
@@ -80,6 +80,7 @@ public abstract class RdfCloudTripleStoreConfiguration extends Configuration {
     public static final String INFER_INCLUDE_SUBPROPOF = "infer.include.subpropof";
     public static final String INFER_INCLUDE_SYMMPROP = "infer.include.symmprop";
     public static final String INFER_INCLUDE_TRANSITIVEPROP = "infer.include.transprop";
+    public static final String INFER_INCLUDE_HAS_SELF = "infer.include.hasself";
 
     public static final String RDF_DAO_CLASS = "class.rdf.dao";
     public static final String RDF_EVAL_STATS_DAO_CLASS = "class.rdf.evalstats";
@@ -416,6 +417,15 @@ public abstract class RdfCloudTripleStoreConfiguration extends Configuration {
         setBoolean(INFER_INCLUDE_SUBPROPOF, val);
     }
 
+    public Boolean hasSelf() {
+        return getBoolean(INFER_INCLUDE_HAS_SELF, true);
+    }
+
+    public void setHasSelf(final Boolean val) {
+        Preconditions.checkNotNull(val);
+        setBoolean(INFER_INCLUDE_HAS_SELF, val);
+    }
+
     public Boolean isInferSymmetricProperty() {
         return getBoolean(INFER_INCLUDE_SYMMPROP, true);
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/d47190b1/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 40a69b0..212fb2a 100644
--- a/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
+++ b/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
@@ -53,6 +53,7 @@ import org.apache.rya.rdftriplestore.evaluation.RdfCloudTripleStoreSelectivityEv
 import org.apache.rya.rdftriplestore.evaluation.SeparateFilterJoinsVisitor;
 import org.apache.rya.rdftriplestore.inference.AllValuesFromVisitor;
 import org.apache.rya.rdftriplestore.inference.DomainRangeVisitor;
+import org.apache.rya.rdftriplestore.inference.HasSelfVisitor;
 import org.apache.rya.rdftriplestore.inference.HasValueVisitor;
 import org.apache.rya.rdftriplestore.inference.InferenceEngine;
 import org.apache.rya.rdftriplestore.inference.IntersectionOfVisitor;
@@ -366,6 +367,7 @@ public class RdfCloudTripleStoreConnection extends SailConnectionBase {
                     tupleExpr.visit(new SubClassOfVisitor(queryConf, inferenceEngine));
                     tupleExpr.visit(new SameAsVisitor(queryConf, inferenceEngine));
                     tupleExpr.visit(new OneOfVisitor(queryConf, inferenceEngine));
+                    tupleExpr.visit(new HasSelfVisitor(queryConf, inferenceEngine));
                 } catch (final Exception e) {
                     logger.error("Error encountered while visiting query node.", e);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/d47190b1/sail/src/main/java/org/apache/rya/rdftriplestore/inference/HasSelfVisitor.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/org/apache/rya/rdftriplestore/inference/HasSelfVisitor.java b/sail/src/main/java/org/apache/rya/rdftriplestore/inference/HasSelfVisitor.java
new file mode 100644
index 0000000..6776387
--- /dev/null
+++ b/sail/src/main/java/org/apache/rya/rdftriplestore/inference/HasSelfVisitor.java
@@ -0,0 +1,106 @@
+/*
+ * 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.rya.rdftriplestore.inference;
+
+import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
+import org.openrdf.model.Resource;
+import org.openrdf.model.URI;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.query.algebra.Extension;
+import org.openrdf.query.algebra.ExtensionElem;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.Var;
+
+/**
+ * Expands the query tree to account for any relevant has-self 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-self expression
+ * in the ontology. When applicable, replaces the node with one or more
+ * {@link InferUnion}s, one of whose leaves is the original StatementPattern.
+ *
+ * A has-self restriction defines the set of resources that are connected to
+ * themselves by a property. If the ontology states that a type is a resource
+ * that has a self referencing property, then the inference engine should:
+ * <li>1. Rewrite queries of the from ?x rdf:type :T to find all resources
+ * matching ?x :P ?x (as well as anything explicitly stated to be of type :T)
+ * </li>
+ * <li>2. Rewrite queries of the from :CONST :P ?o or ?subj :P :CONST to match
+ * :CONST if :CONST is known to have the type :T
+ * </li>
+ */
+public class HasSelfVisitor extends AbstractInferVisitor {
+    private static final Var TYPE_VAR = new Var(RDF.TYPE.stringValue(), RDF.TYPE);
+
+    /**
+     * Creates a new {@link HasSelfVisitor}, which is enabled by default.
+     * @param conf The {@link RdfCloudTripleStoreConfiguration}.
+     * @param inferenceEngine The InferenceEngine containing the relevant ontology.
+     */
+    public HasSelfVisitor(final RdfCloudTripleStoreConfiguration conf, final InferenceEngine inferenceEngine) {
+        super(conf, inferenceEngine);
+        include = conf.hasSelf();
+    }
+
+    @Override
+    protected void meetSP(final StatementPattern node) throws Exception {
+        final URI pred = (URI) node.getPredicateVar().getValue();
+        final Var obj = node.getObjectVar();
+        //if originalSP like (?s rdf:type :C1):  require that C1 is defined, i.e. not a variable
+        // node <- originalSP
+        final StatementPattern clone = node.clone();
+        if (RDF.TYPE.equals(pred) && obj.isConstant()) {
+            //for property in getHasSelfImplyingType(C1):
+            if (obj.getValue() instanceof URI) {
+                for (final URI property : inferenceEngine.getHasSelfImplyingType((URI) obj.getValue())) {
+                    //node <- InferUnion(node, StatementPattern(?s, property, ?s)).
+                    final InferUnion union = new InferUnion(clone,
+                            new StatementPattern(clone.getSubjectVar(),
+                                                 new Var(property.stringValue(), property),
+                                                 clone.getSubjectVar()));
+                    //originalSP.replaceWith(node)
+                    node.replaceWith(union);
+                }
+            }
+        //else if originalSP like (s :p o):  where p is not a variable and at least one of s and o are variables
+        } else if (node.getPredicateVar().isConstant()
+               && (!node.getSubjectVar().isConstant() ||
+                   !node.getObjectVar().isConstant())) {
+            //for type in getHasSelfImplyingProperty(p):
+            for (final Resource type : inferenceEngine.getHasSelfImplyingProperty(pred)) {
+                final Extension extension;
+                if(obj.isConstant()) { // subject is the variable
+                    //Extension(StatementPattern(o, rdf:type, type), ExtensionElem(o, "s"))
+                    extension = new Extension(
+                            new StatementPattern(obj, TYPE_VAR, new Var(type.stringValue(), type)),
+                            new ExtensionElem(obj, node.getSubjectVar().getName()));
+                } else { //o is a variable and s may either be defined or a variable
+                    //Extension(StatementPattern(s, rdf:type, type), ExtensionElem(s, "o"))
+                    extension = new Extension(
+                            new StatementPattern(node.getSubjectVar(), TYPE_VAR, new Var(type.stringValue(), type)),
+                            new ExtensionElem(node.getSubjectVar(), obj.getName()));
+                }
+                // node <- InferUnion(node, newNode)
+                final InferUnion union = new InferUnion(extension, clone);
+                node.replaceWith(union);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/d47190b1/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 a0a5cfc..622ba33 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
@@ -73,6 +73,8 @@ import info.aduna.iteration.CloseableIteration;
  */
 public class InferenceEngine {
     private static final Logger log = Logger.getLogger(InferenceEngine.class);
+    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
+    private static final URI HAS_SELF = VF.createURI(OWL.NAMESPACE, "hasSelf");
 
     private Graph subClassOfGraph;
     private Graph subPropertyOfGraph;
@@ -87,6 +89,9 @@ public class InferenceEngine {
     private Map<Resource, Map<Resource, URI>> allValuesFromByValueType;
     private final ConcurrentHashMap<Resource, List<Set<Resource>>> intersections = new ConcurrentHashMap<>();
     private final ConcurrentHashMap<Resource, Set<Resource>> enumerations = new ConcurrentHashMap<>();
+    // hasSelf maps.
+    private Map<URI, Set<Resource>> hasSelfByProperty;
+    private Map<Resource, Set<URI>> hasSelfByType;
 
     private RyaDAO<?> ryaDAO;
     private RdfCloudTripleStoreConfiguration conf;
@@ -141,7 +146,6 @@ public class InferenceEngine {
     }
 
     public void refreshGraph() throws InferenceEngineException {
-        final ValueFactory vf = ValueFactoryImpl.getInstance();
         try {
             CloseableIteration<Statement, QueryEvaluationException> iter;
             //get all subclassof
@@ -170,7 +174,7 @@ public class InferenceEngine {
                                 final Statement firstStatement = listIter.next();
                                 if (firstStatement.getObject() instanceof Resource) {
                                     final Resource subclass = (Resource) firstStatement.getObject();
-                                    final Statement subclassStatement = vf.createStatement(subclass, RDFS.SUBCLASSOF, unionType);
+                                    final Statement subclassStatement = VF.createStatement(subclass, RDFS.SUBCLASSOF, unionType);
                                     addStatementEdge(graph, RDFS.SUBCLASSOF.stringValue(), subclassStatement);
                                 }
                             }
@@ -251,7 +255,7 @@ public class InferenceEngine {
             inverseOfMap = invProp;
 
             iter = RyaDAOHelper.query(ryaDAO, null,
-                    vf.createURI("http://www.w3.org/2002/07/owl#propertyChainAxiom"),
+                    VF.createURI("http://www.w3.org/2002/07/owl#propertyChainAxiom"),
                     null, conf);
             final Map<URI,URI> propertyChainPropertiesToBNodes = new HashMap<>();
             propertyChainPropertyToChain = new HashMap<>();
@@ -269,7 +273,7 @@ public class InferenceEngine {
             for (final URI propertyChainProperty : propertyChainPropertiesToBNodes.keySet()){
                 final URI bNode = propertyChainPropertiesToBNodes.get(propertyChainProperty);
                 // query for the list of indexed properties
-                iter = RyaDAOHelper.query(ryaDAO, bNode, vf.createURI("http://www.w3.org/2000/10/swap/list#index"),
+                iter = RyaDAOHelper.query(ryaDAO, bNode, VF.createURI("http://www.w3.org/2000/10/swap/list#index"),
                         null, conf);
                 final TreeMap<Integer, URI> orderedProperties = new TreeMap<>();
                 // TODO refactor this.  Wish I could execute sparql
@@ -278,7 +282,7 @@ public class InferenceEngine {
                         final Statement st = iter.next();
                         final String indexedElement = st.getObject().stringValue();
                         log.info(indexedElement);
-                        CloseableIteration<Statement, QueryEvaluationException>  iter2 = RyaDAOHelper.query(ryaDAO, vf.createURI(st.getObject().stringValue()), RDF.FIRST,
+                        CloseableIteration<Statement, QueryEvaluationException>  iter2 = RyaDAOHelper.query(ryaDAO, VF.createURI(st.getObject().stringValue()), RDF.FIRST,
                                 null, conf);
                         String integerValue = "";
                         Value anonPropNode = null;
@@ -291,7 +295,7 @@ public class InferenceEngine {
                             }
                             iter2.close();
                         }
-                        iter2 = RyaDAOHelper.query(ryaDAO, vf.createURI(st.getObject().stringValue()), RDF.REST,
+                        iter2 = RyaDAOHelper.query(ryaDAO, VF.createURI(st.getObject().stringValue()), RDF.REST,
                                 null, conf);
                         if (iter2 != null){
                             while (iter2.hasNext()){
@@ -301,7 +305,7 @@ public class InferenceEngine {
                             }
                             iter2.close();
                             if (anonPropNode != null){
-                                iter2 = RyaDAOHelper.query(ryaDAO, vf.createURI(anonPropNode.stringValue()), RDF.FIRST,
+                                iter2 = RyaDAOHelper.query(ryaDAO, VF.createURI(anonPropNode.stringValue()), RDF.FIRST,
                                         null, conf);
                                 while (iter2.hasNext()){
                                     final Statement iter2Statement = iter2.next();
@@ -314,7 +318,7 @@ public class InferenceEngine {
                         if (!integerValue.isEmpty() && propURI!=null) {
                             try {
                                 final int indexValue = Integer.parseInt(integerValue);
-                                final URI chainPropURI = vf.createURI(propURI.stringValue());
+                                final URI chainPropURI = VF.createURI(propURI.stringValue());
                                 orderedProperties.put(indexValue, chainPropURI);
                             }
                             catch (final Exception ex){
@@ -350,7 +354,7 @@ public class InferenceEngine {
                         Value currentPropValue = iter2Statement.getObject();
                         while ((currentPropValue != null) && (!currentPropValue.stringValue().equalsIgnoreCase(RDF.NIL.stringValue()))){
                             if (currentPropValue instanceof URI){
-                                iter2 = RyaDAOHelper.query(ryaDAO, vf.createURI(currentPropValue.stringValue()), RDF.FIRST,
+                                iter2 = RyaDAOHelper.query(ryaDAO, VF.createURI(currentPropValue.stringValue()), RDF.FIRST,
                                         null, conf);
                                 if (iter2.hasNext()){
                                     iter2Statement = iter2.next();
@@ -360,7 +364,7 @@ public class InferenceEngine {
                                 }
                                 // otherwise see if there is an inverse declaration
                                 else {
-                                    iter2 = RyaDAOHelper.query(ryaDAO, vf.createURI(currentPropValue.stringValue()), OWL.INVERSEOF,
+                                    iter2 = RyaDAOHelper.query(ryaDAO, VF.createURI(currentPropValue.stringValue()), OWL.INVERSEOF,
                                             null, conf);
                                     if (iter2.hasNext()){
                                         iter2Statement = iter2.next();
@@ -609,6 +613,7 @@ public class InferenceEngine {
         refreshHasValueRestrictions(restrictions);
         refreshSomeValuesFromRestrictions(restrictions);
         refreshAllValuesFromRestrictions(restrictions);
+        refreshHasSelfRestrictions(restrictions);
     }
 
     private void refreshHasValueRestrictions(final Map<Resource, URI> restrictions) throws QueryEvaluationException {
@@ -691,6 +696,39 @@ public class InferenceEngine {
         });
     }
 
+    private void refreshHasSelfRestrictions(final Map<Resource, URI> restrictions) throws QueryEvaluationException {
+        hasSelfByType = new HashMap<>();
+        hasSelfByProperty = new HashMap<>();
+
+        for(final Resource type : restrictions.keySet()) {
+            final URI property = restrictions.get(type);
+            final CloseableIteration<Statement, QueryEvaluationException> iter = RyaDAOHelper.query(ryaDAO, type, HAS_SELF, null, conf);
+            try {
+                if (iter.hasNext()) {
+                    Set<URI> typeSet = hasSelfByType.get(type);
+                    Set<Resource> propSet = hasSelfByProperty.get(property);
+
+                    if (typeSet == null) {
+                        typeSet = new HashSet<>();
+                    }
+                    if (propSet == null) {
+                        propSet = new HashSet<>();
+                    }
+
+                    typeSet.add(property);
+                    propSet.add(type);
+
+                    hasSelfByType.put(type, typeSet);
+                    hasSelfByProperty.put(property, propSet);
+                }
+            } finally {
+                if (iter != null) {
+                    iter.close();
+                }
+            }
+        }
+    }
+
     private void refreshIntersectionOf() throws QueryEvaluationException {
         final Map<Resource, List<Set<Resource>>> intersectionsProp = new HashMap<>();
 
@@ -829,6 +867,72 @@ public class InferenceEngine {
     }
 
     /**
+     * For a given type, return any properties such that some owl:hasSelf
+     * restrictions implies that properties of this type have the value of
+     * themselves for this type.
+     *
+     * This takes into account type hierarchy, where children of a type that
+     * have this property are also assumed to have the property.
+     * 
+     * @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 Set<URI> getHasSelfImplyingType(final Resource type){
+        // return properties that imply this type if reflexive
+        final Set<URI> properties = new HashSet<>();
+        Set<URI> tempProperties = hasSelfByType.get(type);
+
+        if (tempProperties != null) {
+            properties.addAll(tempProperties);
+        }
+        //findParent gets all subclasses, add self.
+        if (type instanceof URI) {
+            for (final URI subtype : findParents(subClassOfGraph, (URI) type)) {
+                tempProperties = hasSelfByType.get(subtype);
+                if (tempProperties != null) {
+                    properties.addAll(tempProperties);
+                }
+            }
+        }
+
+        // make map hasSelfByType[]
+        return properties;
+    }
+
+    /**
+     * For a given property, return any types such that some owl:hasSelf restriction implies that members
+     * of the type have the value of themselves for this property.
+     *
+     * This takes into account type hierarchy, where children of a type that have
+     * this property are also assumed to have the property.
+     * @param property The property whose owl:hasSelf restrictions to return
+     * @return A set of types that possess the implied property.
+     */
+    public Set<Resource> getHasSelfImplyingProperty(final URI property) {
+        // return types that imply this type if reflexive
+        final Set<Resource> types = new HashSet<>();
+        final Set<Resource> baseTypes = hasSelfByProperty.get(property);
+
+        if (baseTypes != null) {
+            types.addAll(baseTypes);
+
+            // findParent gets all subclasses, add self.
+            for (final Resource baseType : baseTypes) {
+                if (baseType instanceof URI) {
+                    types.addAll(findParents(subClassOfGraph, (URI) baseType));
+                }
+            }
+        }
+
+        // make map hasSelfByProperty[]
+        return types;
+    }
+
+    /**
      * Queries for all items that are in a list of the form:
      * <pre>
      *     <:A> ?x _:bnode1 .
@@ -1187,7 +1291,7 @@ public class InferenceEngine {
 
     public void setRyaDAO(final RyaDAO<?> ryaDAO) {
         this.ryaDAO = ryaDAO;
-        this.ryaDaoQueryWrapper = new RyaDaoQueryWrapper(ryaDAO);
+        ryaDaoQueryWrapper = new RyaDaoQueryWrapper(ryaDAO);
     }
 
     public RdfCloudTripleStoreConfiguration getConf() {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/d47190b1/sail/src/test/java/org/apache/rya/rdftriplestore/inference/HasSelfVisitorTest.java
----------------------------------------------------------------------
diff --git a/sail/src/test/java/org/apache/rya/rdftriplestore/inference/HasSelfVisitorTest.java b/sail/src/test/java/org/apache/rya/rdftriplestore/inference/HasSelfVisitorTest.java
new file mode 100644
index 0000000..ba6b7da
--- /dev/null
+++ b/sail/src/test/java/org/apache/rya/rdftriplestore/inference/HasSelfVisitorTest.java
@@ -0,0 +1,133 @@
+/*
+ * 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.rya.rdftriplestore.inference;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.rya.accumulo.AccumuloRdfConfiguration;
+import org.junit.Assert;
+import org.junit.Test;
+import org.openrdf.model.Resource;
+import org.openrdf.model.URI;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.query.algebra.Extension;
+import org.openrdf.query.algebra.ExtensionElem;
+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.Union;
+import org.openrdf.query.algebra.Var;
+
+public class HasSelfVisitorTest {
+    private final AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
+    private static final ValueFactory VF = new ValueFactoryImpl();
+
+    private static final URI narcissist = VF.createURI("urn:Narcissist");
+    private static final URI love = VF.createURI("urn:love");
+    private static final URI self = VF.createURI("urn:self");
+
+    @Test
+    public void testTypePattern() throws Exception {
+        final InferenceEngine inferenceEngine = mock(InferenceEngine.class);
+        final Set<URI> narcissistProps = new HashSet<>();
+        narcissistProps.add(love);
+        when(inferenceEngine.getHasSelfImplyingType(narcissist)).thenReturn(narcissistProps);
+        final Var subj = new Var("s");
+        final Var obj = new Var("o", narcissist);
+        obj.setConstant(true);
+        final Var pred = new Var("p", RDF.TYPE);
+        pred.setConstant(true);
+
+        final Projection query = new Projection(new StatementPattern(subj, pred, obj),
+                new ProjectionElemList(new ProjectionElem("s", "subject")));
+        query.visit(new HasSelfVisitor(conf, inferenceEngine));
+
+        Assert.assertTrue(query.getArg() instanceof Union);
+        final Union union = (Union) query.getArg();
+        Assert.assertTrue(union.getRightArg() instanceof StatementPattern);
+        Assert.assertTrue(union.getLeftArg() instanceof StatementPattern);
+        final StatementPattern expectedLeft = new StatementPattern(subj, pred, obj);
+        final StatementPattern expectedRight = new StatementPattern(subj, new Var("urn:love", love), subj);
+        Assert.assertEquals(expectedLeft, union.getLeftArg());
+        Assert.assertEquals(expectedRight, union.getRightArg());
+    }
+
+    @Test
+    public void testPropertyPattern_constantSubj() throws Exception {
+        final InferenceEngine inferenceEngine = mock(InferenceEngine.class);
+        final Set<Resource> loveTypes = new HashSet<>();
+        loveTypes.add(narcissist);
+        when(inferenceEngine.getHasSelfImplyingProperty(love)).thenReturn(loveTypes);
+        final Var subj = new Var("s", self);
+        subj.setConstant(true);
+        final Var obj = new Var("o");
+        final Var pred = new Var("p", love);
+        pred.setConstant(true);
+
+        final Projection query = new Projection(new StatementPattern(subj, pred, obj),
+                new ProjectionElemList(new ProjectionElem("s", "subject")));
+        query.visit(new HasSelfVisitor(conf, inferenceEngine));
+
+        Assert.assertTrue(query.getArg() instanceof Union);
+        final Union union = (Union) query.getArg();
+        Assert.assertTrue(union.getRightArg() instanceof StatementPattern);
+        Assert.assertTrue(union.getLeftArg() instanceof Extension);
+        final StatementPattern expectedRight = new StatementPattern(subj, pred, obj);
+        final Extension expectedLeft = new Extension(
+                new StatementPattern(subj, new Var(RDF.TYPE.stringValue(), RDF.TYPE), new Var("urn:Narcissist", narcissist)),
+                new ExtensionElem(subj, "o"));
+        Assert.assertEquals(expectedLeft, union.getLeftArg());
+        Assert.assertEquals(expectedRight, union.getRightArg());
+    }
+
+    @Test
+    public void testPropertyPattern_constantObj() throws Exception {
+        final InferenceEngine inferenceEngine = mock(InferenceEngine.class);
+        final Set<Resource> loveTypes = new HashSet<>();
+        loveTypes.add(narcissist);
+        when(inferenceEngine.getHasSelfImplyingProperty(love)).thenReturn(loveTypes);
+        final Var subj = new Var("s");
+        final Var obj = new Var("o", self);
+        obj.setConstant(true);
+        final Var pred = new Var("p", love);
+        pred.setConstant(true);
+
+        final Projection query = new Projection(new StatementPattern(subj, pred, obj),
+                new ProjectionElemList(new ProjectionElem("s", "subject")));
+        query.visit(new HasSelfVisitor(conf, inferenceEngine));
+
+        Assert.assertTrue(query.getArg() instanceof Union);
+        final Union union = (Union) query.getArg();
+        Assert.assertTrue(union.getRightArg() instanceof StatementPattern);
+        Assert.assertTrue(union.getLeftArg() instanceof Extension);
+        final StatementPattern expectedRight = new StatementPattern(subj, pred, obj);
+        final Extension expectedLeft = new Extension(
+                new StatementPattern(obj, new Var(RDF.TYPE.stringValue(), RDF.TYPE), new Var("urn:Narcissist", narcissist)),
+                new ExtensionElem(obj, "s"));
+        Assert.assertEquals(expectedLeft, union.getLeftArg());
+        Assert.assertEquals(expectedRight, union.getRightArg());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/d47190b1/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
index 7ef56c5..e9725e7 100644
--- a/sail/src/test/java/org/apache/rya/rdftriplestore/inference/InferenceEngineTest.java
+++ b/sail/src/test/java/org/apache/rya/rdftriplestore/inference/InferenceEngineTest.java
@@ -564,4 +564,19 @@ public class InferenceEngineTest extends TestCase {
         final Set<Resource> actualCardRanks = inferenceEngine.getEnumeration(ranks);
         Assert.assertEquals(enumerationImplyingRanks, actualCardRanks);
    }
+
+    @Test
+    public void hasSelfTest() throws Exception {
+        final String ontology = "INSERT DATA { GRAPH <http://updated/test> {\n"
+                + "  <urn:Narcissist> owl:onProperty <urn:love> ; owl:hasSelf \"true\" . \n" + "}}";
+        conn.prepareUpdate(QueryLanguage.SPARQL, ontology).execute();
+        inferenceEngine.refreshGraph();
+        final Set<Resource> expectedTypes = new HashSet<>();
+        expectedTypes.add(vf.createURI("urn:Narcissist"));
+        Assert.assertEquals(expectedTypes, inferenceEngine.getHasSelfImplyingProperty(vf.createURI("urn:love")));
+
+        final Set<URI> expectedProperties = new HashSet<>();
+        expectedProperties.add(vf.createURI("urn:love"));
+        Assert.assertEquals(expectedProperties, inferenceEngine.getHasSelfImplyingType(vf.createURI("urn:Narcissist")));
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/d47190b1/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
index 11dfeb0..5a3e35b 100644
--- a/sail/src/test/java/org/apache/rya/rdftriplestore/inference/InferenceIT.java
+++ b/sail/src/test/java/org/apache/rya/rdftriplestore/inference/InferenceIT.java
@@ -557,4 +557,53 @@ public class InferenceIT extends TestCase {
         Assert.assertEquals(expectedCardRanks.size(), solutions.size());
         Assert.assertEquals(expectedCardRanks, new HashSet<>(solutions));
     }
+
+    @Test
+    public void testHasSelfQuery() throws Exception {
+        final String ontology = "INSERT DATA { GRAPH <http://updated/test> {\n"
+                + "  <urn:Narcissist> owl:onProperty <urn:love> ; owl:hasSelf \"true\" . \n"
+                + "}}";
+        final String instances = "INSERT DATA { GRAPH <http://updated/test> {\n"
+                + "  <urn:Alice> a <urn:Narcissist> . \n"
+                + "  <urn:Narcissus> <urn:love> <urn:Narcissus> . \n"
+                + "}}";
+        conn.prepareUpdate(QueryLanguage.SPARQL, ontology).execute();
+        conn.prepareUpdate(QueryLanguage.SPARQL, instances).execute();
+        inferenceEngine.refreshGraph();
+
+        String query = "SELECT ?who ?self { GRAPH <http://updated/test> { ?self <urn:love> ?who } } \n";
+        conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate(resultHandler);
+        final Set<BindingSet> expected = new HashSet<BindingSet>();
+        final List<String> varNames = new LinkedList<>();
+        varNames.add("who");
+        varNames.add("self");
+        expected.add(new ListBindingSet(varNames, vf.createURI("urn:Alice"), vf.createURI("urn:Alice")));
+        expected.add(new ListBindingSet(varNames, vf.createURI("urn:Narcissus"), vf.createURI("urn:Narcissus")));
+        Assert.assertEquals(expected, new HashSet<>(solutions));
+
+        query = "SELECT ?self { GRAPH <http://updated/test> { <urn:Alice> <urn:love> ?self } } \n";
+        conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate(resultHandler);
+        expected.clear();
+        varNames.clear();
+        varNames.add("self");
+        expected.add(new ListBindingSet(varNames, vf.createURI("urn:Alice")));
+        Assert.assertEquals(expected, new HashSet<>(solutions));
+
+        query = "SELECT ?who { GRAPH <http://updated/test> { ?who <urn:love> <urn:Alice> } } \n";
+        conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate(resultHandler);
+        expected.clear();
+        varNames.clear();
+        varNames.add("who");
+        expected.add(new ListBindingSet(varNames, vf.createURI("urn:Alice")));
+        Assert.assertEquals(expected, new HashSet<>(solutions));
+
+        query = "SELECT ?who { GRAPH <http://updated/test> { ?who a <urn:Narcissist> } } \n";
+        conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate(resultHandler);
+        expected.clear();
+        varNames.clear();
+        varNames.add("who");
+        expected.add(new ListBindingSet(varNames, vf.createURI("urn:Narcissus")));
+        expected.add(new ListBindingSet(varNames, vf.createURI("urn:Alice")));
+        Assert.assertEquals(expected, new HashSet<>(solutions));
+    }
 }