You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by pu...@apache.org on 2016/09/27 15:36:16 UTC

[1/3] incubator-rya git commit: RYA-168 initial checkin

Repository: incubator-rya
Updated Branches:
  refs/heads/master 35e8634aa -> 5867d545d


RYA-168 initial checkin


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

Branch: refs/heads/master
Commit: da8a2e609d942f01bef3b69ec174fcb539fc3159
Parents: 35e8634
Author: pujav65 <pu...@gmail.com>
Authored: Thu Aug 25 13:40:23 2016 -0400
Committer: pujav65 <pu...@gmail.com>
Committed: Tue Sep 27 11:21:27 2016 -0400

----------------------------------------------------------------------
 .../src/main/java/MongoRyaDirectExample.java    | 140 +++++++++++++------
 .../RdfCloudTripleStoreConnection.java          |   2 +
 .../inference/InferenceEngine.java              | 138 ++++++++++++++++--
 .../inference/PropertyChainVisitor.java         | 111 +++++++++++++++
 4 files changed, 336 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da8a2e60/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
----------------------------------------------------------------------
diff --git a/extras/indexingExample/src/main/java/MongoRyaDirectExample.java b/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
index 60b437e..2bc0924 100644
--- a/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
+++ b/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
@@ -85,9 +85,10 @@ public class MongoRyaDirectExample {
             testAddNamespaces(conn);
             testAddPointAndWithinSearch(conn);
             testAddAndFreeTextSearchWithPCJ(conn);
-            // to test out inference, set inference to true in the conf
+           //  to test out inference, set inference to true in the conf
             if (USE_INFER){
             	testInfer(conn, sail);
+            	testPropertyChainInference(conn, sail);
             }
 
             log.info("TIME: " + (System.currentTimeMillis() - start) / 1000.);
@@ -281,49 +282,106 @@ public class MongoRyaDirectExample {
     }
 
 
-
     public static void testAddAndDelete(SailRepositoryConnection conn) throws MalformedQueryException, RepositoryException,
-            UpdateExecutionException, QueryEvaluationException, TupleQueryResultHandlerException {
-
-        // Add data
-        String query = "INSERT DATA\n"//
-                + "{ GRAPH <http://updated/test> {\n"//
-                + "  <http://acme.com/people/Mike> " //
-                + "       <http://acme.com/actions/likes> \"A new book\" ;\n"//
-                + "       <http://acme.com/actions/likes> \"Avocados\" .\n" + "} }";
-
-        log.info("Performing Query");
-
-        Update update = conn.prepareUpdate(QueryLanguage.SPARQL, query);
-        update.execute();
-
-         query = "select ?p ?o { GRAPH <http://updated/test> {<http://acme.com/people/Mike> ?p ?o . }}";
-         CountingResultHandler resultHandler = new CountingResultHandler();
-         TupleQuery tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query);
-         tupleQuery.evaluate(resultHandler);
-         log.info("Result count : " + resultHandler.getCount());
-        
-         Validate.isTrue(resultHandler.getCount() == 2);
-        
-         resultHandler.resetCount();
-        
-         // Delete Data
-         query = "DELETE DATA\n" //
-         + "{ GRAPH <http://updated/test> {\n"
-         + "  <http://acme.com/people/Mike> <http://acme.com/actions/likes> \"A new book\" ;\n"
-         + "   <http://acme.com/actions/likes> \"Avocados\" .\n" + "}}";
-        
-         update = conn.prepareUpdate(QueryLanguage.SPARQL, query);
-         update.execute();
-        
-         query = "select ?p ?o { GRAPH <http://updated/test> {<http://acme.com/people/Mike> ?p ?o . }}";
-         tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query);
-         tupleQuery.evaluate(resultHandler);
-         log.info("Result count : " + resultHandler.getCount());
-        
-         Validate.isTrue(resultHandler.getCount() == 0);
+    UpdateExecutionException, QueryEvaluationException, TupleQueryResultHandlerException {
+
+    	// Add data
+    	String query = "INSERT DATA\n"//
+    			+ "{ GRAPH <http://updated/test> {\n"//
+    			+ "  <http://acme.com/people/Mike> " //
+    			+ "       <http://acme.com/actions/likes> \"A new book\" ;\n"//
+    			+ "       <http://acme.com/actions/likes> \"Avocados\" .\n" + "} }";
+
+    	log.info("Performing Query");
+
+    	Update update = conn.prepareUpdate(QueryLanguage.SPARQL, query);
+    	update.execute();
+
+    	query = "select ?p ?o { GRAPH <http://updated/test> {<http://acme.com/people/Mike> ?p ?o . }}";
+    	CountingResultHandler resultHandler = new CountingResultHandler();
+    	TupleQuery tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query);
+    	tupleQuery.evaluate(resultHandler);
+    	log.info("Result count : " + resultHandler.getCount());
+
+    	Validate.isTrue(resultHandler.getCount() == 2);
+
+    	resultHandler.resetCount();
+
+    	// Delete Data
+    	query = "DELETE DATA\n" //
+    			+ "{ GRAPH <http://updated/test> {\n"
+    			+ "  <http://acme.com/people/Mike> <http://acme.com/actions/likes> \"A new book\" ;\n"
+    			+ "   <http://acme.com/actions/likes> \"Avocados\" .\n" + "}}";
+
+    	update = conn.prepareUpdate(QueryLanguage.SPARQL, query);
+    	update.execute();
+
+    	query = "select ?p ?o { GRAPH <http://updated/test> {<http://acme.com/people/Mike> ?p ?o . }}";
+    	tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query);
+    	tupleQuery.evaluate(resultHandler);
+    	log.info("Result count : " + resultHandler.getCount());
+
+    	Validate.isTrue(resultHandler.getCount() == 0);
     }
 
+    public static void testPropertyChainInference(SailRepositoryConnection conn, Sail sail) throws MalformedQueryException, RepositoryException,
+    UpdateExecutionException, QueryEvaluationException, TupleQueryResultHandlerException, InferenceEngineException {
+
+    	// Add data
+    	String query = "INSERT DATA\n"//
+    			+ "{ GRAPH <http://updated/test> {\n"//
+    			+ "  <urn:paulGreatGrandfather> <urn:father> <urn:paulGrandfather> . "
+    			+ "  <urn:paulGrandfather> <urn:father> <urn:paulFather> . " + 
+    			" <urn:paulFather> <urn:father> <urn:paul> . " + 
+    			" <urn:paul> <urn:father> <urn:paulSon> .  }}";
+
+    	log.info("Performing Query");
+
+    	Update update = conn.prepareUpdate(QueryLanguage.SPARQL, query);
+    	update.execute();
+
+    	query = "select ?p { GRAPH <http://updated/test> {<urn:paulGreatGrandfather> <urn:father>/<urn:father> ?p}}";
+    	CountingResultHandler resultHandler = new CountingResultHandler();
+    	TupleQuery tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query);
+    	tupleQuery.evaluate(resultHandler);
+    	log.info("Result count : " + resultHandler.getCount());
+
+
+    	// try adding a property chain and querying for it
+    	query = "INSERT DATA\n"//
+    			+ "{ GRAPH <http://updated/test> {\n"//
+    			+ "  <urn:greatGrandfather> owl:propertyChainAxiom <urn:1234>  . " + 
+    			" <urn:1234> <http://www.w3.org/2000/10/swap/list#length> 3 . " + 
+    			" <urn:1234> <http://www.w3.org/2000/10/swap/list#index> (0 <urn:father>) . " + 
+    			" <urn:1234> <http://www.w3.org/2000/10/swap/list#index> (1 <urn:father>) . " + 
+    			" <urn:1234> <http://www.w3.org/2000/10/swap/list#index> (2 <urn:father>) .  }}";
+    	update = conn.prepareUpdate(QueryLanguage.SPARQL, query);
+    	update.execute();
+    	query = "INSERT DATA\n"//
+    			+ "{ GRAPH <http://updated/test> {\n"//
+    			+ "  <urn:grandfather> owl:propertyChainAxiom <urn:12344>  . " + 
+    			" <urn:12344> <http://www.w3.org/2000/10/swap/list#length> 2 . " + 
+    			" <urn:12344> <http://www.w3.org/2000/10/swap/list#index> (0 <urn:father>) . " + 
+    			" <urn:12344> <http://www.w3.org/2000/10/swap/list#index> (1 <urn:father>) .  }}";
+    	update = conn.prepareUpdate(QueryLanguage.SPARQL, query);
+    	update.execute();
+    	((RdfCloudTripleStore) sail).getInferenceEngine().refreshGraph();
+
+    	resultHandler.resetCount();
+    	query = "select ?p { GRAPH <http://updated/test> {<urn:paulGreatGrandfather> <urn:greatGrandfather> ?p}}";
+    	resultHandler = new CountingResultHandler();
+    	tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query);
+    	tupleQuery.evaluate(resultHandler);
+    	log.info("Result count : " + resultHandler.getCount());
+
+    	resultHandler.resetCount();
+    	query = "select ?s ?p { GRAPH <http://updated/test> {?s <urn:grandfather> ?p}}";
+    	resultHandler = new CountingResultHandler();
+    	tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query);
+    	tupleQuery.evaluate(resultHandler);
+    	log.info("Result count : " + resultHandler.getCount());
+
+    }
     
     public static void testInfer(SailRepositoryConnection conn, Sail sail) throws MalformedQueryException, RepositoryException, 
     UpdateExecutionException, QueryEvaluationException, TupleQueryResultHandlerException, InferenceEngineException {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da8a2e60/sail/src/main/java/mvm/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/mvm/rya/rdftriplestore/RdfCloudTripleStoreConnection.java b/sail/src/main/java/mvm/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
index 24ec109..048046f 100644
--- a/sail/src/main/java/mvm/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
+++ b/sail/src/main/java/mvm/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
@@ -50,6 +50,7 @@ import mvm.rya.rdftriplestore.evaluation.RdfCloudTripleStoreSelectivityEvaluatio
 import mvm.rya.rdftriplestore.evaluation.SeparateFilterJoinsVisitor;
 import mvm.rya.rdftriplestore.inference.InferenceEngine;
 import mvm.rya.rdftriplestore.inference.InverseOfVisitor;
+import mvm.rya.rdftriplestore.inference.PropertyChainVisitor;
 import mvm.rya.rdftriplestore.inference.SameAsVisitor;
 import mvm.rya.rdftriplestore.inference.SubClassOfVisitor;
 import mvm.rya.rdftriplestore.inference.SubPropertyOfVisitor;
@@ -343,6 +344,7 @@ public class RdfCloudTripleStoreConnection extends SailConnectionBase {
                     && this.inferenceEngine != null
                     ) {
                 try {
+                    tupleExpr.visit(new PropertyChainVisitor(queryConf, inferenceEngine));
                     tupleExpr.visit(new TransitivePropertyVisitor(queryConf, inferenceEngine));
                     tupleExpr.visit(new SymmetricPropertyVisitor(queryConf, inferenceEngine));
                     tupleExpr.visit(new InverseOfVisitor(queryConf, inferenceEngine));

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da8a2e60/sail/src/main/java/mvm/rya/rdftriplestore/inference/InferenceEngine.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/mvm/rya/rdftriplestore/inference/InferenceEngine.java b/sail/src/main/java/mvm/rya/rdftriplestore/inference/InferenceEngine.java
index 0e2e930..c323892 100644
--- a/sail/src/main/java/mvm/rya/rdftriplestore/inference/InferenceEngine.java
+++ b/sail/src/main/java/mvm/rya/rdftriplestore/inference/InferenceEngine.java
@@ -1,5 +1,30 @@
 package mvm.rya.rdftriplestore.inference;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.TreeMap;
+
+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.StatementImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.model.vocabulary.OWL;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.model.vocabulary.RDFS;
+import org.openrdf.query.QueryEvaluationException;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -26,25 +51,12 @@ import com.tinkerpop.blueprints.Edge;
 import com.tinkerpop.blueprints.Graph;
 import com.tinkerpop.blueprints.Vertex;
 import com.tinkerpop.blueprints.impls.tg.TinkerGraphFactory;
+
 import info.aduna.iteration.CloseableIteration;
 import mvm.rya.api.RdfCloudTripleStoreConfiguration;
 import mvm.rya.api.persist.RyaDAO;
 import mvm.rya.api.persist.RyaDAOException;
 import mvm.rya.api.persist.utils.RyaDAOHelper;
-import org.openrdf.model.Resource;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.model.Value;
-import org.openrdf.model.impl.StatementImpl;
-import org.openrdf.model.vocabulary.OWL;
-import org.openrdf.model.vocabulary.RDF;
-import org.openrdf.model.vocabulary.RDFS;
-import org.openrdf.query.QueryEvaluationException;
-
-import java.util.*;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
 
 /**
  * Will pull down inference relationships from dao every x seconds. <br>
@@ -66,6 +78,7 @@ public class InferenceEngine {
 
     private long refreshGraphSchedule = 5 * 60 * 1000; //5 min
     private Timer timer;
+	private HashMap<URI, List<URI>> propertyChainPropertyToChain = new HashMap<URI, List<URI>>();
     public static final String URI_PROP = "uri";
 
     public void init() throws InferenceEngineException {
@@ -204,6 +217,92 @@ public class InferenceEngine {
                 }
             }
             inverseOfMap = invProp;
+            
+            ValueFactory vf = ValueFactoryImpl.getInstance();
+            iter = RyaDAOHelper.query(ryaDAO, null, 
+            		vf.createURI("http://www.w3.org/2002/07/owl#propertyChainAxiom"),
+            		null, conf);
+            Map<URI,URI> propertyChainPropertiesToBNodes = new HashMap<URI, URI>();
+            propertyChainPropertyToChain = new HashMap<URI, List<URI>>();
+            try {
+            	while (iter.hasNext()){
+            		Statement st = iter.next();
+            		propertyChainPropertiesToBNodes.put((URI)st.getSubject(), (URI)st.getObject());
+            	}
+            } finally {
+                if (iter != null) {
+                    iter.close();
+                }
+            }
+            // now for each property chain bNode, get the indexed list of properties associated with that chain
+            for (URI propertyChainProperty : propertyChainPropertiesToBNodes.keySet()){
+            	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"),
+            			null, conf);
+            	TreeMap<Integer, URI> orderedProperties = new TreeMap<Integer, URI>();
+            	// TODO refactor this.  Wish I could execute sparql
+            	try {
+            		while (iter.hasNext()){
+            		  Statement st = iter.next();
+            		  String indexedElement = st.getObject().stringValue();
+            		  System.out.println(indexedElement);
+            		  CloseableIteration<Statement, QueryEvaluationException>  iter2 = RyaDAOHelper.query(ryaDAO, vf.createURI(st.getObject().stringValue()), RDF.FIRST,
+                    			null, conf);
+            		  String integerValue = "";
+            		  Value anonPropNode = null;
+            		  Value propURI = null;
+            		  if (iter2 != null){
+            			  while (iter2.hasNext()){
+            				  Statement iter2Statement = iter2.next();
+            				  integerValue = iter2Statement.getObject().stringValue();
+            				  break;
+            			  }
+            			  iter2.close();
+            		  }
+            		  iter2 = RyaDAOHelper.query(ryaDAO, vf.createURI(st.getObject().stringValue()), RDF.REST,
+                  			null, conf);
+            		  if (iter2 != null){
+            			  while (iter2.hasNext()){
+            				  Statement iter2Statement = iter2.next();
+            				  anonPropNode = iter2Statement.getObject();
+            				  break;
+            			  }
+            			  iter2.close();
+            			  if (anonPropNode != null){
+            				  iter2 = RyaDAOHelper.query(ryaDAO, vf.createURI(anonPropNode.stringValue()), RDF.FIRST,
+                            			null, conf);
+            				  while (iter2.hasNext()){
+                				  Statement iter2Statement = iter2.next();
+                				  propURI = iter2Statement.getObject();
+                				  break;
+                			  }
+                			  iter2.close();
+            			  }
+            		  }
+            		  if (!integerValue.isEmpty() && propURI!=null) {
+            			  try {
+                			  int indexValue = Integer.parseInt(integerValue);
+                			  URI chainPropURI = vf.createURI(propURI.stringValue());
+                			  orderedProperties.put(indexValue, chainPropURI);
+            			  }
+            			  catch (Exception ex){
+            				  // TODO log an error here
+            				  
+            			  }
+            		  }
+            		}
+            	} finally{
+            		if (iter != null){
+            			iter.close();
+            		}
+            	}
+            	List<URI> properties = new ArrayList<URI>();
+            	for (Map.Entry<Integer, URI> entry : orderedProperties.entrySet()){
+            		properties.add(entry.getValue());
+            	}
+            	propertyChainPropertyToChain.put(propertyChainProperty, properties);
+            }
         } catch (QueryEvaluationException e) {
             throw new InferenceEngineException(e);
         }
@@ -365,6 +464,17 @@ public class InferenceEngine {
         return subClassOfGraph;
     }
 
+    public Map<URI, List<URI>> getPropertyChainMap() {
+        return propertyChainPropertyToChain;
+    }
+
+    public List<URI> getPropertyChain(URI chainProp) {
+    	if (propertyChainPropertyToChain.containsKey(chainProp)){
+    		return propertyChainPropertyToChain.get(chainProp);
+    	}
+        return new ArrayList<URI>();
+    }
+
     public Graph getSubPropertyOfGraph() {
         return subPropertyOfGraph;
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da8a2e60/sail/src/main/java/mvm/rya/rdftriplestore/inference/PropertyChainVisitor.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/mvm/rya/rdftriplestore/inference/PropertyChainVisitor.java b/sail/src/main/java/mvm/rya/rdftriplestore/inference/PropertyChainVisitor.java
new file mode 100644
index 0000000..06d557f
--- /dev/null
+++ b/sail/src/main/java/mvm/rya/rdftriplestore/inference/PropertyChainVisitor.java
@@ -0,0 +1,111 @@
+package mvm.rya.rdftriplestore.inference;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import org.openrdf.model.URI;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.model.vocabulary.RDFS;
+import org.openrdf.model.vocabulary.SESAME;
+import org.openrdf.query.algebra.Join;
+import org.openrdf.query.algebra.QueryModelNode;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.TupleExpr;
+import org.openrdf.query.algebra.Var;
+
+/*
+ * 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 mvm.rya.api.RdfCloudTripleStoreConfiguration;
+import mvm.rya.api.utils.NullableStatementImpl;
+import mvm.rya.rdftriplestore.utils.FixedStatementPattern;
+import mvm.rya.rdftriplestore.utils.FixedStatementPattern;
+
+/**
+ * All predicates are changed
+ * Class SubPropertyOfVisitor
+ * Date: Mar 29, 2011
+ * Time: 11:28:34 AM
+ */
+public class PropertyChainVisitor extends AbstractInferVisitor {
+
+    public PropertyChainVisitor(RdfCloudTripleStoreConfiguration conf, InferenceEngine inferenceEngine) {
+        super(conf, inferenceEngine);
+        include = true;
+    }
+
+
+    @Override
+    protected void meetSP(StatementPattern node) throws Exception {
+        StatementPattern sp = node.clone();
+        final Var predVar = sp.getPredicateVar();
+
+        URI pred = (URI) predVar.getValue();
+        String predNamespace = pred.getNamespace();
+
+        final Var objVar = sp.getObjectVar();
+        final Var cntxtVar = sp.getContextVar();
+        if (objVar != null &&
+                !RDF.NAMESPACE.equals(predNamespace) &&
+                !SESAME.NAMESPACE.equals(predNamespace) &&
+                !RDFS.NAMESPACE.equals(predNamespace)
+                && !EXPANDED.equals(cntxtVar)) {
+ 
+            URI chainPropURI = (URI) predVar.getValue();
+            List<URI> chain = inferenceEngine.getPropertyChain(chainPropURI);
+            List<StatementPattern> expandedPatterns = new ArrayList<StatementPattern>();
+            if (chain.size() > 0) {
+            	Var originalSubj = sp.getSubjectVar();
+            	Var originalObj = sp.getObjectVar();
+            	
+            	Var nextSubj = originalSubj;
+            	StatementPattern lastStatementPatternAdded = null;
+            	for (URI chainElement : chain ){
+            		String s = UUID.randomUUID().toString();
+                    Var currentObj = new Var("c-" + s);
+            		StatementPattern statementPattern = new StatementPattern(nextSubj, new Var(chainElement.stringValue()), currentObj, sp.getContextVar());
+            		expandedPatterns.add(statementPattern);
+            		lastStatementPatternAdded = statementPattern;
+            		nextSubj = statementPattern.getObjectVar();
+            	}
+            	lastStatementPatternAdded.setObjectVar(originalObj);
+            	
+            	TupleExpr lastRight = null;
+            	// replace the statement pattern with a series of joins
+            	for (StatementPattern pattern : expandedPatterns){
+            		if (lastRight == null){
+            			lastRight = pattern;
+            		}
+            		else {
+            			Join newJoin = new Join(pattern, lastRight);
+            			lastRight = newJoin;
+            		}
+            		System.out.println(pattern);
+            	}
+            	if (lastRight != null){
+                	node.replaceWith(lastRight);  	
+            	}
+            	
+            }
+        }
+    }
+}


[2/3] incubator-rya git commit: RYA-168 adding test and support for alternate representation

Posted by pu...@apache.org.
RYA-168 adding test and support for alternate representation


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

Branch: refs/heads/master
Commit: 5edfd65e58f97f1ff6188a69c1d719f9701a94e4
Parents: da8a2e6
Author: pujav65 <pu...@gmail.com>
Authored: Mon Sep 26 12:30:05 2016 -0400
Committer: pujav65 <pu...@gmail.com>
Committed: Tue Sep 27 11:25:20 2016 -0400

----------------------------------------------------------------------
 .../mvm/rya/sail/config/RyaSailFactory.java     |   5 +-
 .../src/main/java/MongoRyaDirectExample.java    |  51 +++++++++
 .../src/main/java/RyaDirectExample.java         |  22 ++++
 .../inference/InferenceEngine.java              |  57 ++++++++++
 .../rdftriplestore/inference/InverseURI.java    |  46 ++++++++
 .../inference/PropertyChainVisitor.java         |   9 +-
 .../inference/PropertyChainTest.java            | 106 +++++++++++++++++++
 7 files changed, 290 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5edfd65e/extras/indexing/src/main/java/mvm/rya/sail/config/RyaSailFactory.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/sail/config/RyaSailFactory.java b/extras/indexing/src/main/java/mvm/rya/sail/config/RyaSailFactory.java
index afffb68..b0a719b 100644
--- a/extras/indexing/src/main/java/mvm/rya/sail/config/RyaSailFactory.java
+++ b/extras/indexing/src/main/java/mvm/rya/sail/config/RyaSailFactory.java
@@ -89,9 +89,8 @@ public class RyaSailFactory {
                 final MongoRyaInstanceDetailsRepository ryaDetailsRepo = new MongoRyaInstanceDetailsRepository(client, mongoConfig.getCollectionName());
                 RyaDetailsToConfiguration.addRyaDetailsToConfiguration(ryaDetailsRepo.getRyaInstanceDetails(), mongoConfig);
             } catch (final RyaDetailsRepositoryException e) {
-                LOG.info("Instance does not have a rya details collection, skipping.");
-            }
-            dao = getMongoDAO((MongoDBRdfConfiguration)rdfConfig, client);
+               LOG.info("Instance does not have a rya details collection, skipping.");
+           }            dao = getMongoDAO((MongoDBRdfConfiguration)rdfConfig, client);
         } else {
             rdfConfig = new AccumuloRdfConfiguration(config);
             user = rdfConfig.get(ConfigUtils.CLOUDBASE_USER);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5edfd65e/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
----------------------------------------------------------------------
diff --git a/extras/indexingExample/src/main/java/MongoRyaDirectExample.java b/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
index 2bc0924..8a2bac5 100644
--- a/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
+++ b/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
@@ -89,6 +89,7 @@ public class MongoRyaDirectExample {
             if (USE_INFER){
             	testInfer(conn, sail);
             	testPropertyChainInference(conn, sail);
+            	testPropertyChainInferenceAltRepresentation(conn, sail);
             }
 
             log.info("TIME: " + (System.currentTimeMillis() - start) / 1000.);
@@ -324,6 +325,55 @@ public class MongoRyaDirectExample {
     	Validate.isTrue(resultHandler.getCount() == 0);
     }
 
+    
+    public static void testPropertyChainInferenceAltRepresentation(SailRepositoryConnection conn, Sail sail) throws MalformedQueryException, RepositoryException,
+    UpdateExecutionException, QueryEvaluationException, TupleQueryResultHandlerException, InferenceEngineException {
+
+    	// Add data
+    	String query = "INSERT DATA\n"//
+    			+ "{ GRAPH <http://updated/test> {\n"//
+    			+ "  <urn:jenGreatGranMother> <urn:Motherof> <urn:jenGranMother> . "
+    			+ "  <urn:jenGranMother> <urn:isChildOf> <urn:jenGreatGranMother> . "
+    			+ "  <urn:jenGranMother> <urn:Motherof> <urn:jenMother> . " 
+    			+ "  <urn:jenMother> <urn:isChildOf> <urn:jenGranMother> . "
+    			+ " <urn:jenMother> <urn:Motherof> <urn:jen> . "
+    			+ "  <urn:jen> <urn:isChildOf> <urn:jenMother> . "
+    			+ " <urn:jen> <urn:Motherof> <urn:jenDaughter> .  }}";
+
+    	log.info("Performing Query");
+
+    	Update update = conn.prepareUpdate(QueryLanguage.SPARQL, query);
+    	update.execute();
+
+    	query = "select ?p { GRAPH <http://updated/test> {?s <urn:Motherof>/<urn:Motherof> ?p}}";
+    	CountingResultHandler resultHandler = new CountingResultHandler();
+    	TupleQuery tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query);
+    	tupleQuery.evaluate(resultHandler);
+    	log.info("Result count : " + resultHandler.getCount());
+
+
+    	// try adding a property chain and querying for it
+    	query = "INSERT DATA\n"//
+    			+ "{ GRAPH <http://updated/test> {\n"//
+    			+ "  <urn:greatMother> owl:propertyChainAxiom <urn:12342>  . " + 
+    			" <urn:12342> <http://www.w3.org/1999/02/22-rdf-syntax-ns#first> _:node1atjakcvbx15023 . " + 
+    			" _:node1atjakcvbx15023 <http://www.w3.org/2002/07/owl#inverseOf> <urn:isChildOf> . " + 
+    			" <urn:12342> <http://www.w3.org/1999/02/22-rdf-syntax-ns#rest> _:node1atjakcvbx15123 . " + 
+       			" _:node1atjakcvbx15123 <http://www.w3.org/1999/02/22-rdf-syntax-ns#rest> <http://www.w3.org/1999/02/22-rdf-syntax-ns#nil> . " + 
+    			" _:node1atjakcvbx15123 <http://www.w3.org/1999/02/22-rdf-syntax-ns#first> <urn:MotherOf> .  }}";
+    	update = conn.prepareUpdate(QueryLanguage.SPARQL, query);
+    	update.execute();
+    	((RdfCloudTripleStore) sail).getInferenceEngine().refreshGraph();
+
+    	resultHandler.resetCount();
+    	query = "select ?x { GRAPH <http://updated/test> {<urn:jenGreatGranMother> <urn:greatMother> ?x}}";
+    	resultHandler = new CountingResultHandler();
+    	tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query);
+    	tupleQuery.evaluate(resultHandler);
+    	log.info("Result count : " + resultHandler.getCount());
+
+    }
+    
     public static void testPropertyChainInference(SailRepositoryConnection conn, Sail sail) throws MalformedQueryException, RepositoryException,
     UpdateExecutionException, QueryEvaluationException, TupleQueryResultHandlerException, InferenceEngineException {
 
@@ -480,6 +530,7 @@ public class MongoRyaDirectExample {
         @Override
         public void handleSolution(BindingSet arg0) throws TupleQueryResultHandlerException {
             count++;
+            System.out.println(arg0);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5edfd65e/extras/indexingExample/src/main/java/RyaDirectExample.java
----------------------------------------------------------------------
diff --git a/extras/indexingExample/src/main/java/RyaDirectExample.java b/extras/indexingExample/src/main/java/RyaDirectExample.java
index 03712ba..acc5617 100644
--- a/extras/indexingExample/src/main/java/RyaDirectExample.java
+++ b/extras/indexingExample/src/main/java/RyaDirectExample.java
@@ -17,7 +17,10 @@
  * under the License.
  */
 
+import java.io.File;
+import java.io.FileInputStream;
 import java.net.UnknownHostException;
+import java.util.Collection;
 import java.util.List;
 
 import org.apache.accumulo.core.client.AccumuloException;
@@ -33,6 +36,7 @@ import org.apache.log4j.Logger;
 import org.apache.rya.indexing.pcj.storage.PcjException;
 import org.apache.rya.indexing.pcj.storage.accumulo.PcjTables;
 import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
+import org.openrdf.model.Statement;
 import org.openrdf.model.URI;
 import org.openrdf.model.ValueFactory;
 import org.openrdf.model.impl.LiteralImpl;
@@ -52,6 +56,10 @@ import org.openrdf.query.UpdateExecutionException;
 import org.openrdf.repository.RepositoryException;
 import org.openrdf.repository.sail.SailRepository;
 import org.openrdf.repository.sail.SailRepositoryConnection;
+import org.openrdf.rio.RDFFormat;
+import org.openrdf.rio.RDFParser;
+import org.openrdf.rio.Rio;
+import org.openrdf.rio.helpers.StatementCollector;
 import org.openrdf.sail.Sail;
 import org.openrdf.sail.SailException;
 
@@ -81,6 +89,20 @@ public class RyaDirectExample {
 	private static final String AUTHS = "U";
 
 	public static void main(final String[] args) throws Exception {
+		
+		File inputFile = new File("C:\\SPARTADEV\\propChain.ttl");
+		RDFParser parser = Rio.createParser(RDFFormat.TURTLE);
+		StatementCollector coll = new StatementCollector();
+		parser.setRDFHandler(coll);
+		parser.parse(new FileInputStream(inputFile), "");
+		Collection<Statement> statements = coll.getStatements();
+		System.out.println("Start");
+		for (Statement statement : statements){
+			System.out.println(statement);
+		}
+		System.out.println("End");
+		System.exit(-1);
+		
 		final Configuration conf = getConf();
 		conf.setBoolean(ConfigUtils.DISPLAY_QUERY_PLAN, PRINT_QUERIES);
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5edfd65e/sail/src/main/java/mvm/rya/rdftriplestore/inference/InferenceEngine.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/mvm/rya/rdftriplestore/inference/InferenceEngine.java b/sail/src/main/java/mvm/rya/rdftriplestore/inference/InferenceEngine.java
index c323892..b7288b2 100644
--- a/sail/src/main/java/mvm/rya/rdftriplestore/inference/InferenceEngine.java
+++ b/sail/src/main/java/mvm/rya/rdftriplestore/inference/InferenceEngine.java
@@ -303,6 +303,63 @@ public class InferenceEngine {
             	}
             	propertyChainPropertyToChain.put(propertyChainProperty, properties);
             }
+            
+            // could also be represented as a list of properties (some of which may be blank nodes)
+            for (URI propertyChainProperty : propertyChainPropertiesToBNodes.keySet()){
+            	List<URI> existingChain = propertyChainPropertyToChain.get(propertyChainProperty);
+            	// if we didn't get a chain, try to get it through following the collection
+            	if ((existingChain == null) || existingChain.isEmpty()) {
+            		
+          		  CloseableIteration<Statement, QueryEvaluationException>  iter2 = RyaDAOHelper.query(ryaDAO, propertyChainPropertiesToBNodes.get(propertyChainProperty), RDF.FIRST,
+              			null, conf);
+          		  List<URI> properties = new ArrayList<URI>();
+          		  URI previousBNode = propertyChainPropertiesToBNodes.get(propertyChainProperty);
+            	  if (iter2.hasNext()) {
+            		  Statement iter2Statement = iter2.next();
+            		  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,
+                          			null, conf);
+                			  if (iter2.hasNext()){
+                				  iter2Statement = iter2.next();
+                				  if (iter2Statement.getObject() instanceof URI){
+                					  properties.add((URI)iter2Statement.getObject());
+                				  }
+                			  }
+                			  // otherwise see if there is an inverse declaration
+                			  else {
+                				  iter2 = RyaDAOHelper.query(ryaDAO, vf.createURI(currentPropValue.stringValue()), OWL.INVERSEOF,
+                                			null, conf);
+                				  if (iter2.hasNext()){
+                    				  iter2Statement = iter2.next();
+                    				  if (iter2Statement.getObject() instanceof URI){
+                    					  properties.add(new InverseURI((URI)iter2Statement.getObject()));
+                    				  }
+                    			  }
+                			  }
+            				  // get the next prop pointer
+            				  iter2 = RyaDAOHelper.query(ryaDAO, previousBNode, RDF.REST,
+                            			null, conf);
+            				  if (iter2.hasNext()){
+                				  iter2Statement = iter2.next();
+                				  previousBNode = (URI)currentPropValue;
+                				  currentPropValue = iter2Statement.getObject();
+                			  }
+            				  else {
+            					  currentPropValue = null;
+            				  }
+                		  }
+                		  else {
+                		    currentPropValue = null;
+                		  }
+            			  
+            		  }
+                  	propertyChainPropertyToChain.put(propertyChainProperty, properties);
+            	  }
+            	}
+            }
+            
         } catch (QueryEvaluationException e) {
             throw new InferenceEngineException(e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5edfd65e/sail/src/main/java/mvm/rya/rdftriplestore/inference/InverseURI.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/mvm/rya/rdftriplestore/inference/InverseURI.java b/sail/src/main/java/mvm/rya/rdftriplestore/inference/InverseURI.java
new file mode 100644
index 0000000..c182fb5
--- /dev/null
+++ b/sail/src/main/java/mvm/rya/rdftriplestore/inference/InverseURI.java
@@ -0,0 +1,46 @@
+package mvm.rya.rdftriplestore.inference;
+
+import org.openrdf.model.URI;
+import org.openrdf.model.impl.URIImpl;
+
+public class InverseURI implements URI {
+
+	private URI impl;
+	
+	public InverseURI(URI uri) {
+		this.impl = uri;
+	}
+
+	@Override
+	public int hashCode() {
+		final int prime = 31;
+		int result = 1;
+		result = prime * result + ((impl == null) ? 0 : impl.hashCode());
+		return result;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof InverseURI){
+			return impl.equals(((InverseURI) obj).impl);
+		}
+		return false;
+	}
+
+	@Override
+	public String stringValue() {
+		return impl.stringValue();
+	}
+
+	@Override
+	public String getNamespace() {
+		return impl.getNamespace();
+	}
+
+	@Override
+	public String getLocalName() {
+		return impl.getLocalName();
+	}
+	
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5edfd65e/sail/src/main/java/mvm/rya/rdftriplestore/inference/PropertyChainVisitor.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/mvm/rya/rdftriplestore/inference/PropertyChainVisitor.java b/sail/src/main/java/mvm/rya/rdftriplestore/inference/PropertyChainVisitor.java
index 06d557f..6e487b2 100644
--- a/sail/src/main/java/mvm/rya/rdftriplestore/inference/PropertyChainVisitor.java
+++ b/sail/src/main/java/mvm/rya/rdftriplestore/inference/PropertyChainVisitor.java
@@ -82,10 +82,14 @@ public class PropertyChainVisitor extends AbstractInferVisitor {
             	for (URI chainElement : chain ){
             		String s = UUID.randomUUID().toString();
                     Var currentObj = new Var("c-" + s);
-            		StatementPattern statementPattern = new StatementPattern(nextSubj, new Var(chainElement.stringValue()), currentObj, sp.getContextVar());
+                    StatementPattern statementPattern = new StatementPattern(nextSubj, new Var(chainElement.stringValue()), currentObj, sp.getContextVar());
+                    if (chainElement instanceof InverseURI){
+                    	statementPattern = new StatementPattern(currentObj, new Var(chainElement.stringValue()), nextSubj, sp.getContextVar());
+                    }
             		expandedPatterns.add(statementPattern);
             		lastStatementPatternAdded = statementPattern;
-            		nextSubj = statementPattern.getObjectVar();
+                    nextSubj = currentObj;
+           		
             	}
             	lastStatementPatternAdded.setObjectVar(originalObj);
             	
@@ -99,7 +103,6 @@ public class PropertyChainVisitor extends AbstractInferVisitor {
             			Join newJoin = new Join(pattern, lastRight);
             			lastRight = newJoin;
             		}
-            		System.out.println(pattern);
             	}
             	if (lastRight != null){
                 	node.replaceWith(lastRight);  	

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5edfd65e/sail/src/test/java/mvm/rya/triplestore/inference/PropertyChainTest.java
----------------------------------------------------------------------
diff --git a/sail/src/test/java/mvm/rya/triplestore/inference/PropertyChainTest.java b/sail/src/test/java/mvm/rya/triplestore/inference/PropertyChainTest.java
new file mode 100644
index 0000000..8592e0a
--- /dev/null
+++ b/sail/src/test/java/mvm/rya/triplestore/inference/PropertyChainTest.java
@@ -0,0 +1,106 @@
+package mvm.rya.triplestore.inference;
+
+import java.util.List;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.TablePermission;
+import org.junit.Assert;
+import org.junit.Test;
+import org.openrdf.model.URI;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.QueryLanguage;
+import org.openrdf.query.Update;
+import org.openrdf.repository.sail.SailRepository;
+import org.openrdf.repository.sail.SailRepositoryConnection;
+
+import junit.framework.TestCase;
+import mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.accumulo.AccumuloRyaDAO;
+import mvm.rya.api.RdfCloudTripleStoreConstants;
+import mvm.rya.rdftriplestore.RdfCloudTripleStore;
+import mvm.rya.rdftriplestore.inference.InferenceEngine;
+import mvm.rya.rdftriplestore.inference.InverseURI;
+
+public class PropertyChainTest extends TestCase {
+    private String user = "user";
+    private String pwd = "pwd";
+    private String instance = "myinstance";
+    private String tablePrefix = "t_";
+    private Authorizations auths = Constants.NO_AUTHS;
+    private Connector connector;
+    private AccumuloRyaDAO ryaDAO;
+    private ValueFactory vf = new ValueFactoryImpl();
+    private String namespace = "urn:test#";
+    private AccumuloRdfConfiguration conf;
+
+    @Override
+    public void setUp() throws Exception {
+        super.setUp();
+        connector = new MockInstance(instance).getConnector(user, pwd.getBytes());
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
+        connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
+        SecurityOperations secOps = connector.securityOperations();
+        secOps.createUser(user, pwd.getBytes(), auths);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX, TablePermission.READ);
+        secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX, TablePermission.READ);
+
+        conf = new AccumuloRdfConfiguration();
+        ryaDAO = new AccumuloRyaDAO();
+        ryaDAO.setConnector(connector);
+        conf.setTablePrefix(tablePrefix);
+        ryaDAO.setConf(conf);
+        ryaDAO.init();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        super.tearDown();
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
+        connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_NS_SUFFIX);
+    }
+
+    @Test
+    public void testGraphConfiguration() throws Exception {
+        // build a connection
+        RdfCloudTripleStore store = new RdfCloudTripleStore();
+        store.setConf(conf);
+        store.setRyaDAO(ryaDAO);
+        InferenceEngine inferenceEngine = new InferenceEngine();
+        inferenceEngine.setRyaDAO(ryaDAO);
+        store.setInferenceEngine(inferenceEngine);
+        inferenceEngine.refreshGraph();
+        store.initialize();
+        SailRepository repository = new SailRepository(store);
+        SailRepositoryConnection conn = repository.getConnection();
+        
+
+        
+    	String query = "INSERT DATA\n"//
+    			+ "{ GRAPH <http://updated/test> {\n"//
+    			+ "  <urn:greatMother> owl:propertyChainAxiom <urn:12342>  . " + 
+    			" <urn:12342> <http://www.w3.org/1999/02/22-rdf-syntax-ns#first> _:node1atjakcvbx15023 . " + 
+    			" _:node1atjakcvbx15023 <http://www.w3.org/2002/07/owl#inverseOf> <urn:isChildOf> . " + 
+    			" <urn:12342> <http://www.w3.org/1999/02/22-rdf-syntax-ns#rest> _:node1atjakcvbx15123 . " + 
+       			" _:node1atjakcvbx15123 <http://www.w3.org/1999/02/22-rdf-syntax-ns#rest> <http://www.w3.org/1999/02/22-rdf-syntax-ns#nil> . " + 
+    			" _:node1atjakcvbx15123 <http://www.w3.org/1999/02/22-rdf-syntax-ns#first> <urn:MotherOf> .  }}";
+    	Update update = conn.prepareUpdate(QueryLanguage.SPARQL, query);
+    	update.execute();
+        inferenceEngine.refreshGraph();
+       List<URI> chain = inferenceEngine.getPropertyChain(vf.createURI("urn:greatMother"));
+       Assert.assertEquals(chain.size(), 2);
+       Assert.assertEquals(chain.get(0), new InverseURI(vf.createURI("urn:isChildOf")));
+       Assert.assertEquals(chain.get(1), vf.createURI("urn:MotherOf"));
+ 
+    }
+}


[3/3] incubator-rya git commit: RYA-169 updating RAT licenses. Closes #87.

Posted by pu...@apache.org.
RYA-169 updating RAT licenses.  Closes #87.


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

Branch: refs/heads/master
Commit: 5867d545d5bc1e9f00fc8cc8fcd331734cb7bfcb
Parents: 5edfd65
Author: pujav65 <pu...@gmail.com>
Authored: Tue Sep 27 11:34:43 2016 -0400
Committer: pujav65 <pu...@gmail.com>
Committed: Tue Sep 27 11:34:43 2016 -0400

----------------------------------------------------------------------
 .../rdftriplestore/inference/InverseURI.java    | 38 ++++++++++++++++++--
 .../inference/PropertyChainTest.java            | 38 ++++++++++++++++++--
 2 files changed, 72 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5867d545/sail/src/main/java/mvm/rya/rdftriplestore/inference/InverseURI.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/mvm/rya/rdftriplestore/inference/InverseURI.java b/sail/src/main/java/mvm/rya/rdftriplestore/inference/InverseURI.java
index c182fb5..c9e14cd 100644
--- a/sail/src/main/java/mvm/rya/rdftriplestore/inference/InverseURI.java
+++ b/sail/src/main/java/mvm/rya/rdftriplestore/inference/InverseURI.java
@@ -1,8 +1,42 @@
 package mvm.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 org.openrdf.model.URI;
 import org.openrdf.model.impl.URIImpl;
-
+/*
+ * 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.
+ */
 public class InverseURI implements URI {
 
 	private URI impl;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5867d545/sail/src/test/java/mvm/rya/triplestore/inference/PropertyChainTest.java
----------------------------------------------------------------------
diff --git a/sail/src/test/java/mvm/rya/triplestore/inference/PropertyChainTest.java b/sail/src/test/java/mvm/rya/triplestore/inference/PropertyChainTest.java
index 8592e0a..3aab403 100644
--- a/sail/src/test/java/mvm/rya/triplestore/inference/PropertyChainTest.java
+++ b/sail/src/test/java/mvm/rya/triplestore/inference/PropertyChainTest.java
@@ -1,5 +1,22 @@
 package mvm.rya.triplestore.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.List;
 
 import org.apache.accumulo.core.Constants;
@@ -25,7 +42,24 @@ import mvm.rya.api.RdfCloudTripleStoreConstants;
 import mvm.rya.rdftriplestore.RdfCloudTripleStore;
 import mvm.rya.rdftriplestore.inference.InferenceEngine;
 import mvm.rya.rdftriplestore.inference.InverseURI;
-
+/*
+ * 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.
+ */
 public class PropertyChainTest extends TestCase {
     private String user = "user";
     private String pwd = "pwd";