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 2018/05/16 15:48:56 UTC

[1/3] incubator-rya git commit: RYA-492 Added language support for Literals; Closes #294

Repository: incubator-rya
Updated Branches:
  refs/heads/master 2c1efd225 -> 2396ebb87


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
index b50fddb..55f3fa1 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
@@ -26,6 +26,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.resolver.RdfToRyaConversions;
 import org.apache.rya.indexing.GeoConstants;
 import org.apache.rya.indexing.IndexingExpr;
@@ -428,10 +429,12 @@ public class GeoTemporalMongoDBStorageStrategyTest {
         Value object = VF.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
 
         Statement statement = VF.createStatement(subject, predicate, object, context);
-        DBObject actual = adapter.serialize(RdfToRyaConversions.convertStatement(statement));
+        RyaStatement ryaStatement = RdfToRyaConversions.convertStatement(statement);
+        int expectedId = ryaStatement.getSubject().hashCode();
+        DBObject actual = adapter.serialize(ryaStatement);
         String expectedString =
             "{ "
-            + "\"_id\" : -852305321 , "
+            + "\"_id\" : " + expectedId + ", "
             + "\"location\" : { "
               + "\"coordinates\" : [ -77.03524 , 38.889468] , "
               + "\"type\" : \"Point\""
@@ -444,10 +447,12 @@ public class GeoTemporalMongoDBStorageStrategyTest {
         predicate = VF.createIRI("Property:event:time");
         object = VF.createLiteral("2015-12-30T12:00:00Z");
         statement = VF.createStatement(subject, predicate, object, context);
-        actual = adapter.serialize(RdfToRyaConversions.convertStatement(statement));
+        ryaStatement = RdfToRyaConversions.convertStatement(statement);
+        expectedId = ryaStatement.getSubject().hashCode();
+        actual = adapter.serialize(ryaStatement);
         expectedString =
                 "{"
-                  +"_id : -852305321, "
+                  +"_id : " + expectedId + ", "
                   +"time: {"
                     + "instant : {"
                       +"\"$date\" : \"2015-12-30T12:00:00.000Z\""
@@ -461,10 +466,12 @@ public class GeoTemporalMongoDBStorageStrategyTest {
         predicate = VF.createIRI("Property:circa");
         object = VF.createLiteral("[1969-12-31T19:00:00-05:00,1969-12-31T19:00:01-05:00]");
         statement = VF.createStatement(subject, predicate, object, context);
-        actual = adapter.serialize(RdfToRyaConversions.convertStatement(statement));
+        ryaStatement = RdfToRyaConversions.convertStatement(statement);
+        expectedId = ryaStatement.getSubject().hashCode();
+        actual = adapter.serialize(ryaStatement);
         expectedString =
                 "{"
-                +"_id : -852305321, "
+                +"_id : " + expectedId + ", "
                 +"time: {"
                   + "start : {"
                     +"\"$date\" : \"1970-01-01T00:00:00.000Z\""

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/rya.reasoning/src/main/java/org/apache/rya/reasoning/Fact.java
----------------------------------------------------------------------
diff --git a/extras/rya.reasoning/src/main/java/org/apache/rya/reasoning/Fact.java b/extras/rya.reasoning/src/main/java/org/apache/rya/reasoning/Fact.java
index ad0ba14..2873761 100644
--- a/extras/rya.reasoning/src/main/java/org/apache/rya/reasoning/Fact.java
+++ b/extras/rya.reasoning/src/main/java/org/apache/rya/reasoning/Fact.java
@@ -27,6 +27,7 @@ import java.nio.charset.StandardCharsets;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.resolver.RyaToRdfConversions;
+import org.apache.rya.api.utils.LiteralLanguageUtils;
 import org.eclipse.rdf4j.model.IRI;
 import org.eclipse.rdf4j.model.Literal;
 import org.eclipse.rdf4j.model.Resource;
@@ -63,14 +64,14 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
     /**
      * A fact containing a triple and no generating rule.
      */
-    public Fact(Statement stmt) {
+    public Fact(final Statement stmt) {
         this.triple = stmt;
     }
 
     /**
      * A fact containing a triple and no generating rule.
      */
-    public Fact(Resource s, IRI p, Value o) {
+    public Fact(final Resource s, final IRI p, final Value o) {
         this.triple = VF.createStatement(s, p, o);
     }
 
@@ -78,8 +79,8 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
      * A fact which contains a triple and was generated using a
      * particular rule by a reasoner for a particular node.
      */
-    public Fact(Resource s, IRI p, Value o, int iteration,
-            OwlRule rule, Resource node) {
+    public Fact(final Resource s, final IRI p, final Value o, final int iteration,
+            final OwlRule rule, final Resource node) {
         this.triple = VF.createStatement(s, p, o);
         this.derivation = new Derivation(iteration, rule, node);
     }
@@ -142,14 +143,14 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
     /**
      * Assign a particular statement to this fact.
      */
-    public void setTriple(Statement stmt) {
+    public void setTriple(final Statement stmt) {
         triple = stmt;
     }
 
     /**
      * Assign a particular statement to this fact.
      */
-    public void setTriple(RyaStatement rs) {
+    public void setTriple(final RyaStatement rs) {
         setTriple(RyaToRdfConversions.convertStatement(rs));
     }
 
@@ -157,14 +158,14 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
      * Set a flag if this triple *could* be used in future derivations
      * (may only actually happen if certain other facts are seen as well.)
      */
-    public void setUseful(boolean useful) {
+    public void setUseful(final boolean useful) {
         this.useful = useful;
     }
 
     /**
      * Set derivation. Allows reconstructing a fact and the way it was produced.
      */
-    public void setDerivation(Derivation d) {
+    public void setDerivation(final Derivation d) {
         this.derivation = d;
     }
 
@@ -173,7 +174,7 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
      * of the fact from the way it was produced.
      */
     public Derivation unsetDerivation() {
-        Derivation d = getDerivation();
+        final Derivation d = getDerivation();
         this.derivation = null;
         return d;
     }
@@ -183,7 +184,7 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
      * @param   multiline    Print a multi-line tree as opposed to a nested list
      * @param   schema       Use schema knowledge to further explain BNodes
      */
-    public String explain(boolean multiline, Schema schema) {
+    public String explain(final boolean multiline, final Schema schema) {
         return explain(multiline, "", schema);
     }
 
@@ -192,15 +193,15 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
      * schema information.
      * @param   multiline    Print a multi-line tree as opposed to a nested list
      */
-    public String explain(boolean multiline) {
+    public String explain(final boolean multiline) {
         return explain(multiline, "", null);
     }
 
     /**
      * Recursively generate a String to show this fact's derivation.
      */
-    String explain(boolean multiline, String prefix, Schema schema) {
-        StringBuilder sb = new StringBuilder();
+    String explain(final boolean multiline, final String prefix, final Schema schema) {
+        final StringBuilder sb = new StringBuilder();
         String sep = " ";
         if (multiline) {
             sep = "\n" + prefix;
@@ -209,15 +210,15 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
             sb.append("(empty)").append(sep);
         }
         else {
-            Resource s = getSubject();
-            IRI p = getPredicate();
-            Value o = getObject();
+            final Resource s = getSubject();
+            final IRI p = getPredicate();
+            final Value o = getObject();
             sb.append("<").append(s.toString()).append(">").append(sep);
             sb.append("<").append(p.toString()).append(">").append(sep);
             sb.append("<").append(o.toString()).append(">");
             // Restrictions warrant further explanation
             if (schema != null && p.equals(RDF.TYPE)) {
-                Resource objClass = (Resource) o;
+                final Resource objClass = (Resource) o;
                 if (schema.hasRestriction(objClass)) {
                     sb.append(" { ");
                     sb.append(schema.explainRestriction(objClass));
@@ -240,7 +241,7 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
      */
     @Override
     public String toString() {
-        StringBuilder sb = new StringBuilder();
+        final StringBuilder sb = new StringBuilder();
         if (triple != null) {
             sb.append("<").append(getSubject().toString()).append("> ");
             sb.append("<").append(getPredicate().toString()).append("> ");
@@ -255,25 +256,25 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
     }
 
     @Override
-    public void write(DataOutput out) throws IOException {
+    public void write(final DataOutput out) throws IOException {
         if (triple == null) {
             out.writeInt(0);
         }
         else {
-            StringBuilder sb = new StringBuilder();
+            final StringBuilder sb = new StringBuilder();
             if (triple.getContext() != null) {
                 sb.append(triple.getContext().toString());
             }
             sb.append(SEP).append(getSubject().toString());
             sb.append(SEP).append(getPredicate().toString());
             sb.append(SEP).append(getObject().toString());
-            byte[] encoded = sb.toString().getBytes(StandardCharsets.UTF_8);
+            final byte[] encoded = sb.toString().getBytes(StandardCharsets.UTF_8);
             out.writeInt(encoded.length);
             out.write(encoded);
         }
         out.writeBoolean(useful);
         // Write the derivation if there is one
-        boolean derived = isInference();
+        final boolean derived = isInference();
         out.writeBoolean(derived);
         if (derived) {
             derivation.write(out);
@@ -281,21 +282,21 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
     }
 
     @Override
-    public void readFields(DataInput in) throws IOException {
+    public void readFields(final DataInput in) throws IOException {
         derivation = null;
-        int tripleLength = in.readInt();
+        final int tripleLength = in.readInt();
         if (tripleLength == 0) {
             triple = null;
         }
         else {
-            byte[] tripleBytes = new byte[tripleLength];
+            final byte[] tripleBytes = new byte[tripleLength];
             in.readFully(tripleBytes);
-            String tripleString = new String(tripleBytes, StandardCharsets.UTF_8);
-            String[] parts = tripleString.split(SEP);
-            ValueFactory factory = SimpleValueFactory.getInstance();
-            String context = parts[0];
+            final String tripleString = new String(tripleBytes, StandardCharsets.UTF_8);
+            final String[] parts = tripleString.split(SEP);
+            final ValueFactory factory = SimpleValueFactory.getInstance();
+            final String context = parts[0];
             Resource s = null;
-            IRI p = factory.createIRI(parts[2]);
+            final IRI p = factory.createIRI(parts[2]);
             Value o = null;
             // Subject: either bnode or URI
             if (parts[1].startsWith("_")) {
@@ -310,17 +311,17 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
             }
             else if (parts[3].startsWith("\"")) {
                 //literal: may have language or datatype
-                int close = parts[3].lastIndexOf("\"");
-                int length = parts[3].length();
-                String label = parts[3].substring(1, close);
+                final int close = parts[3].lastIndexOf("\"");
+                final int length = parts[3].length();
+                final String label = parts[3].substring(1, close);
                 if (close == length - 1) {
                     // Just a string enclosed in quotes
                     o = factory.createLiteral(label);
                 }
                 else {
-                    String data = parts[3].substring(close + 1);
-                    if (data.startsWith("@")) {
-                        String lang = data.substring(1);
+                    final String data = parts[3].substring(close + 1);
+                    if (data.startsWith(LiteralLanguageUtils.LANGUAGE_DELIMITER)) {
+                        final String lang = data.substring(1);
                         o = factory.createLiteral(label, lang);
                     }
                     else if (data.startsWith("^^<")) {
@@ -354,7 +355,7 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
      * (represent no triple), compare their derivations instead.
      */
     @Override
-    public int compareTo(Fact other) {
+    public int compareTo(final Fact other) {
         if (this.equals(other)) {
             return 0;
         }
@@ -395,14 +396,14 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
      * compare their derivations.
      */
     @Override
-    public boolean equals(Object o) {
+    public boolean equals(final Object o) {
         if (this == o) {
             return true;
         }
         if (o == null || this.getClass() != o.getClass()) {
             return false;
         }
-        Fact other = (Fact) o;
+        final Fact other = (Fact) o;
         if (this.triple == null) {
             if (other.triple == null) {
                 // Derivations only matter if both facts are empty
@@ -433,7 +434,7 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
 
     @Override
     public Fact clone() {
-        Fact other = new Fact();
+        final Fact other = new Fact();
         other.triple = this.triple;
         other.useful = this.useful;
         if (this.derivation != null) {
@@ -446,7 +447,7 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
      * Specify a source. Wrapper for Derivation.addSource. Instantiates a
      * derivation if none exists.
      */
-    public void addSource(Fact other) {
+    public void addSource(final Fact other) {
         if (derivation == null) {
             derivation = new Derivation();
         }
@@ -477,14 +478,14 @@ public class Fact implements WritableComparable<Fact>, Cloneable {
      * Return whether a particular fact is identical to one used to derive this.
      * Wrapper for Derivation.hasSource.
      */
-    public boolean hasSource(Fact other) {
+    public boolean hasSource(final Fact other) {
         return derivation != null && derivation.hasSource(other);
     }
 
     /**
      * Return whether this fact was derived using a particular rule.
      */
-    public boolean hasRule(OwlRule rule) {
+    public boolean hasRule(final OwlRule rule) {
         return derivation != null && derivation.getRule() == rule;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/mapreduce/src/main/java/org/apache/rya/accumulo/mr/RyaTypeWritable.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/rya/accumulo/mr/RyaTypeWritable.java b/mapreduce/src/main/java/org/apache/rya/accumulo/mr/RyaTypeWritable.java
index be90180..3050b61 100644
--- a/mapreduce/src/main/java/org/apache/rya/accumulo/mr/RyaTypeWritable.java
+++ b/mapreduce/src/main/java/org/apache/rya/accumulo/mr/RyaTypeWritable.java
@@ -25,6 +25,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.rya.api.domain.RyaType;
+import org.apache.rya.api.utils.LiteralLanguageUtils;
 import org.eclipse.rdf4j.model.IRI;
 import org.eclipse.rdf4j.model.impl.SimpleValueFactory;
 
@@ -38,10 +39,10 @@ public class RyaTypeWritable implements WritableComparable<RyaTypeWritable>{
      * @return The next individual field, as a byte array.
      * @throws IOException if reading from the stream fails.
      */
-    protected byte[] read(DataInput dataInput) throws IOException {
+    protected byte[] read(final DataInput dataInput) throws IOException {
         if (dataInput.readBoolean()) {
-            int len = dataInput.readInt();
-            byte[] bytes = new byte[len];
+            final int len = dataInput.readInt();
+            final byte[] bytes = new byte[len];
             dataInput.readFully(bytes);
             return bytes;
         }else {
@@ -50,19 +51,24 @@ public class RyaTypeWritable implements WritableComparable<RyaTypeWritable>{
     }
 
     @Override
-    public void readFields(DataInput dataInput) throws IOException {
-        SimpleValueFactory vfi = SimpleValueFactory.getInstance();
-        String data = dataInput.readLine();
-        String dataTypeString = dataInput.readLine();
-        IRI dataType = vfi.createIRI(dataTypeString);
+    public void readFields(final DataInput dataInput) throws IOException {
+        final SimpleValueFactory vfi = SimpleValueFactory.getInstance();
+        final String data = dataInput.readLine();
+        final String dataTypeString = dataInput.readLine();
+        final String language = dataInput.readLine();
+        final IRI dataType = vfi.createIRI(dataTypeString);
+        final String validatedLanguage = LiteralLanguageUtils.validateLanguage(language, dataType);
         ryatype.setData(data);
         ryatype.setDataType(dataType);
+        ryatype.setLanguage(validatedLanguage);
+
     }
 
     @Override
-    public void write(DataOutput dataOutput) throws IOException {
+    public void write(final DataOutput dataOutput) throws IOException {
         dataOutput.writeChars(ryatype.getData());
         dataOutput.writeChars(ryatype.getDataType().toString());
+        dataOutput.writeChars(ryatype.getLanguage());
     }
 
     /**
@@ -77,12 +83,12 @@ public class RyaTypeWritable implements WritableComparable<RyaTypeWritable>{
      * @param   ryaStatement    The statement to be represented by this
      *                          RyaStatementWritable.
      */
-    public void setRyaType(RyaType ryatype) {
+    public void setRyaType(final RyaType ryatype) {
         this.ryatype = ryatype;
     }
 
     @Override
-    public int compareTo(RyaTypeWritable o) {
+    public int compareTo(final RyaTypeWritable o) {
         return ryatype.compareTo(o.ryatype);
     }
 
@@ -93,15 +99,15 @@ public class RyaTypeWritable implements WritableComparable<RyaTypeWritable>{
      *          RyaTypes.
      */
     @Override
-    public boolean equals(Object o) {
+    public boolean equals(final Object o) {
         if (o == this) {
             return true;
         }
         if (o == null || !(o instanceof RyaTypeWritable)) {
             return false;
         }
-        RyaType rtThis = ryatype;
-        RyaType rtOther = ((RyaTypeWritable) o).ryatype;
+        final RyaType rtThis = ryatype;
+        final RyaType rtOther = ((RyaTypeWritable) o).ryatype;
         if (rtThis == null) {
             return rtOther == null;
         }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/mapreduce/src/test/java/org/apache/rya/accumulo/mr/GraphXInputFormatTest.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/java/org/apache/rya/accumulo/mr/GraphXInputFormatTest.java b/mapreduce/src/test/java/org/apache/rya/accumulo/mr/GraphXInputFormatTest.java
index 93df522..9012cac 100644
--- a/mapreduce/src/test/java/org/apache/rya/accumulo/mr/GraphXInputFormatTest.java
+++ b/mapreduce/src/test/java/org/apache/rya/accumulo/mr/GraphXInputFormatTest.java
@@ -20,13 +20,6 @@ package org.apache.rya.accumulo.mr;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.rya.accumulo.AccumuloRdfConfiguration;
-import org.apache.rya.accumulo.AccumuloRyaDAO;
-import org.apache.rya.accumulo.mr.GraphXInputFormat.RyaStatementRecordReader;
-import org.apache.rya.api.domain.RyaStatement;
-import org.apache.rya.api.domain.RyaType;
-import org.apache.rya.api.domain.RyaIRI;
-
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.mock.MockInstance;
@@ -40,6 +33,13 @@ import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.task.JobContextImpl;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.rya.accumulo.AccumuloRdfConfiguration;
+import org.apache.rya.accumulo.AccumuloRyaDAO;
+import org.apache.rya.accumulo.mr.GraphXInputFormat.RyaStatementRecordReader;
+import org.apache.rya.api.domain.RyaIRI;
+import org.apache.rya.api.domain.RyaStatement;
+import org.apache.rya.api.domain.RyaType;
+import org.apache.rya.api.utils.LiteralLanguageUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -47,8 +47,8 @@ import org.junit.Test;
 
 public class GraphXInputFormatTest {
 
-    private String username = "root", table = "rya_eci";
-    private PasswordToken password = new PasswordToken("");
+    private final String username = "root", table = "rya_eci";
+    private final PasswordToken password = new PasswordToken("");
 
     private Instance instance;
     private AccumuloRyaDAO apiImpl;
@@ -56,10 +56,10 @@ public class GraphXInputFormatTest {
     @Before
     public void init() throws Exception {
         instance = new MockInstance(GraphXInputFormatTest.class.getName() + ".mock_instance");
-        Connector connector = instance.getConnector(username, password);
+        final Connector connector = instance.getConnector(username, password);
         connector.tableOperations().create(table);
 
-        AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
+        final AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
         conf.setTablePrefix("rya_");
         conf.setDisplayQueryPlan(false);
         conf.setBoolean("sc.use_entity", true);
@@ -77,7 +77,7 @@ public class GraphXInputFormatTest {
 
     @Test
     public void testInputFormat() throws Exception {
-        RyaStatement input = RyaStatement.builder()
+        final RyaStatement input = RyaStatement.builder()
             .setSubject(new RyaIRI("http://www.google.com"))
             .setPredicate(new RyaIRI("http://some_other_uri"))
             .setObject(new RyaIRI("http://www.yahoo.com"))
@@ -87,7 +87,7 @@ public class GraphXInputFormatTest {
 
         apiImpl.add(input);
 
-        Job jobConf = Job.getInstance();
+        final Job jobConf = Job.getInstance();
 
         GraphXInputFormat.setMockInstance(jobConf, instance.getInstanceName());
         GraphXInputFormat.setConnectorInfo(jobConf, username, password);
@@ -98,31 +98,33 @@ public class GraphXInputFormatTest {
         GraphXInputFormat.setLocalIterators(jobConf, false);
         GraphXInputFormat.setOfflineTableScan(jobConf, false);
 
-        GraphXInputFormat inputFormat = new GraphXInputFormat();
+        final GraphXInputFormat inputFormat = new GraphXInputFormat();
 
-        JobContext context = new JobContextImpl(jobConf.getConfiguration(), jobConf.getJobID());
+        final JobContext context = new JobContextImpl(jobConf.getConfiguration(), jobConf.getJobID());
 
-        List<InputSplit> splits = inputFormat.getSplits(context);
+        final List<InputSplit> splits = inputFormat.getSplits(context);
 
         Assert.assertEquals(1, splits.size());
 
-        TaskAttemptContext taskAttemptContext = new TaskAttemptContextImpl(context.getConfiguration(), new TaskAttemptID(new TaskID(), 1));
+        final TaskAttemptContext taskAttemptContext = new TaskAttemptContextImpl(context.getConfiguration(), new TaskAttemptID(new TaskID(), 1));
 
-        RecordReader<Object, RyaTypeWritable> reader = inputFormat.createRecordReader(splits.get(0), taskAttemptContext);
+        final RecordReader<Object, RyaTypeWritable> reader = inputFormat.createRecordReader(splits.get(0), taskAttemptContext);
 
-        RyaStatementRecordReader ryaStatementRecordReader = (RyaStatementRecordReader)reader;
+        final RyaStatementRecordReader ryaStatementRecordReader = (RyaStatementRecordReader)reader;
         ryaStatementRecordReader.initialize(splits.get(0), taskAttemptContext);
 
-        List<RyaType> results = new ArrayList<RyaType>();
+        final List<RyaType> results = new ArrayList<RyaType>();
         System.out.println("before while");
         while(ryaStatementRecordReader.nextKeyValue()) {
             System.out.println("in while");
-            RyaTypeWritable writable = ryaStatementRecordReader.getCurrentValue();
-            RyaType value = writable.getRyaType();
-            Object text = ryaStatementRecordReader.getCurrentKey();
-            RyaType type = new RyaType();
+            final RyaTypeWritable writable = ryaStatementRecordReader.getCurrentValue();
+            final RyaType value = writable.getRyaType();
+            final Object text = ryaStatementRecordReader.getCurrentKey();
+            final RyaType type = new RyaType();
+            final String validatedLanguage = LiteralLanguageUtils.validateLanguage(value.getLanguage(), value.getDataType());
             type.setData(value.getData());
             type.setDataType(value.getDataType());
+            type.setLanguage(validatedLanguage);
             results.add(type);
 
             System.out.println(value.getData());

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/pig/accumulo.pig/src/test/java/org/apache/rya/accumulo/pig/StatementPatternStorageTest.java
----------------------------------------------------------------------
diff --git a/pig/accumulo.pig/src/test/java/org/apache/rya/accumulo/pig/StatementPatternStorageTest.java b/pig/accumulo.pig/src/test/java/org/apache/rya/accumulo/pig/StatementPatternStorageTest.java
index 942510b..2b1c652 100644
--- a/pig/accumulo.pig/src/test/java/org/apache/rya/accumulo/pig/StatementPatternStorageTest.java
+++ b/pig/accumulo.pig/src/test/java/org/apache/rya/accumulo/pig/StatementPatternStorageTest.java
@@ -8,9 +8,9 @@ package org.apache.rya.accumulo.pig;
  * 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
@@ -20,13 +20,14 @@ package org.apache.rya.accumulo.pig;
  */
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 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.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.hadoop.conf.Configuration;
@@ -36,14 +37,16 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.apache.pig.data.Tuple;
 import org.apache.rya.accumulo.AccumuloRdfConfiguration;
 import org.apache.rya.accumulo.AccumuloRyaDAO;
 import org.apache.rya.api.RdfCloudTripleStoreConstants;
+import org.apache.rya.api.domain.RyaIRI;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaType;
-import org.apache.rya.api.domain.RyaIRI;
+import org.eclipse.rdf4j.model.vocabulary.RDF;
 
 import junit.framework.TestCase;
 
@@ -55,26 +58,27 @@ import junit.framework.TestCase;
  */
 public class StatementPatternStorageTest 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 final String user = "user";
+    private final String pwd = "pwd";
+    private final String instance = "myinstance";
+    private final String tablePrefix = "t_";
+    private final Authorizations auths = Authorizations.EMPTY;
     private Connector connector;
     private AccumuloRyaDAO ryaDAO;
-    private String namespace = "urn:test#";
+    private final String namespace = "urn:test#";
     private AccumuloRdfConfiguration conf;
 
     @Override
     public void setUp() throws Exception {
         super.setUp();
-        connector = new MockInstance(instance).getConnector(user, pwd.getBytes());
+        connector = new MockInstance(instance).getConnector(user, new PasswordToken(pwd.getBytes(StandardCharsets.UTF_8)));
         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);
+        final SecurityOperations secOps = connector.securityOperations();
+        secOps.createLocalUser(user, new PasswordToken(pwd.getBytes(StandardCharsets.UTF_8)));
+        secOps.changeUserAuthorizations(user, 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);
@@ -101,73 +105,75 @@ public class StatementPatternStorageTest extends TestCase {
         ryaDAO.add(new RyaStatement(new RyaIRI(namespace, "a"),new RyaIRI(namespace,"p"), new RyaType("l")));
         ryaDAO.add(new RyaStatement(new RyaIRI(namespace, "b"), new RyaIRI(namespace, "p"), new RyaType("l")));
         ryaDAO.add(new RyaStatement(new RyaIRI(namespace, "c"), new RyaIRI(namespace, "n"), new RyaType("l")));
-        
+        ryaDAO.add(new RyaStatement(new RyaIRI(namespace, "d"), new RyaIRI(namespace, "p"), new RyaType(RDF.LANGSTRING, "l", "en-US")));
+
 
         int count = 0;
-        List<StatementPatternStorage> storages = createStorages("accumulo://" + tablePrefix + "?instance=" + instance + "&user=" + user + "&password=" + pwd + "&predicate=<" + namespace + "p>&mock=true");
-        for (StatementPatternStorage storage : storages) {
+        final List<StatementPatternStorage> storages = createStorages("accumulo://" + tablePrefix + "?instance=" + instance + "&user=" + user + "&password=" + pwd + "&predicate=<" + namespace + "p>&mock=true");
+        for (final StatementPatternStorage storage : storages) {
             while (true) {
-                Tuple next = storage.getNext();
+                final Tuple next = storage.getNext();
                 if (next == null) {
                     break;
                 }
                 count++;
             }
         }
-        assertEquals(2, count);
+        assertEquals(3, count);
         ryaDAO.destroy();
     }
 
     public void testContext() throws Exception {
         ryaDAO.add(new RyaStatement(new RyaIRI(namespace, "a"), new RyaIRI(namespace, "p"), new RyaType("l1")));
         ryaDAO.add(new RyaStatement(new RyaIRI(namespace, "a"), new RyaIRI(namespace, "p"), new RyaType("l2"), new RyaIRI(namespace, "g1")));
-        
+        ryaDAO.add(new RyaStatement(new RyaIRI(namespace, "a"), new RyaIRI(namespace, "p"), new RyaType(RDF.LANGSTRING, "l1", "en-US"), new RyaIRI(namespace, "g1")));
+
 
         int count = 0;
         List<StatementPatternStorage> storages = createStorages("accumulo://" + tablePrefix + "?instance=" + instance + "&user=" + user + "&password=" + pwd + "&predicate=<" + namespace + "p>&mock=true");
-        for (StatementPatternStorage storage : storages) {
+        for (final StatementPatternStorage storage : storages) {
             while (true) {
-                Tuple next = storage.getNext();
+                final Tuple next = storage.getNext();
                 if (next == null) {
                     break;
                 }
                 count++;
             }
         }
-        assertEquals(2, count);
+        assertEquals(3, count);
 
         count = 0;
         storages = createStorages("accumulo://" + tablePrefix + "?instance=" + instance + "&user=" + user + "&password=" + pwd + "&predicate=<" + namespace + "p>&context=<"+namespace+"g1>&mock=true");
-        for (StatementPatternStorage storage : storages) {
+        for (final StatementPatternStorage storage : storages) {
             while (true) {
-                Tuple next = storage.getNext();
+                final Tuple next = storage.getNext();
                 if (next == null) {
                     break;
                 }
                 count++;
             }
         }
-        assertEquals(1, count);
+        assertEquals(2, count);
 
         ryaDAO.destroy();
     }
 
-    protected List<StatementPatternStorage> createStorages(String location) throws IOException, InterruptedException {
-        List<StatementPatternStorage> storages = new ArrayList<StatementPatternStorage>();
+    protected List<StatementPatternStorage> createStorages(final String location) throws IOException, InterruptedException {
+        final List<StatementPatternStorage> storages = new ArrayList<StatementPatternStorage>();
         StatementPatternStorage storage = new StatementPatternStorage();
-        InputFormat inputFormat = storage.getInputFormat();
-        Job job = new Job(new Configuration());
+        final InputFormat<?, ?> inputFormat = storage.getInputFormat();
+        Job job = Job.getInstance(new Configuration());
         storage.setLocation(location, job);
-        List<InputSplit> splits = inputFormat.getSplits(job);
+        final List<InputSplit> splits = inputFormat.getSplits(job);
         assertNotNull(splits);
 
-        for (InputSplit inputSplit : splits) {
+        for (final InputSplit inputSplit : splits) {
             storage = new StatementPatternStorage();
-            job = new Job(new Configuration());
+            job = Job.getInstance(new Configuration());
             storage.setLocation(location, job);
-            TaskAttemptContext taskAttemptContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                    new TaskAttemptID("jtid", 0, false, 0, 0));
-            RecordReader recordReader = inputFormat.createRecordReader(inputSplit,
+            final TaskAttemptContext taskAttemptContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                    new TaskAttemptID("jtid", 0, TaskType.REDUCE, 0, 0));
+            final RecordReader<?, ?> recordReader = inputFormat.createRecordReader(inputSplit,
                     taskAttemptContext);
             recordReader.initialize(inputSplit, taskAttemptContext);
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/sail/src/main/java/org/apache/rya/rdftriplestore/RyaSailRepositoryConnection.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/org/apache/rya/rdftriplestore/RyaSailRepositoryConnection.java b/sail/src/main/java/org/apache/rya/rdftriplestore/RyaSailRepositoryConnection.java
index 145d2d0..8b26ff4 100644
--- a/sail/src/main/java/org/apache/rya/rdftriplestore/RyaSailRepositoryConnection.java
+++ b/sail/src/main/java/org/apache/rya/rdftriplestore/RyaSailRepositoryConnection.java
@@ -8,9 +8,9 @@ package org.apache.rya.rdftriplestore;
  * 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
@@ -30,9 +30,11 @@ import org.eclipse.rdf4j.repository.RepositoryException;
 import org.eclipse.rdf4j.repository.sail.SailRepository;
 import org.eclipse.rdf4j.repository.sail.SailRepositoryConnection;
 import org.eclipse.rdf4j.repository.util.RDFLoader;
+import org.eclipse.rdf4j.rio.ParserConfig;
 import org.eclipse.rdf4j.rio.RDFFormat;
 import org.eclipse.rdf4j.rio.RDFHandlerException;
 import org.eclipse.rdf4j.rio.RDFParseException;
+import org.eclipse.rdf4j.rio.helpers.BasicParserSettings;
 import org.eclipse.rdf4j.sail.SailConnection;
 
 /**
@@ -40,65 +42,60 @@ import org.eclipse.rdf4j.sail.SailConnection;
  */
 public class RyaSailRepositoryConnection extends SailRepositoryConnection {
 
-    protected RyaSailRepositoryConnection(SailRepository repository, SailConnection sailConnection) {
+    protected RyaSailRepositoryConnection(final SailRepository repository, final SailConnection sailConnection) {
         super(repository, sailConnection);
     }
 
     @Override
-    public void add(InputStream in, String baseURI, RDFFormat dataFormat, Resource... contexts) throws IOException, RDFParseException,
+    public ParserConfig getParserConfig() {
+        final ParserConfig parserConfig = super.getParserConfig();
+        parserConfig.set(BasicParserSettings.VERIFY_URI_SYNTAX, false);
+        return parserConfig;
+    }
+
+    @Override
+    public void add(final InputStream in, final String baseURI, final RDFFormat dataFormat, final Resource... contexts) throws IOException, RDFParseException,
             RepositoryException {
         OpenRDFUtil.verifyContextNotNull(contexts);
 
-        CombineContextsRdfInserter rdfInserter = new CombineContextsRdfInserter(this);
+        final CombineContextsRdfInserter rdfInserter = new CombineContextsRdfInserter(this);
         rdfInserter.enforceContext(contexts);
 
-        boolean localTransaction = startLocalTransaction();
+        final boolean localTransaction = startLocalTransaction();
         try {
-            RDFLoader loader = new RDFLoader(getParserConfig(), getValueFactory());
+            final RDFLoader loader = new RDFLoader(getParserConfig(), getValueFactory());
             loader.load(in, baseURI, dataFormat, rdfInserter);
 
             conditionalCommit(localTransaction);
-        } catch (RDFHandlerException e) {
+        } catch (final RDFHandlerException e) {
             conditionalRollback(localTransaction);
 
             throw ((RepositoryException) e.getCause());
-        } catch (RDFParseException e) {
-            conditionalRollback(localTransaction);
-            throw e;
-        } catch (IOException e) {
-            conditionalRollback(localTransaction);
-            throw e;
-        } catch (RuntimeException e) {
+        } catch (final IOException | RuntimeException e) {
             conditionalRollback(localTransaction);
             throw e;
         }
     }
 
     @Override
-    public void add(Reader reader, String baseURI, RDFFormat dataFormat, Resource... contexts) throws IOException, RDFParseException,
+    public void add(final Reader reader, final String baseURI, final RDFFormat dataFormat, final Resource... contexts) throws IOException, RDFParseException,
             RepositoryException {
         OpenRDFUtil.verifyContextNotNull(contexts);
 
-        CombineContextsRdfInserter rdfInserter = new CombineContextsRdfInserter(this);
+        final CombineContextsRdfInserter rdfInserter = new CombineContextsRdfInserter(this);
         rdfInserter.enforceContext(contexts);
 
-        boolean localTransaction = startLocalTransaction();
+        final boolean localTransaction = startLocalTransaction();
         try {
-            RDFLoader loader = new RDFLoader(getParserConfig(), getValueFactory());
+            final RDFLoader loader = new RDFLoader(getParserConfig(), getValueFactory());
             loader.load(reader, baseURI, dataFormat, rdfInserter);
 
             conditionalCommit(localTransaction);
-        } catch (RDFHandlerException e) {
+        } catch (final RDFHandlerException e) {
             conditionalRollback(localTransaction);
 
             throw ((RepositoryException) e.getCause());
-        } catch (RDFParseException e) {
-            conditionalRollback(localTransaction);
-            throw e;
-        } catch (IOException e) {
-            conditionalRollback(localTransaction);
-            throw e;
-        } catch (RuntimeException e) {
+        } catch (final IOException | RuntimeException e) {
             conditionalRollback(localTransaction);
             throw e;
         }



[2/3] incubator-rya git commit: RYA-492 Added language support for Literals; Closes #294

Posted by pu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java
index 35525ad..6996fcd 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/aggregation/AggregationPipelineQueryNode.java
@@ -22,6 +22,7 @@ import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.CONTEXT;
 import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
 import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT;
 import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_HASH;
+import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_LANGUAGE;
 import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.OBJECT_TYPE;
 import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE;
 import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.PREDICATE_HASH;
@@ -42,9 +43,9 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.function.Function;
 
+import org.apache.rya.api.domain.RyaIRI;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaType;
-import org.apache.rya.api.domain.RyaIRI;
 import org.apache.rya.api.domain.StatementMetadata;
 import org.apache.rya.api.resolver.RdfToRyaConversions;
 import org.apache.rya.mongodb.MongoDbRdfConstants;
@@ -127,7 +128,7 @@ public class AggregationPipelineQueryNode extends ExternalSet {
     private static final Bson DEFAULT_METADATA = new Document("$literal",
             StatementMetadata.EMPTY_METADATA.toString());
 
-    private static boolean isValidFieldName(String name) {
+    private static boolean isValidFieldName(final String name) {
         return !(name == null || name.contains(".") || name.contains("$")
                 || name.equals("_id"));
     }
@@ -144,13 +145,13 @@ public class AggregationPipelineQueryNode extends ExternalSet {
         private final Map<String, String> varToTripleType = new HashMap<>();
         private final BiMap<String, String> varToOriginalName;
 
-        String valueField(String varName) {
+        String valueField(final String varName) {
             return varToTripleValue.get(varName);
         }
-        String hashField(String varName) {
+        String hashField(final String varName) {
             return varToTripleHash.get(varName);
         }
-        String typeField(String varName) {
+        String typeField(final String varName) {
             return varToTripleType.get(varName);
         }
 
@@ -158,18 +159,18 @@ public class AggregationPipelineQueryNode extends ExternalSet {
             return varToTripleValue.keySet();
         }
 
-        private String replace(String original) {
+        private String replace(final String original) {
             if (varToOriginalName.containsValue(original)) {
                 return varToOriginalName.inverse().get(original);
             }
             else {
-                String replacement = "field-" + UUID.randomUUID();
+                final String replacement = "field-" + UUID.randomUUID();
                 varToOriginalName.put(replacement, original);
                 return replacement;
             }
         }
 
-        private String sanitize(String name) {
+        private String sanitize(final String name) {
             if (varToOriginalName.containsValue(name)) {
                 return varToOriginalName.inverse().get(name);
             }
@@ -179,26 +180,27 @@ public class AggregationPipelineQueryNode extends ExternalSet {
             return name;
         }
 
-        StatementVarMapping(StatementPattern sp, BiMap<String, String> varToOriginalName) {
+        StatementVarMapping(final StatementPattern sp, final BiMap<String, String> varToOriginalName) {
             this.varToOriginalName = varToOriginalName;
             if (sp.getSubjectVar() != null && !sp.getSubjectVar().hasValue()) {
-                String name = sanitize(sp.getSubjectVar().getName());
+                final String name = sanitize(sp.getSubjectVar().getName());
                 varToTripleValue.put(name, SUBJECT);
                 varToTripleHash.put(name, SUBJECT_HASH);
             }
             if (sp.getPredicateVar() != null && !sp.getPredicateVar().hasValue()) {
-                String name = sanitize(sp.getPredicateVar().getName());
+                final String name = sanitize(sp.getPredicateVar().getName());
                 varToTripleValue.put(name, PREDICATE);
                 varToTripleHash.put(name, PREDICATE_HASH);
             }
             if (sp.getObjectVar() != null && !sp.getObjectVar().hasValue()) {
-                String name = sanitize(sp.getObjectVar().getName());
+                final String name = sanitize(sp.getObjectVar().getName());
                 varToTripleValue.put(name, OBJECT);
                 varToTripleHash.put(name, OBJECT_HASH);
                 varToTripleType.put(name, OBJECT_TYPE);
+                varToTripleType.put(name, OBJECT_LANGUAGE);
             }
             if (sp.getContextVar() != null && !sp.getContextVar().hasValue()) {
-                String name = sanitize(sp.getContextVar().getName());
+                final String name = sanitize(sp.getContextVar().getName());
                 varToTripleValue.put(name, CONTEXT);
             }
         }
@@ -207,12 +209,12 @@ public class AggregationPipelineQueryNode extends ExternalSet {
             return getProjectExpression(new LinkedList<>(), str -> "$" + str);
         }
 
-        Bson getProjectExpression(Iterable<String> alsoInclude,
-                Function<String, String> getFieldExpr) {
-            Document values = new Document();
-            Document hashes = new Document();
-            Document types = new Document();
-            for (String varName : varNames()) {
+        Bson getProjectExpression(final Iterable<String> alsoInclude,
+                final Function<String, String> getFieldExpr) {
+            final Document values = new Document();
+            final Document hashes = new Document();
+            final Document types = new Document();
+            for (final String varName : varNames()) {
                 values.append(varName, getFieldExpr.apply(valueField(varName)));
                 if (varToTripleHash.containsKey(varName)) {
                     hashes.append(varName, getFieldExpr.apply(hashField(varName)));
@@ -221,12 +223,12 @@ public class AggregationPipelineQueryNode extends ExternalSet {
                     types.append(varName, getFieldExpr.apply(typeField(varName)));
                 }
             }
-            for (String varName : alsoInclude) {
+            for (final String varName : alsoInclude) {
                 values.append(varName, 1);
                 hashes.append(varName, 1);
                 types.append(varName, 1);
             }
-            List<Bson> fields = new LinkedList<>();
+            final List<Bson> fields = new LinkedList<>();
             fields.add(Projections.excludeId());
             fields.add(Projections.computed(VALUES, values));
             fields.add(Projections.computed(HASHES, hashes));
@@ -251,7 +253,7 @@ public class AggregationPipelineQueryNode extends ExternalSet {
      *  "x" followed by "y".
      * @return The argument of a "$match" query
      */
-    private static BasicDBObject getMatchExpression(StatementPattern sp, String ... path) {
+    private static BasicDBObject getMatchExpression(final StatementPattern sp, final String ... path) {
         final Var subjVar = sp.getSubjectVar();
         final Var predVar = sp.getPredicateVar();
         final Var objVar = sp.getObjectVar();
@@ -272,34 +274,34 @@ public class AggregationPipelineQueryNode extends ExternalSet {
         if (contextVar != null && contextVar.getValue() instanceof IRI) {
             c = RdfToRyaConversions.convertIRI((IRI) contextVar.getValue());
         }
-        RyaStatement rs = new RyaStatement(s, p, o, c);
-        DBObject obj = strategy.getQuery(rs);
+        final RyaStatement rs = new RyaStatement(s, p, o, c);
+        final DBObject obj = strategy.getQuery(rs);
         // Add path prefix, if given
         if (path.length > 0) {
-            StringBuilder sb = new StringBuilder();
-            for (String str : path) {
+            final StringBuilder sb = new StringBuilder();
+            for (final String str : path) {
                 sb.append(str).append(".");
             }
-            String prefix = sb.toString();
-            Set<String> originalKeys = new HashSet<>(obj.keySet());
+            final String prefix = sb.toString();
+            final Set<String> originalKeys = new HashSet<>(obj.keySet());
             originalKeys.forEach(key -> {
-                Object value = obj.removeField(key);
+                final Object value = obj.removeField(key);
                 obj.put(prefix + key, value);
             });
         }
         return (BasicDBObject) obj;
     }
 
-    private static String valueFieldExpr(String varName) {
+    private static String valueFieldExpr(final String varName) {
         return "$" + VALUES + "." + varName;
     }
-    private static String hashFieldExpr(String varName) {
+    private static String hashFieldExpr(final String varName) {
         return "$" + HASHES + "." + varName;
     }
-    private static String typeFieldExpr(String varName) {
+    private static String typeFieldExpr(final String varName) {
         return "$" + TYPES + "." + varName;
     }
-    private static String joinFieldExpr(String triplePart) {
+    private static String joinFieldExpr(final String triplePart) {
         return "$" + JOINED_TRIPLE + "." + triplePart;
     }
 
@@ -307,7 +309,7 @@ public class AggregationPipelineQueryNode extends ExternalSet {
      * Get an object representing the value field of some value expression, or
      * return null if the expression isn't supported.
      */
-    private Object valueFieldExpr(ValueExpr expr) {
+    private Object valueFieldExpr(final ValueExpr expr) {
         if (expr instanceof Var) {
             return valueFieldExpr(((Var) expr).getName());
         }
@@ -325,12 +327,12 @@ public class AggregationPipelineQueryNode extends ExternalSet {
     private final Set<String> bindingNames;
     private final BiMap<String, String> varToOriginalName;
 
-    private String replace(String original) {
+    private String replace(final String original) {
         if (varToOriginalName.containsValue(original)) {
             return varToOriginalName.inverse().get(original);
         }
         else {
-            String replacement = "field-" + UUID.randomUUID();
+            final String replacement = "field-" + UUID.randomUUID();
             varToOriginalName.put(replacement, original);
             return replacement;
         }
@@ -341,11 +343,11 @@ public class AggregationPipelineQueryNode extends ExternalSet {
      * @param collection The collection of triples to query.
      * @param baseSP The leaf node in the query tree.
      */
-    public AggregationPipelineQueryNode(MongoCollection<Document> collection, StatementPattern baseSP) {
+    public AggregationPipelineQueryNode(final MongoCollection<Document> collection, final StatementPattern baseSP) {
         this.collection = Preconditions.checkNotNull(collection);
         Preconditions.checkNotNull(baseSP);
         this.varToOriginalName = HashBiMap.create();
-        StatementVarMapping mapping = new StatementVarMapping(baseSP, varToOriginalName);
+        final StatementVarMapping mapping = new StatementVarMapping(baseSP, varToOriginalName);
         this.assuredBindingNames = new HashSet<>(mapping.varNames());
         this.bindingNames = new HashSet<>(mapping.varNames());
         this.pipeline = new LinkedList<>();
@@ -353,9 +355,9 @@ public class AggregationPipelineQueryNode extends ExternalSet {
         this.pipeline.add(Aggregates.project(mapping.getProjectExpression()));
     }
 
-    AggregationPipelineQueryNode(MongoCollection<Document> collection,
-            List<Bson> pipeline, Set<String> assuredBindingNames,
-            Set<String> bindingNames, BiMap<String, String> varToOriginalName) {
+    AggregationPipelineQueryNode(final MongoCollection<Document> collection,
+            final List<Bson> pipeline, final Set<String> assuredBindingNames,
+            final Set<String> bindingNames, final BiMap<String, String> varToOriginalName) {
         this.collection = Preconditions.checkNotNull(collection);
         this.pipeline = Preconditions.checkNotNull(pipeline);
         this.assuredBindingNames = Preconditions.checkNotNull(assuredBindingNames);
@@ -364,12 +366,12 @@ public class AggregationPipelineQueryNode extends ExternalSet {
     }
 
     @Override
-    public boolean equals(Object o) {
+    public boolean equals(final Object o) {
         if (this == o) {
             return true;
         }
         if (o instanceof AggregationPipelineQueryNode) {
-            AggregationPipelineQueryNode other = (AggregationPipelineQueryNode) o;
+            final AggregationPipelineQueryNode other = (AggregationPipelineQueryNode) o;
             if (this.collection.equals(other.collection)
                     && this.assuredBindingNames.equals(other.assuredBindingNames)
                     && this.bindingNames.equals(other.bindingNames)
@@ -379,8 +381,8 @@ public class AggregationPipelineQueryNode extends ExternalSet {
                 // have well-behaved equals methods, so check for equivalent
                 // string representations.
                 for (int i = 0; i < this.pipeline.size(); i++) {
-                    Bson doc1 = this.pipeline.get(i);
-                    Bson doc2 = other.pipeline.get(i);
+                    final Bson doc1 = this.pipeline.get(i);
+                    final Bson doc2 = other.pipeline.get(i);
                     if (!doc1.toString().equals(doc2.toString())) {
                         return false;
                     }
@@ -398,15 +400,15 @@ public class AggregationPipelineQueryNode extends ExternalSet {
     }
 
     @Override
-    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(BindingSet bindings)
+    public CloseableIteration<BindingSet, QueryEvaluationException> evaluate(final BindingSet bindings)
             throws QueryEvaluationException {
         return new PipelineResultIteration(collection.aggregate(pipeline), varToOriginalName, bindings);
     }
 
     @Override
     public Set<String> getAssuredBindingNames() {
-        Set<String> names = new HashSet<>();
-        for (String name : assuredBindingNames) {
+        final Set<String> names = new HashSet<>();
+        for (final String name : assuredBindingNames) {
             names.add(varToOriginalName.getOrDefault(name, name));
         }
         return names;
@@ -414,8 +416,8 @@ public class AggregationPipelineQueryNode extends ExternalSet {
 
     @Override
     public Set<String> getBindingNames() {
-        Set<String> names = new HashSet<>();
-        for (String name : bindingNames) {
+        final Set<String> names = new HashSet<>();
+        for (final String name : bindingNames) {
             names.add(varToOriginalName.getOrDefault(name, name));
         }
         return names;
@@ -433,19 +435,19 @@ public class AggregationPipelineQueryNode extends ExternalSet {
     @Override
     public String getSignature() {
         super.getSignature();
-        Set<String> assured = getAssuredBindingNames();
-        Set<String> any = getBindingNames();
-        StringBuilder sb = new StringBuilder("AggregationPipelineQueryNode (binds: ");
+        final Set<String> assured = getAssuredBindingNames();
+        final Set<String> any = getBindingNames();
+        final StringBuilder sb = new StringBuilder("AggregationPipelineQueryNode (binds: ");
         sb.append(String.join(", ", assured));
         if (any.size() > assured.size()) {
-            Set<String> optionalBindingNames = any;
+            final Set<String> optionalBindingNames = any;
             optionalBindingNames.removeAll(assured);
             sb.append(" [")
                 .append(String.join(", ", optionalBindingNames))
                 .append("]");
         }
         sb.append(")\n");
-        for (Bson doc : pipeline) {
+        for (final Bson doc : pipeline) {
             sb.append(doc.toString()).append("\n");
         }
         return sb.toString();
@@ -468,15 +470,15 @@ public class AggregationPipelineQueryNode extends ExternalSet {
      * @param sp The statement pattern to join with
      * @return true if the join was successfully added to the pipeline.
      */
-    public boolean joinWith(StatementPattern sp) {
+    public boolean joinWith(final StatementPattern sp) {
         Preconditions.checkNotNull(sp);
         // 1. Determine shared variables and new variables
-        StatementVarMapping spMap = new StatementVarMapping(sp, varToOriginalName);
-        NavigableSet<String> sharedVars = new ConcurrentSkipListSet<>(spMap.varNames());
+        final StatementVarMapping spMap = new StatementVarMapping(sp, varToOriginalName);
+        final NavigableSet<String> sharedVars = new ConcurrentSkipListSet<>(spMap.varNames());
         sharedVars.retainAll(assuredBindingNames);
         // 2. Join on one shared variable
-        String joinKey =  sharedVars.pollFirst();
-        String collectionName = collection.getNamespace().getCollectionName();
+        final String joinKey =  sharedVars.pollFirst();
+        final String collectionName = collection.getNamespace().getCollectionName();
         Bson join;
         if (joinKey == null) {
             return false;
@@ -494,16 +496,16 @@ public class AggregationPipelineQueryNode extends ExternalSet {
         // 4. (Optional) If there are any shared variables that weren't used as
         //   the join key, project all existing fields plus a new field that
         //   tests the equality of those shared variables.
-        BasicDBObject matchOpts = getMatchExpression(sp, JOINED_TRIPLE);
+        final BasicDBObject matchOpts = getMatchExpression(sp, JOINED_TRIPLE);
         if (!sharedVars.isEmpty()) {
-            List<Bson> eqTests = new LinkedList<>();
-            for (String varName : sharedVars) {
-                String oldField = valueFieldExpr(varName);
-                String newField = joinFieldExpr(spMap.valueField(varName));
-                Bson eqTest = new Document("$eq", Arrays.asList(oldField, newField));
+            final List<Bson> eqTests = new LinkedList<>();
+            for (final String varName : sharedVars) {
+                final String oldField = valueFieldExpr(varName);
+                final String newField = joinFieldExpr(spMap.valueField(varName));
+                final Bson eqTest = new Document("$eq", Arrays.asList(oldField, newField));
                 eqTests.add(eqTest);
             }
-            Bson eqProjectOpts = Projections.fields(
+            final Bson eqProjectOpts = Projections.fields(
                     Projections.computed(FIELDS_MATCH, Filters.and(eqTests)),
                     Projections.include(JOINED_TRIPLE, VALUES, HASHES, TYPES, LEVEL, TIMESTAMP));
             pipeline.add(Aggregates.project(eqProjectOpts));
@@ -516,7 +518,7 @@ public class AggregationPipelineQueryNode extends ExternalSet {
         // 6. Project the results to include variables from the new SP (with
         // appropriate renaming) and variables referenced only in the base
         // pipeline (with previous names).
-        Bson finalProjectOpts = new StatementVarMapping(sp, varToOriginalName)
+        final Bson finalProjectOpts = new StatementVarMapping(sp, varToOriginalName)
                 .getProjectExpression(assuredBindingNames,
                         str -> joinFieldExpr(str));
         assuredBindingNames.addAll(spMap.varNames());
@@ -537,23 +539,23 @@ public class AggregationPipelineQueryNode extends ExternalSet {
      *  at this stage of the query into a set of variables.
      * @return true if the projection(s) were added to the pipeline.
      */
-    public boolean project(Iterable<ProjectionElemList> projections) {
+    public boolean project(final Iterable<ProjectionElemList> projections) {
         if (projections == null || !projections.iterator().hasNext()) {
             return false;
         }
-        List<Bson> projectOpts = new LinkedList<>();
-        Set<String> bindingNamesUnion = new HashSet<>();
+        final List<Bson> projectOpts = new LinkedList<>();
+        final Set<String> bindingNamesUnion = new HashSet<>();
         Set<String> bindingNamesIntersection = null;
-        for (ProjectionElemList projection : projections) {
+        for (final ProjectionElemList projection : projections) {
             if (projection.getElements().isEmpty()) {
                 // Empty projections are unsupported -- fail when seen
                 return false;
             }
-            Document valueDoc = new Document();
-            Document hashDoc = new Document();
-            Document typeDoc = new Document();
-            Set<String> projectionBindingNames = new HashSet<>();
-            for (ProjectionElem elem : projection.getElements()) {
+            final Document valueDoc = new Document();
+            final Document hashDoc = new Document();
+            final Document typeDoc = new Document();
+            final Set<String> projectionBindingNames = new HashSet<>();
+            for (final ProjectionElem elem : projection.getElements()) {
                 String to = elem.getTargetName();
                 // If the 'to' name is invalid, replace it internally
                 if (!isValidFieldName(to)) {
@@ -594,8 +596,8 @@ public class AggregationPipelineQueryNode extends ExternalSet {
             pipeline.add(Aggregates.project(projectOpts.get(0)));
         }
         else {
-            String listKey = "PROJECTIONS";
-            Bson projectIndividual = Projections.fields(
+            final String listKey = "PROJECTIONS";
+            final Bson projectIndividual = Projections.fields(
                     Projections.computed(VALUES, "$" + listKey + "." + VALUES),
                     Projections.computed(HASHES, "$" + listKey + "." + HASHES),
                     Projections.computed(TYPES, "$" + listKey + "." + TYPES),
@@ -625,17 +627,17 @@ public class AggregationPipelineQueryNode extends ExternalSet {
      * @return True if the extension was successfully converted into a pipeline
      *  step, false otherwise.
      */
-    public boolean extend(Iterable<ExtensionElem> extensionElements) {
-        List<Bson> valueFields = new LinkedList<>();
-        List<Bson> hashFields = new LinkedList<>();
-        List<Bson> typeFields = new LinkedList<>();
-        for (String varName : bindingNames) {
+    public boolean extend(final Iterable<ExtensionElem> extensionElements) {
+        final List<Bson> valueFields = new LinkedList<>();
+        final List<Bson> hashFields = new LinkedList<>();
+        final List<Bson> typeFields = new LinkedList<>();
+        for (final String varName : bindingNames) {
             valueFields.add(Projections.include(varName));
             hashFields.add(Projections.include(varName));
             typeFields.add(Projections.include(varName));
         }
-        Set<String> newVarNames = new HashSet<>();
-        for (ExtensionElem elem : extensionElements) {
+        final Set<String> newVarNames = new HashSet<>();
+        for (final ExtensionElem elem : extensionElements) {
             String name = elem.getName();
             if (!isValidFieldName(name)) {
                 // If the field name is invalid, replace it internally
@@ -643,18 +645,18 @@ public class AggregationPipelineQueryNode extends ExternalSet {
             }
             // We can only handle certain kinds of value expressions; return
             // failure for any others.
-            ValueExpr expr = elem.getExpr();
+            final ValueExpr expr = elem.getExpr();
             final Object valueField;
             final Object hashField;
             final Object typeField;
             if (expr instanceof Var) {
-                String varName = ((Var) expr).getName();
+                final String varName = ((Var) expr).getName();
                 valueField = "$" + varName;
                 hashField = "$" + varName;
                 typeField = "$" + varName;
             }
             else if (expr instanceof ValueConstant) {
-                Value val = ((ValueConstant) expr).getValue();
+                final Value val = ((ValueConstant) expr).getValue();
                 valueField = new Document("$literal", val.stringValue());
                 hashField = new Document("$literal", SimpleMongoDBStorageStrategy.hash(val.stringValue()));
                 if (val instanceof Literal) {
@@ -677,7 +679,7 @@ public class AggregationPipelineQueryNode extends ExternalSet {
         }
         assuredBindingNames.addAll(newVarNames);
         bindingNames.addAll(newVarNames);
-        Bson projectOpts = Projections.fields(
+        final Bson projectOpts = Projections.fields(
                 Projections.computed(VALUES, Projections.fields(valueFields)),
                 Projections.computed(HASHES, Projections.fields(hashFields)),
                 Projections.computed(TYPES, Projections.fields(typeFields)),
@@ -698,12 +700,12 @@ public class AggregationPipelineQueryNode extends ExternalSet {
      * @return True if the filter was successfully converted into a pipeline
      *  step, false otherwise.
      */
-    public boolean filter(ValueExpr condition) {
+    public boolean filter(final ValueExpr condition) {
         if (condition instanceof Compare) {
-            Compare compare = (Compare) condition;
-            Compare.CompareOp operator = compare.getOperator();
-            Object leftArg = valueFieldExpr(compare.getLeftArg());
-            Object rightArg = valueFieldExpr(compare.getRightArg());
+            final Compare compare = (Compare) condition;
+            final Compare.CompareOp operator = compare.getOperator();
+            final Object leftArg = valueFieldExpr(compare.getLeftArg());
+            final Object rightArg = valueFieldExpr(compare.getRightArg());
             if (leftArg == null || rightArg == null) {
                 // unsupported value expression, can't convert filter
                 return false;
@@ -732,7 +734,7 @@ public class AggregationPipelineQueryNode extends ExternalSet {
                 // unrecognized comparison operator, can't convert filter
                 return false;
             }
-            Document compareDoc = new Document(opFunc, Arrays.asList(leftArg, rightArg));
+            final Document compareDoc = new Document(opFunc, Arrays.asList(leftArg, rightArg));
             pipeline.add(Aggregates.project(Projections.fields(
                     Projections.computed("FILTER", compareDoc),
                     Projections.include(VALUES, HASHES, TYPES, LEVEL, TIMESTAMP))));
@@ -749,12 +751,12 @@ public class AggregationPipelineQueryNode extends ExternalSet {
      * @return True if the distinct operation was successfully appended.
      */
     public boolean distinct() {
-        List<String> key = new LinkedList<>();
-        for (String varName : bindingNames) {
+        final List<String> key = new LinkedList<>();
+        for (final String varName : bindingNames) {
             key.add(hashFieldExpr(varName));
         }
-        List<BsonField> reduceOps = new LinkedList<>();
-        for (String field : FIELDS) {
+        final List<BsonField> reduceOps = new LinkedList<>();
+        for (final String field : FIELDS) {
             reduceOps.add(new BsonField(field, new Document("$first", "$" + field)));
         }
         pipeline.add(Aggregates.group(new Document("$concat", key), reduceOps));
@@ -774,7 +776,7 @@ public class AggregationPipelineQueryNode extends ExternalSet {
      *  query if all of the triples involved in producing that solution have a
      *  lower derivation depth than this. If zero, does nothing.
      */
-    public void requireSourceDerivationDepth(int requiredLevel) {
+    public void requireSourceDerivationDepth(final int requiredLevel) {
         if (requiredLevel > 0) {
             pipeline.add(Aggregates.match(new Document(LEVEL,
                     new Document("$gte", requiredLevel))));
@@ -791,7 +793,7 @@ public class AggregationPipelineQueryNode extends ExternalSet {
      *  all of the triples involved in producing that solution have an earlier
      *  timestamp than this.
      */
-    public void requireSourceTimestamp(long t) {
+    public void requireSourceTimestamp(final long t) {
         pipeline.add(Aggregates.match(new Document(TIMESTAMP,
                 new Document("$gte", t))));
     }
@@ -810,7 +812,7 @@ public class AggregationPipelineQueryNode extends ExternalSet {
      *  pipeline do not have variable names allowing them to be interpreted as
      *  triples (i.e. "subject", "predicate", and "object").
      */
-    public List<Bson> getTriplePipeline(long timestamp, boolean requireNew) {
+    public List<Bson> getTriplePipeline(final long timestamp, final boolean requireNew) {
         if (!assuredBindingNames.contains(SUBJECT)
                 || !assuredBindingNames.contains(PREDICATE)
                 || !assuredBindingNames.contains(OBJECT)) {
@@ -820,8 +822,8 @@ public class AggregationPipelineQueryNode extends ExternalSet {
                     + ", " + OBJECT + ">\nCurrent variable names: "
                     + assuredBindingNames);
         }
-        List<Bson> triplePipeline = new LinkedList<>(pipeline);
-        List<Bson> fields = new LinkedList<>();
+        final List<Bson> triplePipeline = new LinkedList<>(pipeline);
+        final List<Bson> fields = new LinkedList<>();
         fields.add(Projections.computed(SUBJECT, valueFieldExpr(SUBJECT)));
         fields.add(Projections.computed(SUBJECT_HASH, hashFieldExpr(SUBJECT)));
         fields.add(Projections.computed(PREDICATE, valueFieldExpr(PREDICATE)));
@@ -830,6 +832,7 @@ public class AggregationPipelineQueryNode extends ExternalSet {
         fields.add(Projections.computed(OBJECT_HASH, hashFieldExpr(OBJECT)));
         fields.add(Projections.computed(OBJECT_TYPE,
                 ConditionalOperators.ifNull(typeFieldExpr(OBJECT), DEFAULT_TYPE)));
+        fields.add(Projections.computed(OBJECT_LANGUAGE, hashFieldExpr(OBJECT)));
         fields.add(Projections.computed(CONTEXT, DEFAULT_CONTEXT));
         fields.add(Projections.computed(STATEMENT_METADATA, DEFAULT_METADATA));
         fields.add(DEFAULT_DV);
@@ -838,19 +841,19 @@ public class AggregationPipelineQueryNode extends ExternalSet {
         triplePipeline.add(Aggregates.project(Projections.fields(fields)));
         if (requireNew) {
             // Prune any triples that already exist in the data store
-            String collectionName = collection.getNamespace().getCollectionName();
-            Bson includeAll = Projections.include(SUBJECT, SUBJECT_HASH,
+            final String collectionName = collection.getNamespace().getCollectionName();
+            final Bson includeAll = Projections.include(SUBJECT, SUBJECT_HASH,
                     PREDICATE, PREDICATE_HASH, OBJECT, OBJECT_HASH,
-                    OBJECT_TYPE, CONTEXT, STATEMENT_METADATA,
+                    OBJECT_TYPE, OBJECT_LANGUAGE, CONTEXT, STATEMENT_METADATA,
                     DOCUMENT_VISIBILITY, TIMESTAMP, LEVEL);
-            List<Bson> eqTests = new LinkedList<>();
+            final List<Bson> eqTests = new LinkedList<>();
             eqTests.add(new Document("$eq", Arrays.asList("$$this." + PREDICATE_HASH, "$" + PREDICATE_HASH)));
             eqTests.add(new Document("$eq", Arrays.asList("$$this." + OBJECT_HASH, "$" + OBJECT_HASH)));
-            Bson redundantFilter = new Document("$filter", new Document("input", "$" + JOINED_TRIPLE)
+            final Bson redundantFilter = new Document("$filter", new Document("input", "$" + JOINED_TRIPLE)
                     .append("as", "this").append("cond", new Document("$and", eqTests)));
             triplePipeline.add(Aggregates.lookup(collectionName, SUBJECT_HASH,
                     SUBJECT_HASH, JOINED_TRIPLE));
-            String numRedundant = "REDUNDANT";
+            final String numRedundant = "REDUNDANT";
             triplePipeline.add(Aggregates.project(Projections.fields(includeAll,
                     Projections.computed(numRedundant, new Document("$size", redundantFilter)))));
             triplePipeline.add(Aggregates.match(Filters.eq(numRedundant, 0)));

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java
index ec5c7a5..7d4911f 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java
@@ -28,11 +28,12 @@ import java.util.Map;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.log4j.Logger;
+import org.apache.rya.api.domain.RyaIRI;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaType;
-import org.apache.rya.api.domain.RyaIRI;
 import org.apache.rya.api.domain.StatementMetadata;
 import org.apache.rya.api.persist.query.RyaQuery;
+import org.apache.rya.api.utils.LiteralLanguageUtils;
 import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
 import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter;
 import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter.MalformedDocumentVisibilityException;
@@ -57,6 +58,7 @@ public class SimpleMongoDBStorageStrategy implements MongoDBStorageStrategy<RyaS
     public static final String PREDICATE_HASH = "predicate_hash";
     public static final String OBJECT = "object";
     public static final String OBJECT_HASH = "object_hash";
+    public static final String OBJECT_LANGUAGE = "object_language";
     public static final String SUBJECT = "subject";
     public static final String SUBJECT_HASH = "subject_hash";
     public static final String TIMESTAMP = "insertTimestamp";
@@ -68,7 +70,7 @@ public class SimpleMongoDBStorageStrategy implements MongoDBStorageStrategy<RyaS
      * @param value  A value to be stored or accessed (e.g. a IRI or literal).
      * @return the hash associated with that value in MongoDB.
      */
-    public static String hash(String value) {
+    public static String hash(final String value) {
         return DigestUtils.sha256Hex(value);
     }
 
@@ -81,13 +83,16 @@ public class SimpleMongoDBStorageStrategy implements MongoDBStorageStrategy<RyaS
         doc.put(PREDICATE_HASH, 1);
         doc.put(OBJECT_HASH, 1);
         doc.put(OBJECT_TYPE, 1);
+        doc.put(OBJECT_LANGUAGE, 1);
         coll.createIndex(doc);
         doc = new BasicDBObject(PREDICATE_HASH, 1);
         doc.put(OBJECT_HASH, 1);
         doc.put(OBJECT_TYPE, 1);
+        doc.put(OBJECT_LANGUAGE, 1);
         coll.createIndex(doc);
         doc = new BasicDBObject(OBJECT_HASH, 1);
         doc.put(OBJECT_TYPE, 1);
+        doc.put(OBJECT_LANGUAGE, 1);
         doc.put(SUBJECT_HASH, 1);
         coll.createIndex(doc);
     }
@@ -105,6 +110,7 @@ public class SimpleMongoDBStorageStrategy implements MongoDBStorageStrategy<RyaS
         if (object != null){
             query.append(OBJECT_HASH, hash(object.getData()));
             query.append(OBJECT_TYPE, object.getDataType().toString());
+            query.append(OBJECT_LANGUAGE, object.getLanguage());
         }
         if (predicate != null){
             query.append(PREDICATE_HASH, hash(predicate.getData()));
@@ -121,6 +127,7 @@ public class SimpleMongoDBStorageStrategy implements MongoDBStorageStrategy<RyaS
         final String subject = (String) result.get(SUBJECT);
         final String object = (String) result.get(OBJECT);
         final String objectType = (String) result.get(OBJECT_TYPE);
+        final String objectLanguage = (String) result.get(OBJECT_LANGUAGE);
         final String predicate = (String) result.get(PREDICATE);
         final String context = (String) result.get(CONTEXT);
         DocumentVisibility documentVisibility = null;
@@ -132,10 +139,12 @@ public class SimpleMongoDBStorageStrategy implements MongoDBStorageStrategy<RyaS
         final Long timestamp = (Long) result.get(TIMESTAMP);
         final String statementMetadata = (String) result.get(STATEMENT_METADATA);
         RyaType objectRya = null;
+        final String validatedLanguage = LiteralLanguageUtils.validateLanguage(objectLanguage, factory.createIRI(objectType));
         if (objectType.equalsIgnoreCase(ANYURI.stringValue())){
             objectRya = new RyaIRI(object);
-        }
-        else {
+        } else if (validatedLanguage != null) {
+            objectRya = new RyaType(factory.createIRI(objectType), object, validatedLanguage);
+        } else {
             objectRya = new RyaType(factory.createIRI(objectType), object);
         }
 
@@ -173,8 +182,9 @@ public class SimpleMongoDBStorageStrategy implements MongoDBStorageStrategy<RyaS
         if (statement.getContext() != null){
             context = statement.getContext().getData();
         }
+        final String validatedLanguage = LiteralLanguageUtils.validateLanguage(statement.getObject().getLanguage(), statement.getObject().getDataType());
         final String id = statement.getSubject().getData() + " " +
-                statement.getPredicate().getData() + " " +  statement.getObject().getData() + " " + context;
+                statement.getPredicate().getData() + " " +  statement.getObject().getData() + (validatedLanguage != null ? " " + validatedLanguage : "") + " " + context;
         byte[] bytes = id.getBytes(StandardCharsets.UTF_8);
         try {
             final MessageDigest digest = MessageDigest.getInstance("SHA-1");
@@ -194,6 +204,7 @@ public class SimpleMongoDBStorageStrategy implements MongoDBStorageStrategy<RyaS
         .append(OBJECT, statement.getObject().getData())
         .append(OBJECT_HASH, hash(statement.getObject().getData()))
         .append(OBJECT_TYPE, statement.getObject().getDataType().toString())
+        .append(OBJECT_LANGUAGE, statement.getObject().getLanguage())
         .append(CONTEXT, context)
         .append(STATEMENT_METADATA, statement.getMetadata().toString())
         .append(DOCUMENT_VISIBILITY, dvObject.get(DOCUMENT_VISIBILITY))

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/SimpleMongoDBStorageStrategyTest.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/SimpleMongoDBStorageStrategyTest.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/SimpleMongoDBStorageStrategyTest.java
index 576610c..a757ba3 100644
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/SimpleMongoDBStorageStrategyTest.java
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/SimpleMongoDBStorageStrategyTest.java
@@ -24,14 +24,16 @@ import static org.junit.Assert.assertEquals;
 import java.io.IOException;
 
 import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.rya.api.domain.RyaIRI;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
-import org.apache.rya.api.domain.RyaIRI;
+import org.apache.rya.api.domain.RyaType;
 import org.apache.rya.api.persist.RyaDAOException;
 import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
 import org.apache.rya.mongodb.document.util.DocumentVisibilityConversionException;
 import org.apache.rya.mongodb.document.util.DocumentVisibilityUtil;
 import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
+import org.eclipse.rdf4j.model.vocabulary.RDF;
 import org.junit.Test;
 
 import com.mongodb.BasicDBObject;
@@ -47,11 +49,13 @@ public class SimpleMongoDBStorageStrategyTest {
     private static final DocumentVisibility DOCUMENT_VISIBILITY = new DocumentVisibility("A&B");
 
     private static final RyaStatement testStatement;
+    private static final RyaStatement testStatement2;
     private static final DBObject testDBO;
+    private static final DBObject testDBO2;
     private final SimpleMongoDBStorageStrategy storageStrategy = new SimpleMongoDBStorageStrategy();
 
     static {
-        final RyaStatementBuilder builder = new RyaStatementBuilder();
+        RyaStatementBuilder builder = new RyaStatementBuilder();
         builder.setPredicate(new RyaIRI(PREDICATE));
         builder.setSubject(new RyaIRI(SUBJECT));
         builder.setObject(new RyaIRI(OBJECT));
@@ -69,6 +73,7 @@ public class SimpleMongoDBStorageStrategyTest {
         testDBO.put(SimpleMongoDBStorageStrategy.OBJECT, OBJECT);
         testDBO.put(SimpleMongoDBStorageStrategy.OBJECT_HASH, DigestUtils.sha256Hex(OBJECT));
         testDBO.put(SimpleMongoDBStorageStrategy.OBJECT_TYPE, ANYURI.stringValue());
+        testDBO.put(SimpleMongoDBStorageStrategy.OBJECT_LANGUAGE, null);
         testDBO.put(SimpleMongoDBStorageStrategy.CONTEXT, CONTEXT);
         testDBO.put(SimpleMongoDBStorageStrategy.STATEMENT_METADATA, STATEMENT_METADATA);
         try {
@@ -77,18 +82,51 @@ public class SimpleMongoDBStorageStrategyTest {
             e.printStackTrace();
         }
         testDBO.put(SimpleMongoDBStorageStrategy.TIMESTAMP, null);
+
+
+        builder = new RyaStatementBuilder();
+        builder.setPredicate(new RyaIRI(PREDICATE));
+        builder.setSubject(new RyaIRI(SUBJECT));
+        builder.setObject(new RyaType(RDF.LANGSTRING, OBJECT, "en-US"));
+        builder.setContext(new RyaIRI(CONTEXT));
+        builder.setColumnVisibility(DOCUMENT_VISIBILITY.flatten());
+        builder.setTimestamp(null);
+        testStatement2 = builder.build();
+
+        // Check language support
+        testDBO2 = new BasicDBObject();
+        testDBO2.put(SimpleMongoDBStorageStrategy.ID, "580fb5d11f0b62fa735ac98b36bba1fc37ddc3fc");
+        testDBO2.put(SimpleMongoDBStorageStrategy.SUBJECT, SUBJECT);
+        testDBO2.put(SimpleMongoDBStorageStrategy.SUBJECT_HASH, DigestUtils.sha256Hex(SUBJECT));
+        testDBO2.put(SimpleMongoDBStorageStrategy.PREDICATE, PREDICATE);
+        testDBO2.put(SimpleMongoDBStorageStrategy.PREDICATE_HASH, DigestUtils.sha256Hex(PREDICATE));
+        testDBO2.put(SimpleMongoDBStorageStrategy.OBJECT, OBJECT);
+        testDBO2.put(SimpleMongoDBStorageStrategy.OBJECT_HASH, DigestUtils.sha256Hex(OBJECT));
+        testDBO2.put(SimpleMongoDBStorageStrategy.OBJECT_TYPE, RDF.LANGSTRING.stringValue());
+        testDBO2.put(SimpleMongoDBStorageStrategy.OBJECT_LANGUAGE, "en-US");
+        testDBO2.put(SimpleMongoDBStorageStrategy.CONTEXT, CONTEXT);
+        testDBO2.put(SimpleMongoDBStorageStrategy.STATEMENT_METADATA, STATEMENT_METADATA);
+        try {
+            testDBO2.put(SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY, DocumentVisibilityUtil.toMultidimensionalArray(DOCUMENT_VISIBILITY));
+        } catch (final DocumentVisibilityConversionException e) {
+            e.printStackTrace();
+        }
+        testDBO2.put(SimpleMongoDBStorageStrategy.TIMESTAMP, null);
     }
 
     @Test
     public void testSerializeStatementToDBO() throws RyaDAOException, MongoException, IOException {
 
-        final DBObject dbo = storageStrategy.serialize(testStatement);
+        DBObject dbo = storageStrategy.serialize(testStatement);
         assertEquals(testDBO, dbo);
+
+        dbo = storageStrategy.serialize(testStatement2);
+        assertEquals(testDBO2, dbo);
     }
 
     @Test
     public void testDeSerializeStatementToDBO() throws RyaDAOException, MongoException, IOException {
-        final RyaStatement statement = storageStrategy.deserializeDBObject(testDBO);
+        RyaStatement statement = storageStrategy.deserializeDBObject(testDBO);
         /*
          * Since RyaStatement creates a timestamp using JVM time if the timestamp is null, we want to re-null it
          * for this test.  Timestamp is created at insert time by the Server, this test
@@ -96,5 +134,14 @@ public class SimpleMongoDBStorageStrategyTest {
          */
         statement.setTimestamp(null);
         assertEquals(testStatement, statement);
+
+        statement = storageStrategy.deserializeDBObject(testDBO2);
+        /*
+         * Since RyaStatement creates a timestamp using JVM time if the timestamp is null, we want to re-null it
+         * for this test.  Timestamp is created at insert time by the Server, this test
+         * can be found in the RyaDAO.
+         */
+        statement.setTimestamp(null);
+        assertEquals(testStatement2, statement);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/extras/indexing/pom.xml b/extras/indexing/pom.xml
index 837d67a..329b980 100644
--- a/extras/indexing/pom.xml
+++ b/extras/indexing/pom.xml
@@ -133,6 +133,7 @@
                         <excludes>
                             <!-- RDF data Files -->
                             <exclude>**/*.ttl</exclude>
+                            <exclude>**/src/test/resources/rdf_format_files/**</exclude>
 
                             <!-- Services Files -->
                             <exclude>**/resources/META-INF/services/**</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/main/java/org/apache/rya/indexing/StatementSerializer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/StatementSerializer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/StatementSerializer.java
index 8ea44c5..db8dfac 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/StatementSerializer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/StatementSerializer.java
@@ -8,9 +8,9 @@ package org.apache.rya.indexing;
  * 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
@@ -24,6 +24,7 @@ import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.Validate;
+import org.apache.rya.api.utils.LiteralLanguageUtils;
 import org.eclipse.rdf4j.model.IRI;
 import org.eclipse.rdf4j.model.Literal;
 import org.eclipse.rdf4j.model.Resource;
@@ -42,34 +43,34 @@ public class StatementSerializer {
 
     /**
      * Read a {@link Statement} from a {@link String}
-     * 
+     *
      * @param in
      *            the {@link String} to parse
      * @return a {@link Statement}
      */
-    public static Statement readStatement(String in) throws IOException {
-        String[] parts = in.split(SEP);
-        
+    public static Statement readStatement(final String in) throws IOException {
+        final String[] parts = in.split(SEP);
+
         if (parts.length != 4) {
             throw new IOException("Not a valid statement: " + in);
         }
-        
-        String contextString = parts[0];
-        String subjectString = parts[1];
-        String predicateString = parts[2];
-        String objectString = parts[3];
+
+        final String contextString = parts[0];
+        final String subjectString = parts[1];
+        final String predicateString = parts[2];
+        final String objectString = parts[3];
         return readStatement(subjectString, predicateString, objectString, contextString);
     }
 
-    public static Statement readStatement(String subjectString, String predicateString, String objectString) {
+    public static Statement readStatement(final String subjectString, final String predicateString, final String objectString) {
         return readStatement(subjectString, predicateString, objectString, "");
     }
 
-    public static Statement readStatement(String subjectString, String predicateString, String objectString, String contextString) {
-        Resource subject = createResource(subjectString);
-        IRI predicate = VF.createIRI(predicateString);
+    public static Statement readStatement(final String subjectString, final String predicateString, final String objectString, final String contextString) {
+        final Resource subject = createResource(subjectString);
+        final IRI predicate = VF.createIRI(predicateString);
 
-        boolean isObjectLiteral = objectString.startsWith("\"");
+        final boolean isObjectLiteral = objectString.startsWith("\"");
 
         Value object = null;
         if (isObjectLiteral) {
@@ -81,12 +82,12 @@ public class StatementSerializer {
         if (contextString == null || contextString.isEmpty()) {
             return VF.createStatement(subject, predicate, object);
         } else {
-            Resource context = VF.createIRI(contextString);
+            final Resource context = VF.createIRI(contextString);
             return VF.createStatement(subject, predicate, object, context);
         }
     }
 
-    private static Resource createResource(String str) {
+    private static Resource createResource(final String str) {
         if (str.startsWith("_")) {
             return VF.createBNode(str.substring(2));
         }
@@ -94,30 +95,30 @@ public class StatementSerializer {
 
     }
 
-    private static Literal parseLiteral(String fullLiteralString) {
+    private static Literal parseLiteral(final String fullLiteralString) {
         Validate.notNull(fullLiteralString);
         Validate.isTrue(fullLiteralString.length() > 1);
 
         if (fullLiteralString.endsWith("\"")) {
-            String fullLiteralWithoutQuotes = fullLiteralString.substring(1, fullLiteralString.length() - 1);
+            final String fullLiteralWithoutQuotes = fullLiteralString.substring(1, fullLiteralString.length() - 1);
             return VF.createLiteral(fullLiteralWithoutQuotes);
         } else {
 
             // find the closing quote
-            int labelEnd = fullLiteralString.lastIndexOf("\"");
+            final int labelEnd = fullLiteralString.lastIndexOf("\"");
 
-            String label = fullLiteralString.substring(1, labelEnd);
+            final String label = fullLiteralString.substring(1, labelEnd);
 
-            String data = fullLiteralString.substring(labelEnd + 1);
+            final String data = fullLiteralString.substring(labelEnd + 1);
 
-            if (data.startsWith("@")) {
+            if (data.startsWith(LiteralLanguageUtils.LANGUAGE_DELIMITER)) {
                 // the data is "language"
-                String lang = data.substring(1);
+                final String lang = data.substring(1);
                 return VF.createLiteral(label, lang);
             } else if (data.startsWith("^^<")) {
                 // the data is a "datatype"
-                String datatype = data.substring(3, data.length() - 1);
-                IRI datatypeUri = VF.createIRI(datatype);
+                final String datatype = data.substring(3, data.length() - 1);
+                final IRI datatypeUri = VF.createIRI(datatype);
                 return VF.createLiteral(label, datatypeUri);
             }
         }
@@ -125,26 +126,26 @@ public class StatementSerializer {
 
     }
 
-    public static String writeSubject(Statement statement) {
+    public static String writeSubject(final Statement statement) {
         return statement.getSubject().toString();
     }
 
-    public static String writeObject(Statement statement) {
+    public static String writeObject(final Statement statement) {
         return statement.getObject().toString();
     }
 
-    public static String writePredicate(Statement statement) {
+    public static String writePredicate(final Statement statement) {
         return statement.getPredicate().toString();
     }
 
-    public static String writeSubjectPredicate(Statement statement) {
+    public static String writeSubjectPredicate(final Statement statement) {
         Validate.notNull(statement);
         Validate.notNull(statement.getSubject());
         Validate.notNull(statement.getPredicate());
         return statement.getSubject().toString() + SEP + statement.getPredicate().toString();
     }
 
-    public static String writeContext(Statement statement) {
+    public static String writeContext(final Statement statement) {
         if (statement.getContext() == null) {
             return "";
         }
@@ -153,16 +154,16 @@ public class StatementSerializer {
 
     /**
      * Write a {@link Statement} to a {@link String}
-     * 
+     *
      * @param statement
      *            the {@link Statement} to write
      * @return a {@link String} representation of the statement
      */
-    public static String writeStatement(Statement statement) {
-        Resource subject = statement.getSubject();
-        Resource context = statement.getContext();
-        IRI predicate = statement.getPredicate();
-        Value object = statement.getObject();
+    public static String writeStatement(final Statement statement) {
+        final Resource subject = statement.getSubject();
+        final Resource context = statement.getContext();
+        final IRI predicate = statement.getPredicate();
+        final Value object = statement.getObject();
 
         Validate.notNull(subject);
         Validate.notNull(predicate);
@@ -180,7 +181,7 @@ public class StatementSerializer {
     /**
      * Creates a Regular Expression to match serialized statements meeting these constraints. A <code>null</code> or empty parameters imply
      * no constraint. A <code>null</code> return value implies no constraints.
-     * 
+     *
      * @param context
      *            context constraint
      * @param subject
@@ -190,22 +191,22 @@ public class StatementSerializer {
      * @return a regular expression that can be used to match serialized statements. A <code>null</code> return value implies no
      *         constraints.
      */
-    public static String createStatementRegex(StatementConstraints contraints) {
-        Resource context = contraints.getContext();
-        Resource subject = contraints.getSubject();
-        Set<IRI> predicates = contraints.getPredicates();
+    public static String createStatementRegex(final StatementConstraints contraints) {
+        final Resource context = contraints.getContext();
+        final Resource subject = contraints.getSubject();
+        final Set<IRI> predicates = contraints.getPredicates();
         if (context == null && subject == null && (predicates == null || predicates.isEmpty())) {
             return null;
         }
 
         // match on anything but a separator
-        String anyReg = "[^" + SEP + "]*";
+        final String anyReg = "[^" + SEP + "]*";
 
         // if context is empty, match on any context
-        String contextReg = (context == null) ? anyReg : context.stringValue();
+        final String contextReg = (context == null) ? anyReg : context.stringValue();
 
         // if subject is empty, match on any subject
-        String subjectReg = (subject == null) ? anyReg : subject.stringValue();
+        final String subjectReg = (subject == null) ? anyReg : subject.stringValue();
 
         // if the predicates are empty, match on any predicate. Otherwise, "or" the predicates.
         String predicateReg = "";

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/java/org/apache/rya/helper/TestFile.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/helper/TestFile.java b/extras/indexing/src/test/java/org/apache/rya/helper/TestFile.java
new file mode 100644
index 0000000..3aee23c
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/helper/TestFile.java
@@ -0,0 +1,53 @@
+/*
+ * 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.helper;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Holds test file information.
+ */
+public class TestFile {
+    private final String path;
+    private final int expectedCount;
+
+    /**
+     * Creates a new instance of {@link TestFile}.
+     * @param path the path of the file. (not {@code null})
+     * @param expectedCount the number of expected triples it holds.
+     */
+    public TestFile(final String path, final int expectedCount) {
+        this.path = requireNonNull(path);
+        this.expectedCount = expectedCount;
+    }
+
+    /**
+     * @return the path of the file.
+     */
+    public String getPath() {
+        return path;
+    }
+
+    /**
+     * @return the number of expected triples it holds.
+     */
+    public int getExpectedCount() {
+        return expectedCount;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/java/org/apache/rya/helper/TestFileUtils.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/helper/TestFileUtils.java b/extras/indexing/src/test/java/org/apache/rya/helper/TestFileUtils.java
new file mode 100644
index 0000000..19dafe8
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/helper/TestFileUtils.java
@@ -0,0 +1,70 @@
+/*
+ * 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.helper;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableSet;
+
+/**
+ * Holds constants relating to the RDF test files that will be used.
+ */
+public final class TestFileUtils {
+    public static final String RDF_FILE_DIR = "/rdf_format_files/";
+
+    /**
+     * All the test files that are found in
+     * "src/test/resources/rdf_format_files/" with their respective triple
+     * counts.
+     */
+    public static final Set<TestFile> TEST_FILES = ImmutableSet.of(
+            new TestFile(RDF_FILE_DIR + "ntriples_data.nt", 3),
+            new TestFile(RDF_FILE_DIR + "n3_data.n3", 12),
+            new TestFile(RDF_FILE_DIR + "rdfxml_data.owl", 2),
+            new TestFile(RDF_FILE_DIR + "turtle_data.ttl", 7),
+            new TestFile(RDF_FILE_DIR + "trig_data.trig", 5),
+            new TestFile(RDF_FILE_DIR + "trix_data.trix", 3),
+            new TestFile(RDF_FILE_DIR + "nquads_data.nq", 2),
+            new TestFile(RDF_FILE_DIR + "jsonld_data.jsonld", 3),
+            new TestFile(RDF_FILE_DIR + "rdfjson_data.rj", 1),
+            new TestFile(RDF_FILE_DIR + "binary_data.brf", 4)
+        );
+
+    /**
+     * Constant that holds the total number of triples from all the test files
+     * held in {@link #TEST_FILES}.
+     */
+    public static final int TOTAL_TRIPLES =
+        TEST_FILES.stream().map(TestFile::getExpectedCount).collect(Collectors.summingInt(Integer::intValue));
+
+    /**
+     * Convenience map to get the triple count from the test file path held in
+     * {@link #TEST_FILES}.
+     */
+    public static final Map<String, Integer> FILE_TO_COUNT_MAP =
+        TEST_FILES.stream().collect(Collectors.toMap(TestFile::getPath, TestFile::getExpectedCount));
+
+    /**
+     * Private constructor to prevent instantiation.
+     */
+    private TestFileUtils() {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/java/org/apache/rya/sail/config/AccumuloRyaSailFactoryLoadFilesIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/sail/config/AccumuloRyaSailFactoryLoadFilesIT.java b/extras/indexing/src/test/java/org/apache/rya/sail/config/AccumuloRyaSailFactoryLoadFilesIT.java
new file mode 100644
index 0000000..a75b8ba
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/sail/config/AccumuloRyaSailFactoryLoadFilesIT.java
@@ -0,0 +1,190 @@
+/*
+ * 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.sail.config;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.log4j.Logger;
+import org.apache.rya.accumulo.AccumuloRdfConfiguration;
+import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
+import org.apache.rya.api.RdfCloudTripleStoreConstants;
+import org.apache.rya.api.client.accumulo.AccumuloConnectionDetails;
+import org.apache.rya.api.client.accumulo.AccumuloRyaClientFactory;
+import org.apache.rya.helper.TestFile;
+import org.apache.rya.helper.TestFileUtils;
+import org.apache.rya.indexing.mongo.MongoPcjIntegrationTest.CountingResultHandler;
+import org.apache.rya.rdftriplestore.RyaSailRepository;
+import org.apache.rya.rdftriplestore.utils.RdfFormatUtils;
+import org.apache.rya.test.accumulo.AccumuloITBase;
+import org.eclipse.rdf4j.query.MalformedQueryException;
+import org.eclipse.rdf4j.query.QueryEvaluationException;
+import org.eclipse.rdf4j.query.QueryLanguage;
+import org.eclipse.rdf4j.query.TupleQuery;
+import org.eclipse.rdf4j.query.TupleQueryResultHandlerException;
+import org.eclipse.rdf4j.query.Update;
+import org.eclipse.rdf4j.repository.RepositoryConnection;
+import org.eclipse.rdf4j.repository.RepositoryException;
+import org.eclipse.rdf4j.repository.sail.SailRepository;
+import org.eclipse.rdf4j.repository.sail.SailRepositoryConnection;
+import org.eclipse.rdf4j.rio.RDFFormat;
+import org.eclipse.rdf4j.rio.RDFParseException;
+import org.eclipse.rdf4j.sail.Sail;
+import org.eclipse.rdf4j.sail.SailException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Tests loading files through a Accumulo {@link RyaSailFactory}.
+ */
+public class AccumuloRyaSailFactoryLoadFilesIT extends AccumuloITBase {
+    private static final Logger log = Logger.getLogger(AccumuloRyaSailFactoryLoadFilesIT.class);
+
+    private RyaSailRepository ryaRepository;
+    private AccumuloRdfConfiguration conf;
+
+    @Before
+    public void setupTest() throws Exception {
+        final AccumuloConnectionDetails connectionDetails = new AccumuloConnectionDetails(
+                getUsername(),
+                getPassword().toCharArray(),
+                getInstanceName(),
+                getZookeepers());
+
+        AccumuloRyaClientFactory.build(connectionDetails, getConnector());
+
+        conf = new AccumuloRdfConfiguration();
+        conf.setUsername(getUsername());
+        conf.setPassword(getPassword());
+        conf.setInstanceName(getInstanceName());
+        conf.setZookeepers(getZookeepers());
+        conf.setTablePrefix(getRyaInstanceName());
+
+        ryaRepository = createRyaSailRepository(conf);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        if (ryaRepository != null) {
+            close(ryaRepository);
+        }
+    }
+
+    private static RyaSailRepository createRyaSailRepository(final RdfCloudTripleStoreConfiguration config) throws SailException {
+        log.info("Connecting to Sail Repository.");
+
+        try {
+            final Sail extSail = RyaSailFactory.getInstance(config);
+            final RyaSailRepository repository = new RyaSailRepository(extSail);
+            return repository;
+        } catch (final Exception e) {
+            throw new SailException("Failed to create Rya Sail Repository", e);
+        }
+    }
+
+    /**
+     * Shuts the repository down, releasing any resources that it keeps hold of.
+     * Once shut down, the repository can no longer be used until it is
+     * re-initialized.
+     * @param repository the {@link SailRepository} to close.
+     */
+    private static void close(final SailRepository repository) {
+        if (repository != null) {
+            try {
+                repository.shutDown();
+            } catch (final RepositoryException e) {
+                log.error("Encountered an error while closing Sail Repository", e);
+            }
+        }
+    }
+
+    /**
+     * Closes the {@link SailRepositoryConnection}.
+     * @param conn the {@link SailRepositoryConnection}.
+     */
+    private static void closeQuietly(final SailRepositoryConnection conn) {
+        if (conn != null) {
+            try {
+                conn.close();
+            } catch (final RepositoryException e) {
+                // quietly absorb this exception
+            }
+        }
+    }
+
+    private static void addTriples(final SailRepository repo, final InputStream triplesStream, final RDFFormat rdfFormat) throws RDFParseException, RepositoryException, IOException {
+        SailRepositoryConnection conn = null;
+        try {
+            conn = repo.getConnection();
+            conn.begin();
+            conn.add(triplesStream, "", rdfFormat);
+            conn.commit();
+        } finally {
+            closeQuietly(conn);
+        }
+    }
+
+    private static int performTupleQuery(final String query, final RepositoryConnection conn) throws RepositoryException, MalformedQueryException, QueryEvaluationException, TupleQueryResultHandlerException {
+        final TupleQuery tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query);
+        tupleQuery.setMaxExecutionTime(10);
+        final CountingResultHandler handler = new CountingResultHandler();
+        tupleQuery.evaluate(handler);
+        return handler.getCount();
+    }
+
+    @Test
+    public void testFileLoading() throws Exception {
+        log.info("Starting file loading test...");
+        final String query = "SELECT * WHERE { ?s ?p ?o . FILTER(?p != <" + RdfCloudTripleStoreConstants.RTS_VERSION_PREDICATE + ">) }";
+        final String deleteQuery = "DELETE WHERE { ?s ?p ?o . }";
+
+        for (final TestFile testFile : TestFileUtils.TEST_FILES) {
+            final String testFilePath = testFile.getPath();
+            final RDFFormat rdfFormat = RdfFormatUtils.forFileName(testFilePath, null);
+            log.info("Loading file \"" + testFilePath + "\" with RDFFormat: " + rdfFormat.getName());
+            try (final InputStream rdfContent = getClass().getResourceAsStream(testFilePath)) {
+                addTriples(ryaRepository, rdfContent, rdfFormat);
+            }
+
+            SailRepositoryConnection queryConn = null;
+            try {
+                log.info("Querying for triples in the repository from the " + rdfFormat.getName() + " file.");
+                queryConn = ryaRepository.getConnection();
+                final int count = performTupleQuery(query, queryConn);
+                assertEquals("Expected number of triples not found in: " + testFilePath, testFile.getExpectedCount(), count);
+            } finally {
+                closeQuietly(queryConn);
+            }
+
+            SailRepositoryConnection deleteConn = null;
+            try {
+                log.info("Deleting triples in the repository from the " + rdfFormat.getName() + " file.");
+                deleteConn = ryaRepository.getConnection();
+                final Update update = deleteConn.prepareUpdate(QueryLanguage.SPARQL, deleteQuery);
+                update.execute();
+            } finally {
+                closeQuietly(deleteConn);
+            }
+        }
+        log.info("File loading test finished.");
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/java/org/apache/rya/sail/config/MongoDbRyaSailFactoryLoadFilesIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/sail/config/MongoDbRyaSailFactoryLoadFilesIT.java b/extras/indexing/src/test/java/org/apache/rya/sail/config/MongoDbRyaSailFactoryLoadFilesIT.java
new file mode 100644
index 0000000..bdca828
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/sail/config/MongoDbRyaSailFactoryLoadFilesIT.java
@@ -0,0 +1,171 @@
+/*
+ * 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.sail.config;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.log4j.Logger;
+import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
+import org.apache.rya.helper.TestFile;
+import org.apache.rya.helper.TestFileUtils;
+import org.apache.rya.indexing.mongo.MongoPcjIntegrationTest.CountingResultHandler;
+import org.apache.rya.mongodb.MongoRyaITBase;
+import org.apache.rya.rdftriplestore.RyaSailRepository;
+import org.apache.rya.rdftriplestore.utils.RdfFormatUtils;
+import org.eclipse.rdf4j.query.MalformedQueryException;
+import org.eclipse.rdf4j.query.QueryEvaluationException;
+import org.eclipse.rdf4j.query.QueryLanguage;
+import org.eclipse.rdf4j.query.TupleQuery;
+import org.eclipse.rdf4j.query.TupleQueryResultHandlerException;
+import org.eclipse.rdf4j.query.Update;
+import org.eclipse.rdf4j.repository.RepositoryConnection;
+import org.eclipse.rdf4j.repository.RepositoryException;
+import org.eclipse.rdf4j.repository.sail.SailRepository;
+import org.eclipse.rdf4j.repository.sail.SailRepositoryConnection;
+import org.eclipse.rdf4j.rio.RDFFormat;
+import org.eclipse.rdf4j.rio.RDFParseException;
+import org.eclipse.rdf4j.sail.Sail;
+import org.eclipse.rdf4j.sail.SailException;
+import org.junit.After;
+import org.junit.Test;
+
+/**
+ * Tests loading files through a MongoDB {@link RyaSailFactory}.
+ */
+public class MongoDbRyaSailFactoryLoadFilesIT extends MongoRyaITBase {
+    private static final Logger log = Logger.getLogger(MongoDbRyaSailFactoryLoadFilesIT.class);
+
+    private RyaSailRepository ryaRepository;
+
+    @Override
+    public void setupTest() throws Exception {
+        super.setupTest();
+
+        ryaRepository = createRyaSailRepository(conf);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        if (ryaRepository != null) {
+            close(ryaRepository);
+        }
+    }
+
+    private static RyaSailRepository createRyaSailRepository(final RdfCloudTripleStoreConfiguration config) throws SailException {
+        log.info("Connecting to Sail Repository.");
+
+        try {
+            final Sail extSail = RyaSailFactory.getInstance(config);
+            final RyaSailRepository repository = new RyaSailRepository(extSail);
+            return repository;
+        } catch (final Exception e) {
+            throw new SailException("Failed to create Rya Sail Repository", e);
+        }
+    }
+
+    /**
+     * Shuts the repository down, releasing any resources that it keeps hold of.
+     * Once shut down, the repository can no longer be used until it is
+     * re-initialized.
+     * @param repository the {@link SailRepository} to close.
+     */
+    private static void close(final SailRepository repository) {
+        if (repository != null) {
+            try {
+                repository.shutDown();
+            } catch (final RepositoryException e) {
+                log.error("Encountered an error while closing Sail Repository", e);
+            }
+        }
+    }
+
+    /**
+     * Closes the {@link SailRepositoryConnection}.
+     * @param conn the {@link SailRepositoryConnection}.
+     */
+    private static void closeQuietly(final SailRepositoryConnection conn) {
+        if (conn != null) {
+            try {
+                conn.close();
+            } catch (final RepositoryException e) {
+                // quietly absorb this exception
+            }
+        }
+    }
+
+    private static void addTriples(final SailRepository repo, final InputStream triplesStream, final RDFFormat rdfFormat) throws RDFParseException, RepositoryException, IOException {
+        SailRepositoryConnection conn = null;
+        try {
+            conn = repo.getConnection();
+            conn.begin();
+            conn.add(triplesStream, "", rdfFormat);
+            conn.commit();
+        } finally {
+            closeQuietly(conn);
+        }
+    }
+
+    private static int performTupleQuery(final String query, final RepositoryConnection conn) throws RepositoryException, MalformedQueryException, QueryEvaluationException, TupleQueryResultHandlerException {
+        final TupleQuery tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query);
+        tupleQuery.setMaxExecutionTime(10);
+        final CountingResultHandler handler = new CountingResultHandler();
+        tupleQuery.evaluate(handler);
+        return handler.getCount();
+    }
+
+    @Test
+    public void testFileLoading() throws Exception {
+        log.info("Starting file loading test...");
+        final String query = "SELECT * WHERE { ?s ?p ?o . }";
+        final String deleteQuery = "DELETE WHERE { ?s ?p ?o . }";
+
+        for (final TestFile testFile : TestFileUtils.TEST_FILES) {
+            final String testFilePath = testFile.getPath();
+            final RDFFormat rdfFormat = RdfFormatUtils.forFileName(testFilePath, null);
+            log.info("Loading file \"" + testFilePath + "\" with RDFFormat: " + rdfFormat.getName());
+            try (final InputStream rdfContent = getClass().getResourceAsStream(testFilePath)) {
+                addTriples(ryaRepository, rdfContent, rdfFormat);
+            }
+
+            SailRepositoryConnection queryConn = null;
+            try {
+                log.info("Querying for triples in the repository from the " + rdfFormat.getName() + " file.");
+                queryConn = ryaRepository.getConnection();
+                final int count = performTupleQuery(query, queryConn);
+                assertEquals("Expected number of triples not found in: " + testFilePath, testFile.getExpectedCount(), count);
+            } finally {
+                closeQuietly(queryConn);
+            }
+
+            SailRepositoryConnection deleteConn = null;
+            try {
+                log.info("Deleting triples in the repository from the " + rdfFormat.getName() + " file.");
+                deleteConn = ryaRepository.getConnection();
+                final Update update = deleteConn.prepareUpdate(QueryLanguage.SPARQL, deleteQuery);
+                update.execute();
+            } finally {
+                closeQuietly(deleteConn);
+            }
+        }
+        log.info("File loading test finished.");
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/resources/rdf_format_files/binary_data.brf
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/resources/rdf_format_files/binary_data.brf b/extras/indexing/src/test/resources/rdf_format_files/binary_data.brf
new file mode 100644
index 0000000..f04d0c3
Binary files /dev/null and b/extras/indexing/src/test/resources/rdf_format_files/binary_data.brf differ

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/resources/rdf_format_files/jsonld_data.jsonld
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/resources/rdf_format_files/jsonld_data.jsonld b/extras/indexing/src/test/resources/rdf_format_files/jsonld_data.jsonld
new file mode 100644
index 0000000..84025cf
--- /dev/null
+++ b/extras/indexing/src/test/resources/rdf_format_files/jsonld_data.jsonld
@@ -0,0 +1,17 @@
+{
+  "@context":
+  {
+    "name": "http://rya.apache.org/ns/name", 
+    "image": {
+      "@id": "http://rya.apache.org/ns/image", 
+      "@type": "@id" 
+    },
+    "homepage": {
+      "@id": "http://rya.apache.org/ns/url", 
+      "@type": "@id" 
+    }
+  },
+  "name": "Bob Smitch",
+  "homepage": "http://rya.apache.org/bobs_homepage",
+  "image": "http://rya.apache.org/bobs_homepage/images/bobs_party.jpg"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/resources/rdf_format_files/n3_data.n3
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/resources/rdf_format_files/n3_data.n3 b/extras/indexing/src/test/resources/rdf_format_files/n3_data.n3
new file mode 100644
index 0000000..3d48fdf
--- /dev/null
+++ b/extras/indexing/src/test/resources/rdf_format_files/n3_data.n3
@@ -0,0 +1,14 @@
+@prefix : <http://rya.apache.org/ns/sports#> .
+
+:Bob :plays :Football .
+:Susan :coaches :Football .
+:Bob :givenName "Robert" .
+:Bob :aka "Bob"@en-US .
+:Bob :aka "Robert"@en-GB .
+:Bob :aka "Roberto"@es .
+:Bob :aka "Roberto"@es-MEX .
+:Bob :aka "Roberto"@spa .
+:Bob :aka "Roberto"@spa-ESP .
+:Bob :aka "Robert"@de .
+:Bob :aka "Rupert"@de-DE .
+:Bob :aka "#!$*(&^"@und .
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/resources/rdf_format_files/nquads_data.nq
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/resources/rdf_format_files/nquads_data.nq b/extras/indexing/src/test/resources/rdf_format_files/nquads_data.nq
new file mode 100644
index 0000000..7948bff
--- /dev/null
+++ b/extras/indexing/src/test/resources/rdf_format_files/nquads_data.nq
@@ -0,0 +1,2 @@
+<http://rya.apache.org/ns/Bob> <http://rya.apache.org/ns/hasJob> <http://rya.apache.org/ns/Programmer> <http://rya.apache.org/ns/graph1> .
+<http://rya.apache.org/ns/Susan> <http://rya.apache.org/ns/hasJob> <http://rya.apache.org/ns/Manager> <http://rya.apache.org/ns/graph2> .
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/resources/rdf_format_files/ntriples_data.nt
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/resources/rdf_format_files/ntriples_data.nt b/extras/indexing/src/test/resources/rdf_format_files/ntriples_data.nt
new file mode 100644
index 0000000..bbd3b37
--- /dev/null
+++ b/extras/indexing/src/test/resources/rdf_format_files/ntriples_data.nt
@@ -0,0 +1,3 @@
+<http://rya.apache.org/ns/Bobs_Book> <http://www.w3.org/1999/02/22-rdf-syntax-ns#type> <http://rya.apache.org/ns/Book> .
+<http://rya.apache.org/ns/Bobs_Book> <http://www.w3.org/2000/01/rdf-schema#label> "The Book of Smitch" .
+<http://rya.apache.org/ns/Bobs_Book> <http://rya.apache.org/ns/publishedBy> <http://rya.apache.org/ns/DillonPublishing> .
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/resources/rdf_format_files/rdfjson_data.rj
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/resources/rdf_format_files/rdfjson_data.rj b/extras/indexing/src/test/resources/rdf_format_files/rdfjson_data.rj
new file mode 100644
index 0000000..fbb665a
--- /dev/null
+++ b/extras/indexing/src/test/resources/rdf_format_files/rdfjson_data.rj
@@ -0,0 +1,7 @@
+{
+  "http://rya.apache.org/ns/" : {
+      "http://rya.apache.org/ns/title/" : [ { "value" : "Bob's Homepage", 
+                                             "type" : "literal", 
+                                             "lang" : "en" } ] 
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/resources/rdf_format_files/rdfxml_data.owl
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/resources/rdf_format_files/rdfxml_data.owl b/extras/indexing/src/test/resources/rdf_format_files/rdfxml_data.owl
new file mode 100644
index 0000000..4d55cdb
--- /dev/null
+++ b/extras/indexing/src/test/resources/rdf_format_files/rdfxml_data.owl
@@ -0,0 +1,12 @@
+<?xml version="1.0"?>
+
+<rdf:RDF
+xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+xmlns:books="http://rya.apache.org/ns/">
+
+  <rdf:Description>
+    <books:title>Rya Rocks</books:title>
+    <books:author>Bob Smitch</books:author>
+  </rdf:Description>
+
+</rdf:RDF> 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/resources/rdf_format_files/trig_data.trig
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/resources/rdf_format_files/trig_data.trig b/extras/indexing/src/test/resources/rdf_format_files/trig_data.trig
new file mode 100644
index 0000000..c2f93b6
--- /dev/null
+++ b/extras/indexing/src/test/resources/rdf_format_files/trig_data.trig
@@ -0,0 +1,10 @@
+@prefix rdf: <http://www.w3.org/1999/02/22-rdf-syntax-ns#> .
+@prefix personnel: <http://rya.apache.org/ns/personnel/> .
+@prefix : <http://rya.apache.org/ns/> .
+
+:Graph1 { :Bob personnel:name "Bob Smitch" .
+          :Bob personnel:email <ma...@gmail.com> .
+          :Bob personnel:hasSkill personnel:Programming }
+ 
+:Graph2 { :Susan rdf:type personnel:Person .
+          :Susan personnel:hasSkill personnel:Management }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/resources/rdf_format_files/trix_data.trix
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/resources/rdf_format_files/trix_data.trix b/extras/indexing/src/test/resources/rdf_format_files/trix_data.trix
new file mode 100644
index 0000000..192be8d
--- /dev/null
+++ b/extras/indexing/src/test/resources/rdf_format_files/trix_data.trix
@@ -0,0 +1,19 @@
+<TriX>
+    <graph>
+        <triple>
+            <uri>http://rya.apache.org/ns/person#Bob</uri>
+            <uri>http://rya.apache.org/ns/relation#wife</uri>
+            <uri>http://rya.apache.org/ns/person#Susan</uri>
+        </triple>
+        <triple>
+            <uri>http://rya.apache.org/ns/person#Bob</uri>
+            <uri>http://rya.apache.org/ns/name</uri>
+            <plainLiteral>Bob</plainLiteral>
+        </triple>
+        <triple>
+            <uri>http://rya.apache.org/ns/person#Mary</uri>
+            <uri>http://rya.apache.org/ns/age</uri>
+            <typedLiteral datatype="http://www.w3.org/2001/XMLSchema#integer">32</typedLiteral>
+        </triple>
+    </graph>
+</TriX>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/extras/indexing/src/test/resources/rdf_format_files/turtle_data.ttl
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/resources/rdf_format_files/turtle_data.ttl b/extras/indexing/src/test/resources/rdf_format_files/turtle_data.ttl
new file mode 100644
index 0000000..698e3ba
--- /dev/null
+++ b/extras/indexing/src/test/resources/rdf_format_files/turtle_data.ttl
@@ -0,0 +1,11 @@
+@prefix contacts: <http://rya.apache.org/ns/contacts#> .
+
+contacts:bob contacts:homePhone       "(555) 555-1234" .
+contacts:bob contacts:emailAddress    "bobsmitch00@hotmail.com" .
+
+contacts:susan contacts:homePhone     "(555) 555-4321" .
+contacts:susan contacts:emailAddress  "susandillon@gmail.com" .
+
+contacts:ron contacts:homePhone       "(555) 555-9876" .
+contacts:ron contacts:emailAddress    "ronniejones@yahoo.com" .
+contacts:ron contacts:emailAddress    "ron.jones@hotmail.com" .
\ No newline at end of file


[3/3] incubator-rya git commit: RYA-492 Added language support for Literals; Closes #294

Posted by pu...@apache.org.
RYA-492 Added language support for Literals; Closes #294


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

Branch: refs/heads/master
Commit: 2396ebb878d4d49cb64121f0dcf7bbc7f877ebd7
Parents: 2c1efd2
Author: eric.white <Er...@parsons.com>
Authored: Fri Apr 27 15:01:34 2018 -0400
Committer: Valiyil <Pu...@parsons.com>
Committed: Wed May 16 11:48:24 2018 -0400

----------------------------------------------------------------------
 common/rya.api/pom.xml                          |  13 +
 .../java/org/apache/rya/api/domain/RyaType.java | 114 +++--
 .../rya/api/resolver/RdfToRyaConversions.java   | 114 +++--
 .../rya/api/resolver/RyaToRdfConversions.java   |  99 ++--
 .../resolver/impl/CustomDatatypeResolver.java   |  28 +-
 .../api/resolver/impl/RyaTypeResolverImpl.java  |  25 +-
 .../rya/api/utils/LiteralLanguageUtils.java     |  75 +++
 .../org/apache/rya/api/domain/RyaTypeTest.java  |  33 +-
 .../api/resolver/LanguageCodesTestHelper.java   | 123 +++++
 .../api/resolver/RdfToRyaConversionsTest.java   | 133 +++++
 .../apache/rya/api/resolver/RyaContextTest.java |  54 +-
 .../api/resolver/RyaToRdfConversionsTest.java   | 127 +++++
 .../test/resources/ISO-3166-1_Country_Codes.txt | 236 +++++++++
 .../test/resources/ISO-639-1_Language_Codes.txt | 191 +++++++
 .../test/resources/ISO-639-2_Language_Codes.txt | 504 +++++++++++++++++++
 .../AggregationPipelineQueryNode.java           | 225 +++++----
 .../dao/SimpleMongoDBStorageStrategy.java       |  21 +-
 .../SimpleMongoDBStorageStrategyTest.java       |  55 +-
 extras/indexing/pom.xml                         |   1 +
 .../rya/indexing/StatementSerializer.java       |  93 ++--
 .../java/org/apache/rya/helper/TestFile.java    |  53 ++
 .../org/apache/rya/helper/TestFileUtils.java    |  70 +++
 .../AccumuloRyaSailFactoryLoadFilesIT.java      | 190 +++++++
 .../MongoDbRyaSailFactoryLoadFilesIT.java       | 171 +++++++
 .../resources/rdf_format_files/binary_data.brf  | Bin 0 -> 757 bytes
 .../rdf_format_files/jsonld_data.jsonld         |  17 +
 .../test/resources/rdf_format_files/n3_data.n3  |  14 +
 .../resources/rdf_format_files/nquads_data.nq   |   2 +
 .../resources/rdf_format_files/ntriples_data.nt |   3 +
 .../resources/rdf_format_files/rdfjson_data.rj  |   7 +
 .../resources/rdf_format_files/rdfxml_data.owl  |  12 +
 .../resources/rdf_format_files/trig_data.trig   |  10 +
 .../resources/rdf_format_files/trix_data.trix   |  19 +
 .../resources/rdf_format_files/turtle_data.ttl  |  11 +
 .../GeoTemporalMongoDBStorageStrategyTest.java  |  19 +-
 .../java/org/apache/rya/reasoning/Fact.java     |  87 ++--
 .../apache/rya/accumulo/mr/RyaTypeWritable.java |  34 +-
 .../rya/accumulo/mr/GraphXInputFormatTest.java  |  50 +-
 .../pig/StatementPatternStorageTest.java        |  76 +--
 .../RyaSailRepositoryConnection.java            |  51 +-
 40 files changed, 2704 insertions(+), 456 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/pom.xml
----------------------------------------------------------------------
diff --git a/common/rya.api/pom.xml b/common/rya.api/pom.xml
index ea4fb19..5611132 100644
--- a/common/rya.api/pom.xml
+++ b/common/rya.api/pom.xml
@@ -115,6 +115,19 @@ under the License.
     </dependencies>
 
     <build>
+        <pluginManagement>
+            <plugins>
+                <plugin>
+                    <groupId>org.apache.rat</groupId>
+                    <artifactId>apache-rat-plugin</artifactId>
+                    <configuration>
+                        <excludes>
+                            <exclude>**/src/test/resources/**</exclude>
+                        </excludes>
+                    </configuration>
+                </plugin>
+            </plugins>
+        </pluginManagement>
         <plugins>
             <plugin>
                 <artifactId>maven-jar-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/main/java/org/apache/rya/api/domain/RyaType.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/org/apache/rya/api/domain/RyaType.java b/common/rya.api/src/main/java/org/apache/rya/api/domain/RyaType.java
index 8de4667..e40c8d6 100644
--- a/common/rya.api/src/main/java/org/apache/rya/api/domain/RyaType.java
+++ b/common/rya.api/src/main/java/org/apache/rya/api/domain/RyaType.java
@@ -18,6 +18,9 @@
  */
 package org.apache.rya.api.domain;
 
+import java.util.Objects;
+
+import org.apache.commons.lang.builder.CompareToBuilder;
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.eclipse.rdf4j.model.IRI;
 import org.eclipse.rdf4j.model.vocabulary.XMLSchema;
@@ -27,23 +30,47 @@ import org.eclipse.rdf4j.model.vocabulary.XMLSchema;
  * Date: 7/16/12
  * Time: 11:45 AM
  */
-public class RyaType implements Comparable {
+public class RyaType implements Comparable<RyaType> {
 
     private IRI dataType;
     private String data;
+    private String language;
 
+    /**
+     * Creates a new instance of {@link RyaType}.
+     */
     public RyaType() {
-        setDataType(XMLSchema.STRING);
+        this(null);
     }
 
+    /**
+     * Creates a new instance of {@link RyaType} of type
+     * {@link XMLSchema#STRING} and with no language.
+     * @param data the data string.
+     */
     public RyaType(final String data) {
         this(XMLSchema.STRING, data);
     }
 
-
+    /**
+     * Creates a new instance of {@link RyaType} with no language.
+     * @param dataType the {@link IRI} data type.
+     * @param data the data string.
+     */
     public RyaType(final IRI dataType, final String data) {
-        setDataType(dataType);
-        setData(data);
+        this(dataType, data, null);
+    }
+
+    /**
+     * Creates a new instance of {@link RyaType}.
+     * @param dataType the {@link IRI} data type.
+     * @param data the data string.
+     * @param language the language code.
+     */
+    public RyaType(final IRI dataType, final String data, final String language) {
+        this.dataType = dataType;
+        this.data = data;
+        this.language = language;
     }
 
     /**
@@ -67,20 +94,40 @@ public class RyaType implements Comparable {
         this.data = data;
     }
 
+    /**
+     * @return the language code.
+     */
+    public String getLanguage() {
+        return language;
+    }
+
+    /**
+     * Sets the language code.
+     * @param language the language code.
+     */
+    public void setLanguage(final String language) {
+        this.language = language;
+    }
+
     @Override
     public String toString() {
         final StringBuilder sb = new StringBuilder();
         sb.append("RyaType");
         sb.append("{dataType=").append(dataType);
         sb.append(", data='").append(data).append('\'');
+        if (language != null) {
+            sb.append(", language='").append(language).append('\'');
+        }
         sb.append('}');
         return sb.toString();
     }
 
     /**
-     * Determine equality based on string representations of data and datatype.
+     * Determine equality based on string representations of data, datatype, and
+     * language.
      * @param o The object to compare with
-     * @return true if the other object is also a RyaType and both data and datatype match.
+     * @return {@code true} if the other object is also a RyaType and the data,
+     * datatype, and language all match.
      */
     @Override
     public boolean equals(final Object o) {
@@ -93,53 +140,40 @@ public class RyaType implements Comparable {
         final RyaType other = (RyaType) o;
         final EqualsBuilder builder = new EqualsBuilder()
                 .append(getData(), other.getData())
-                .append(getDataType(), other.getDataType());
+                .append(getDataType(), other.getDataType())
+                .append(getLanguage(), other.getLanguage());
         return builder.isEquals();
     }
 
     /**
-     * Generate a hash based on the string representations of both data and datatype.
+     * Generate a hash based on the string representations of data, datatype,
+     * and language.
      * @return A hash consistent with equals.
      */
     @Override
     public int hashCode() {
-        int result = dataType != null ? dataType.hashCode() : 0;
-        result = 31 * result + (data != null ? data.hashCode() : 0);
-        return result;
+        return Objects.hash(dataType, data, language);
     }
 
     /**
-     * Define a natural ordering based on data and datatype.
+     * Define a natural ordering based on data, datatype, and language.
      * @param o The object to compare with
-     * @return 0 if both the data string and the datatype string representation match between the objects,
-     *          where matching is defined by string comparison or both being null;
-     *          Otherwise, an integer whose sign yields a consistent ordering.
+     * @return 0 if the data string, the datatype string, and the language
+     * string representation match between the objects, where matching is
+     * defined by string comparison or all being null;
+     * Otherwise, an integer whose sign yields a consistent ordering.
      */
     @Override
-    public int compareTo(final Object o) {
-        int result = -1;
-        if (o != null && o instanceof RyaType) {
-            result = 0;
-            final RyaType other = (RyaType) o;
-            if (this.data != other.data) {
-                if (this.data == null) {
-                    return 1;
-                }
-                if (other.data == null) {
-                    return -1;
-                }
-                result = this.data.compareTo(other.data);
-            }
-            if (result == 0 && this.dataType != other.dataType) {
-                if (this.dataType == null) {
-                    return 1;
-                }
-                if (other.dataType == null) {
-                    return -1;
-                }
-                result = this.dataType.toString().compareTo(other.dataType.toString());
-            }
+    public int compareTo(final RyaType o) {
+        if (o == null) {
+            return 1;
         }
-        return result;
+        final String dataTypeStr = getDataType() != null ? getDataType().stringValue() : null;
+        final String otherDataTypeStr = o.getDataType() != null ? o.getDataType().stringValue() : null;
+        final CompareToBuilder builder = new CompareToBuilder()
+                .append(getData(), o.getData())
+                .append(dataTypeStr, otherDataTypeStr)
+                .append(getLanguage(), o.getLanguage());
+        return builder.toComparison();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/main/java/org/apache/rya/api/resolver/RdfToRyaConversions.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/org/apache/rya/api/resolver/RdfToRyaConversions.java b/common/rya.api/src/main/java/org/apache/rya/api/resolver/RdfToRyaConversions.java
index 0618c82..8554e77 100644
--- a/common/rya.api/src/main/java/org/apache/rya/api/resolver/RdfToRyaConversions.java
+++ b/common/rya.api/src/main/java/org/apache/rya/api/resolver/RdfToRyaConversions.java
@@ -1,5 +1,3 @@
-package org.apache.rya.api.resolver;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -8,9 +6,9 @@ package org.apache.rya.api.resolver;
  * 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
@@ -18,57 +16,97 @@ package org.apache.rya.api.resolver;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.api.resolver;
+
+import static org.apache.rya.api.utils.LiteralLanguageUtils.UNDETERMINED_LANGUAGE;
 
+import org.apache.log4j.Logger;
 import org.apache.rya.api.domain.RangeIRI;
 import org.apache.rya.api.domain.RangeValue;
+import org.apache.rya.api.domain.RyaIRI;
+import org.apache.rya.api.domain.RyaIRIRange;
 import org.apache.rya.api.domain.RyaSchema;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaType;
 import org.apache.rya.api.domain.RyaTypeRange;
-import org.apache.rya.api.domain.RyaIRI;
-import org.apache.rya.api.domain.RyaIRIRange;
+import org.apache.rya.api.log.LogUtils;
 import org.eclipse.rdf4j.model.BNode;
 import org.eclipse.rdf4j.model.IRI;
 import org.eclipse.rdf4j.model.Literal;
 import org.eclipse.rdf4j.model.Resource;
 import org.eclipse.rdf4j.model.Statement;
 import org.eclipse.rdf4j.model.Value;
+import org.eclipse.rdf4j.model.util.Literals;
 
 /**
- * Date: 7/17/12
- * Time: 8:34 AM
+ * Methods for converting values from their RDF4J object representations into
+ * their Rya object equivalents.
  */
 public class RdfToRyaConversions {
+    private static final Logger log = Logger.getLogger(RdfToRyaConversions.class);
 
-    public static RyaIRI convertIRI(IRI iri) {
-        if (iri == null) return null;
+    /**
+     * Converts a {@link IRI} into a {@link RyaIRI} representation of the
+     * {@code iri}.
+     * @param iri the {@link IRI} to convert.
+     * @return the {@link RyaIRI} representation of the {@code iri}.
+     */
+    public static RyaIRI convertIRI(final IRI iri) {
+        if (iri == null) {
+            return null;
+        }
         if (iri instanceof RangeIRI) {
-            RangeIRI riri = (RangeIRI) iri;
+            final RangeIRI riri = (RangeIRI) iri;
             return new RyaIRIRange(convertIRI(riri.getStart()), convertIRI(riri.getEnd()));
         }
         return new RyaIRI(iri.stringValue());
     }
 
-    public static RyaType convertLiteral(Literal literal) {
-        if (literal == null) return null;
+    /**
+     * Converts a {@link Literal} into a {@link RyaType} representation of the
+     * {@code literal}.
+     * @param literal the {@link Literal} to convert.
+     * @return the {@link RyaType} representation of the {@code literal}.
+     */
+    public static RyaType convertLiteral(final Literal literal) {
+        if (literal == null) {
+            return null;
+        }
         if (literal.getDatatype() != null) {
+            if (Literals.isLanguageLiteral(literal)) {
+                final String language = literal.getLanguage().get();
+                if (Literals.isValidLanguageTag(language)) {
+                    return new RyaType(literal.getDatatype(), literal.stringValue(), language);
+                } else {
+                    log.warn("Invalid language (" + LogUtils.clean(language) + ") found in Literal. Defaulting to: " + UNDETERMINED_LANGUAGE);
+                    // Replace invalid language with "und"
+                    return new RyaType(literal.getDatatype(), literal.stringValue(), UNDETERMINED_LANGUAGE);
+                }
+            }
             return new RyaType(literal.getDatatype(), literal.stringValue());
         }
-        //no language literal conversion yet
         return new RyaType(literal.stringValue());
     }
 
-    public static RyaType convertValue(Value value) {
-        if (value == null) return null;
+    /**
+     * Converts a {@link Value} into a {@link RyaType} representation of the
+     * {@code value}.
+     * @param value the {@link Value} to convert.
+     * @return the {@link RyaType} representation of the {@code value}.
+     */
+    public static RyaType convertValue(final Value value) {
+        if (value == null) {
+            return null;
+        }
         //assuming either IRI or Literal here
-        if(value instanceof Resource) {
+        if (value instanceof Resource) {
             return convertResource((Resource) value);
         }
         if (value instanceof Literal) {
             return convertLiteral((Literal) value);
         }
         if (value instanceof RangeValue) {
-            RangeValue<?> rv = (RangeValue<?>) value;
+            final RangeValue<?> rv = (RangeValue<?>) value;
             if (rv.getStart() instanceof IRI) {
                 return new RyaIRIRange(convertIRI((IRI) rv.getStart()), convertIRI((IRI) rv.getEnd()));
             } else {
@@ -79,25 +117,41 @@ public class RdfToRyaConversions {
         return null;
     }
 
-    public static RyaIRI convertResource(Resource subject) {
-        if(subject == null) return null;
-        if (subject instanceof BNode) {
-            return new RyaIRI(RyaSchema.BNODE_NAMESPACE + ((BNode) subject).getID());
+    /**
+     * Converts a {@link Resource} into a {@link RyaIRI} representation of the
+     * {@code resource}.
+     * @param resource the {@link Resource} to convert. Generally this will be
+     * the subject.
+     * @return the {@link RyaIRI} representation of the {@code resource}.
+     */
+    public static RyaIRI convertResource(final Resource resource) {
+        if (resource == null) {
+            return null;
         }
-        return convertIRI((IRI) subject);
+        if (resource instanceof BNode) {
+            return new RyaIRI(RyaSchema.BNODE_NAMESPACE + ((BNode) resource).getID());
+        }
+        return convertIRI((IRI) resource);
     }
 
-    public static RyaStatement convertStatement(Statement statement) {
-        if (statement == null) return null;
-        Resource subject = statement.getSubject();
-        IRI predicate = statement.getPredicate();
-        Value object = statement.getObject();
-        Resource context = statement.getContext();
+    /**
+     * Converts a {@link Statement} into a {@link RyaStatement} representation
+     * of the {@code statement}.
+     * @param statement the {@link Statement} to convert.
+     * @return the {@link RyaStatement} representation of the {@code statement}.
+     */
+    public static RyaStatement convertStatement(final Statement statement) {
+        if (statement == null) {
+            return null;
+        }
+        final Resource subject = statement.getSubject();
+        final IRI predicate = statement.getPredicate();
+        final Value object = statement.getObject();
+        final Resource context = statement.getContext();
         return new RyaStatement(
                 convertResource(subject),
                 convertIRI(predicate),
                 convertValue(object),
                 convertResource(context));
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/main/java/org/apache/rya/api/resolver/RyaToRdfConversions.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/org/apache/rya/api/resolver/RyaToRdfConversions.java b/common/rya.api/src/main/java/org/apache/rya/api/resolver/RyaToRdfConversions.java
index 88b79bf..27d329a 100644
--- a/common/rya.api/src/main/java/org/apache/rya/api/resolver/RyaToRdfConversions.java
+++ b/common/rya.api/src/main/java/org/apache/rya/api/resolver/RyaToRdfConversions.java
@@ -1,5 +1,3 @@
-package org.apache.rya.api.resolver;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -8,9 +6,9 @@ package org.apache.rya.api.resolver;
  * 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
@@ -18,59 +16,98 @@ package org.apache.rya.api.resolver;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.api.resolver;
 
+import org.apache.rya.api.domain.RyaIRI;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaType;
-import org.apache.rya.api.domain.RyaIRI;
+import org.apache.rya.api.utils.LiteralLanguageUtils;
 import org.eclipse.rdf4j.model.IRI;
 import org.eclipse.rdf4j.model.Literal;
 import org.eclipse.rdf4j.model.Statement;
 import org.eclipse.rdf4j.model.Value;
 import org.eclipse.rdf4j.model.ValueFactory;
 import org.eclipse.rdf4j.model.impl.SimpleValueFactory;
+import org.eclipse.rdf4j.model.util.Literals;
+import org.eclipse.rdf4j.model.vocabulary.RDF;
 import org.eclipse.rdf4j.model.vocabulary.XMLSchema;
 
 /**
- * Date: 7/17/12
- * Time: 8:34 AM
+ * Methods for converting values from their Rya object representations into
+ * their RDF4J object equivalents.
  */
 public class RyaToRdfConversions {
     private static final ValueFactory VF = SimpleValueFactory.getInstance();
 
-    public static IRI convertIRI(RyaIRI iri) {
-        return VF.createIRI(iri.getData());
+    /**
+     * Converts a {@link RyaIRI} into a {@link IRI} representation of the
+     * {@code ryaIri}.
+     * @param ryaIri the {@link RyaIRI} to convert.
+     * @return the {@link IRI} representation of the {@code ryaIri}.
+     */
+    public static IRI convertIRI(final RyaIRI ryaIri) {
+        return VF.createIRI(ryaIri.getData());
     }
-    
-    private static IRI convertIRI(RyaType value) {
-        return VF.createIRI(value.getData());
+
+    /**
+     * Converts a {@link RyaType} into a {@link IRI} representation of the
+     * {@code ryaType}.
+     * @param ryaType the {@link RyaType} to convert.
+     * @return the {@link IRI} representation of the {@code ryaType}.
+     */
+    private static IRI convertIRI(final RyaType ryaType) {
+        return VF.createIRI(ryaType.getData());
     }
 
-    public static Literal convertLiteral(RyaType literal) {
-        if (XMLSchema.STRING.equals(literal.getDataType())) {
-            return VF.createLiteral(literal.getData());
-        } else {
-            return VF.createLiteral(literal.getData(), literal.getDataType());
+    /**
+     * Converts a {@link RyaType} into a {@link Literal} representation of the
+     * {@code ryaType}.
+     * @param ryaType the {@link RyaType} to convert.
+     * @return the {@link Literal} representation of the {@code ryaType}.
+     */
+    public static Literal convertLiteral(final RyaType ryaType) {
+        if (XMLSchema.STRING.equals(ryaType.getDataType())) {
+            return VF.createLiteral(ryaType.getData());
+        } else if (RDF.LANGSTRING.equals(ryaType.getDataType())) {
+            final String data = ryaType.getData();
+            final String language = ryaType.getLanguage();
+            if (language != null && Literals.isValidLanguageTag(language)) {
+                return VF.createLiteral(data, language);
+            } else {
+                return VF.createLiteral(data, LiteralLanguageUtils.UNDETERMINED_LANGUAGE);
+            }
         }
-        //TODO: No Language support yet
+        return VF.createLiteral(ryaType.getData(), ryaType.getDataType());
     }
 
-    public static Value convertValue(RyaType value) {
+    /**
+     * Converts a {@link RyaType} into a {@link Value} representation of the
+     * {@code ryaType}.
+     * @param ryaType the {@link RyaType} to convert.
+     * @return the {@link Value} representation of the {@code ryaType}.
+     */
+    public static Value convertValue(final RyaType ryaType) {
         //assuming either IRI or Literal here
-        return (value instanceof RyaIRI || value.getDataType().equals(XMLSchema.ANYURI)) ? convertIRI(value) : convertLiteral(value);
+        return (ryaType instanceof RyaIRI || ryaType.getDataType().equals(XMLSchema.ANYURI)) ? convertIRI(ryaType) : convertLiteral(ryaType);
     }
 
-    public static Statement convertStatement(RyaStatement statement) {
-        assert statement != null;
-        if (statement.getContext() != null) {
-            return VF.createStatement(convertIRI(statement.getSubject()),
-                    convertIRI(statement.getPredicate()),
-                    convertValue(statement.getObject()),
-                    convertIRI(statement.getContext()));
+    /**
+     * Converts a {@link RyaStatement} into a {@link Statement} representation
+     * of the {@code ryaStatement}.
+     * @param ryaStatement the {@link RyaStatement} to convert.
+     * @return the {@link Statement} representation of the {@code ryaStatement}.
+     */
+    public static Statement convertStatement(final RyaStatement ryaStatement) {
+        assert ryaStatement != null;
+        if (ryaStatement.getContext() != null) {
+            return VF.createStatement(convertIRI(ryaStatement.getSubject()),
+                    convertIRI(ryaStatement.getPredicate()),
+                    convertValue(ryaStatement.getObject()),
+                    convertIRI(ryaStatement.getContext()));
         } else {
-            return VF.createStatement(convertIRI(statement.getSubject()),
-                    convertIRI(statement.getPredicate()),
-                    convertValue(statement.getObject()));
+            return VF.createStatement(convertIRI(ryaStatement.getSubject()),
+                    convertIRI(ryaStatement.getPredicate()),
+                    convertValue(ryaStatement.getObject()));
         }
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/main/java/org/apache/rya/api/resolver/impl/CustomDatatypeResolver.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/org/apache/rya/api/resolver/impl/CustomDatatypeResolver.java b/common/rya.api/src/main/java/org/apache/rya/api/resolver/impl/CustomDatatypeResolver.java
index 075b3f8..da38a8c 100644
--- a/common/rya.api/src/main/java/org/apache/rya/api/resolver/impl/CustomDatatypeResolver.java
+++ b/common/rya.api/src/main/java/org/apache/rya/api/resolver/impl/CustomDatatypeResolver.java
@@ -25,7 +25,10 @@ import java.nio.charset.StandardCharsets;
 
 import org.apache.rya.api.domain.RyaType;
 import org.apache.rya.api.resolver.RyaTypeResolverException;
+import org.apache.rya.api.utils.LiteralLanguageUtils;
 import org.eclipse.rdf4j.model.impl.SimpleValueFactory;
+import org.eclipse.rdf4j.model.util.Literals;
+import org.eclipse.rdf4j.model.vocabulary.RDF;
 
 import com.google.common.primitives.Bytes;
 
@@ -42,7 +45,15 @@ public class CustomDatatypeResolver extends RyaTypeResolverImpl {
 
     @Override
     public byte[][] serializeType(final RyaType ryaType) throws RyaTypeResolverException {
-        final byte[] bytes = serializeData(ryaType.getData()).getBytes(StandardCharsets.UTF_8);
+        final StringBuilder dataBuilder = new StringBuilder();
+        dataBuilder.append(ryaType.getData());
+        final String validatedLanguage = LiteralLanguageUtils.validateLanguage(ryaType.getLanguage(), ryaType.getDataType());
+        if (validatedLanguage != null) {
+            dataBuilder.append(LiteralLanguageUtils.LANGUAGE_DELIMITER);
+            dataBuilder.append(validatedLanguage);
+        }
+        // Combine data and language
+        final byte[] bytes = serializeData(dataBuilder.toString()).getBytes(StandardCharsets.UTF_8);
         return new byte[][]{bytes, Bytes.concat(TYPE_DELIM_BYTES, ryaType.getDataType().stringValue().getBytes(StandardCharsets.UTF_8), TYPE_DELIM_BYTES, markerBytes)};
     }
 
@@ -63,9 +74,20 @@ public class CustomDatatypeResolver extends RyaTypeResolverImpl {
         if (indexOfType < 1) {
             throw new RyaTypeResolverException("Not a datatype literal");
         }
-        final String label = deserializeData(new String(bytes, 0, indexOfType, StandardCharsets.UTF_8));
+        String data = deserializeData(new String(bytes, 0, indexOfType, StandardCharsets.UTF_8));
         rt.setDataType(SimpleValueFactory.getInstance().createIRI(new String(bytes, indexOfType + 1, (length - indexOfType) - 3, StandardCharsets.UTF_8)));
-        rt.setData(label);
+        if (RDF.LANGSTRING.equals(rt.getDataType())) {
+            final int langDelimiterPos = data.lastIndexOf(LiteralLanguageUtils.LANGUAGE_DELIMITER);
+            final String parsedData = data.substring(0, langDelimiterPos);
+            final String language = data.substring(langDelimiterPos + 1, data.length());
+            if (language != null && Literals.isValidLanguageTag(language)) {
+                rt.setLanguage(language);
+            } else {
+                rt.setLanguage(LiteralLanguageUtils.UNDETERMINED_LANGUAGE);
+            }
+            data = parsedData;
+        }
+        rt.setData(data);
         return rt;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/main/java/org/apache/rya/api/resolver/impl/RyaTypeResolverImpl.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/org/apache/rya/api/resolver/impl/RyaTypeResolverImpl.java b/common/rya.api/src/main/java/org/apache/rya/api/resolver/impl/RyaTypeResolverImpl.java
index fba7a29..8b45221 100644
--- a/common/rya.api/src/main/java/org/apache/rya/api/resolver/impl/RyaTypeResolverImpl.java
+++ b/common/rya.api/src/main/java/org/apache/rya/api/resolver/impl/RyaTypeResolverImpl.java
@@ -27,9 +27,12 @@ import org.apache.rya.api.domain.RyaRange;
 import org.apache.rya.api.domain.RyaType;
 import org.apache.rya.api.resolver.RyaTypeResolver;
 import org.apache.rya.api.resolver.RyaTypeResolverException;
+import org.apache.rya.api.utils.LiteralLanguageUtils;
 import org.calrissian.mango.types.LexiTypeEncoders;
 import org.calrissian.mango.types.TypeEncoder;
 import org.eclipse.rdf4j.model.IRI;
+import org.eclipse.rdf4j.model.util.Literals;
+import org.eclipse.rdf4j.model.vocabulary.RDF;
 import org.eclipse.rdf4j.model.vocabulary.XMLSchema;
 
 import com.google.common.primitives.Bytes;
@@ -79,7 +82,14 @@ public class RyaTypeResolverImpl implements RyaTypeResolver {
 
     @Override
     public byte[][] serializeType(final RyaType ryaType) throws RyaTypeResolverException {
-        final byte[] bytes = serializeData(ryaType.getData()).getBytes(StandardCharsets.UTF_8);
+        final StringBuilder dataBuilder = new StringBuilder();
+        dataBuilder.append(ryaType.getData());
+        final String validatedLanguage = LiteralLanguageUtils.validateLanguage(ryaType.getLanguage(), dataType);
+        if (validatedLanguage != null) {
+            dataBuilder.append(LiteralLanguageUtils.LANGUAGE_DELIMITER);
+            dataBuilder.append(validatedLanguage);
+        }
+        final byte[] bytes = serializeData(dataBuilder.toString()).getBytes(StandardCharsets.UTF_8);
         return new byte[][]{bytes, Bytes.concat(TYPE_DELIM_BYTES, markerBytes)};
     }
 
@@ -113,7 +123,18 @@ public class RyaTypeResolverImpl implements RyaTypeResolver {
         }
         final RyaType rt = newInstance();
         rt.setDataType(getRyaDataType());
-        final String data = new String(bytes, 0, bytes.length - 2, StandardCharsets.UTF_8);
+        String data = new String(bytes, 0, bytes.length - 2, StandardCharsets.UTF_8);
+        if (RDF.LANGSTRING.equals(rt.getDataType())) {
+            final int langDelimiterPos = data.lastIndexOf(LiteralLanguageUtils.LANGUAGE_DELIMITER);
+            final String parsedData = data.substring(0, langDelimiterPos);
+            final String language = data.substring(langDelimiterPos + 1, data.length());
+            if (language != null && Literals.isValidLanguageTag(language)) {
+                rt.setLanguage(language);
+            } else {
+                rt.setLanguage(LiteralLanguageUtils.UNDETERMINED_LANGUAGE);
+            }
+            data = parsedData;
+        }
         rt.setData(deserializeData(data));
         return rt;
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/main/java/org/apache/rya/api/utils/LiteralLanguageUtils.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/org/apache/rya/api/utils/LiteralLanguageUtils.java b/common/rya.api/src/main/java/org/apache/rya/api/utils/LiteralLanguageUtils.java
new file mode 100644
index 0000000..d1c81ee
--- /dev/null
+++ b/common/rya.api/src/main/java/org/apache/rya/api/utils/LiteralLanguageUtils.java
@@ -0,0 +1,75 @@
+/*
+ * 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.api.utils;
+
+import org.eclipse.rdf4j.model.IRI;
+import org.eclipse.rdf4j.model.Literal;
+import org.eclipse.rdf4j.model.util.Literals;
+import org.eclipse.rdf4j.model.vocabulary.RDF;
+
+/**
+ * Utility methods and constants for {@link Literal} languages.
+ */
+public final class LiteralLanguageUtils {
+    /**
+     * Special identifier used if there is language content, but the specific
+     * language cannot be determined. Should be avoided if possible.
+     * See <a href=https://www.loc.gov/standards/iso639-2/faq.html#25>here</a>
+     * for more info.
+     */
+    public static final String UNDETERMINED_LANGUAGE = "und";
+
+    /**
+     * Delimiter between the data and the language tag.
+     */
+    public static final String LANGUAGE_DELIMITER = "@";
+
+    /**
+     * Private constructor to prevent instantiation.
+     */
+    private LiteralLanguageUtils() {
+    }
+
+    /**
+     * Validates the language based on the data type.
+     * <p>
+     * This will do one of the following:
+     * <ul>
+     *   <li>Return the original {@code language} if the {@code dataType} is
+     *       {@link RDF#LANGSTRING} and it's of a VALID format.</li>
+     *   <li>Returns {@link UNDETERMINED_LANGUAGE} if the {@code dataType} is
+     *       {@link RDF#LANGSTRING} and it's of an INVALID format.</li>
+     *   <li>Return {@code null} if the dataType is NOT {@link RDF#LANGSTRING}.</li>
+     * </ul>
+     * @param language the language to validate.
+     * @param dataType the {@link IRI} data type to validate against.
+     * @return the validated language.
+     */
+    public static String validateLanguage(final String language, final IRI dataType) {
+        String result = null;
+        if (RDF.LANGSTRING.equals(dataType)) {
+            if (language != null && Literals.isValidLanguageTag(language)) {
+                result = language;
+            } else {
+                result = UNDETERMINED_LANGUAGE;
+            }
+        }
+        return result;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/test/java/org/apache/rya/api/domain/RyaTypeTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/test/java/org/apache/rya/api/domain/RyaTypeTest.java b/common/rya.api/src/test/java/org/apache/rya/api/domain/RyaTypeTest.java
index 7cfc77b..53d02cd 100644
--- a/common/rya.api/src/test/java/org/apache/rya/api/domain/RyaTypeTest.java
+++ b/common/rya.api/src/test/java/org/apache/rya/api/domain/RyaTypeTest.java
@@ -1,5 +1,3 @@
-package org.apache.rya.api.domain;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -18,7 +16,9 @@ package org.apache.rya.api.domain;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.api.domain;
 
+import org.eclipse.rdf4j.model.vocabulary.RDF;
 import org.eclipse.rdf4j.model.vocabulary.XMLSchema;
 import org.junit.Assert;
 import org.junit.Test;
@@ -29,8 +29,12 @@ public class RyaTypeTest {
     static RyaType c = new RyaType(XMLSchema.STRING, "http://www.example.com/Carol");
     static RyaType aIri = new RyaType(XMLSchema.ANYURI, "http://www.example.com/Alice");
     static RyaType bIri = new RyaType(XMLSchema.ANYURI, "http://www.example.com/Bob");
+    static RyaType aLang = new RyaType(RDF.LANGSTRING, "http://www.example.com/Alice", "en");
+    static RyaType aDiffLang = new RyaType(RDF.LANGSTRING, "http://www.example.com/Alice", "fr");
+    static RyaType bLang = new RyaType(RDF.LANGSTRING, "http://www.example.com/Bob", "en");
     RyaType nullData = new RyaType(XMLSchema.STRING, null);
     RyaType nullType = new RyaType(null, "http://www.example.com/Alice");
+    RyaType nullLang = new RyaType(RDF.LANGSTRING, "http://www.example.com/Alice", null);
     RyaType nullBoth = new RyaType(null, null);
     RyaType same = new RyaType(XMLSchema.STRING, "http://www.example.com/Alice");
 
@@ -42,17 +46,21 @@ public class RyaTypeTest {
         Assert.assertFalse("compareTo should return nonzero for same datatype and different data.", bIri.compareTo(aIri) == 0);
         Assert.assertEquals("compareTo should return zero for different objects with matching data and datatype.",
                 0, a.compareTo(same));
+        Assert.assertEquals("compareTo(self) return zero.", 0, aLang.compareTo(aLang));
+        Assert.assertFalse("compareTo should return nonzero for different languages.", aLang.compareTo(aDiffLang) == 0);
+        Assert.assertFalse("compareTo should return nonzero for same datatype and language and different data.", aLang.compareTo(bLang) == 0);
     }
 
     @Test
     public void testCompareToNullFields() throws Exception {
-        Assert.assertEquals("[has no nulls].compareTo([has null data]) should return -1", -1, a.compareTo(nullData));
-        Assert.assertEquals("[has no nulls].compareTo([has null type]) should return -1 if data is equal",
-                -1, a.compareTo(nullType));
-        Assert.assertEquals("[has null data].compareTo([has no nulls]) should return 1", 1, nullData.compareTo(a));
-        Assert.assertEquals("[has null type].compareTo([has no nulls]) should return 1 if data is equal",
-                 1, nullType.compareTo(a));
-        Assert.assertEquals("[has null type].compareTo([has null data]) should return -1", -1, nullType.compareTo(nullData));
+        Assert.assertEquals("[has no nulls].compareTo([has null data]) should return 1", 1, a.compareTo(nullData));
+        Assert.assertEquals("[has no nulls].compareTo([has null type]) should return 1 if data is equal",
+                1, a.compareTo(nullType));
+        Assert.assertEquals("[has null data].compareTo([has no nulls]) should return -1", -1, nullData.compareTo(a));
+        Assert.assertEquals("[has null type].compareTo([has no nulls]) should return -1 if data is equal",
+                 -1, nullType.compareTo(a));
+        Assert.assertEquals("[has null type].compareTo([has null data]) should return 1", 1, nullType.compareTo(nullData));
+        Assert.assertEquals("[has no nulls].compareTo([has null lang]) should return 1", 1, aLang.compareTo(nullLang));
     }
 
     @Test
@@ -71,7 +79,7 @@ public class RyaTypeTest {
 
     @Test
     public void testCompareToTransitive() throws Exception {
-        int sign = Integer.signum(a.compareTo(b));
+        final int sign = Integer.signum(a.compareTo(b));
         Assert.assertEquals("compareTo(a,b) and compareTo(b,c) should have the same sign.",
                 sign, Integer.signum(b.compareTo(c)));
         Assert.assertEquals("if a > b > c, compareTo(a,c) should be consistent.", sign, Integer.signum(a.compareTo(c)));
@@ -89,7 +97,8 @@ public class RyaTypeTest {
         Assert.assertFalse("equals(null) should return false.", a.equals(null));
         Assert.assertFalse("Same data, one null datatype should be unequal.", a.equals(nullType));
         Assert.assertFalse("Same datatype, one null data should be unequal.", a.equals(nullData));
-        RyaType sameNull = new RyaType(null, null);
+        Assert.assertFalse("Same datatype, data, one null lang should be unequal.", aLang.equals(nullLang));
+        final RyaType sameNull = new RyaType(null, null);
         Assert.assertTrue("Matching null fields should be equal.", sameNull.equals(nullBoth));
     }
 
@@ -103,6 +112,8 @@ public class RyaTypeTest {
                 a.equals(aIri), a.compareTo(aIri) == 0);
         Assert.assertEquals("equals and compareTo inconsistent for different values and different types.",
                 a.equals(bIri), a.compareTo(bIri) == 0);
+        Assert.assertEquals("equals and compareTo inconsistent for different lang and same types/data.",
+                aLang.equals(bLang), aLang.compareTo(bLang) == 0);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/test/java/org/apache/rya/api/resolver/LanguageCodesTestHelper.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/test/java/org/apache/rya/api/resolver/LanguageCodesTestHelper.java b/common/rya.api/src/test/java/org/apache/rya/api/resolver/LanguageCodesTestHelper.java
new file mode 100644
index 0000000..e95951c
--- /dev/null
+++ b/common/rya.api/src/test/java/org/apache/rya/api/resolver/LanguageCodesTestHelper.java
@@ -0,0 +1,123 @@
+/*
+ * 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.api.resolver;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+/**
+ * Helper class for reading language code files and generating a list of
+ * languages to test.
+ */
+public final class LanguageCodesTestHelper {
+    private static final String ALPHA_2_LANGUAGE_CODE_FILE = "src/test/resources/ISO-639-1_Language_Codes.txt";
+    private static final String ALPHA_3_LANGUAGE_CODE_FILE = "src/test/resources/ISO-639-2_Language_Codes.txt";
+    private static final String COUNTRY_CODE_FILE = "src/test/resources/ISO-3166-1_Country_Codes.txt";
+
+    private Set<String> languageCodes;
+
+    /**
+     * Private constructor to enforce singleton pattern.
+     * @throws Exception
+     */
+    private LanguageCodesTestHelper() throws Exception {
+        setupLanguageCodes();
+    }
+
+    private static class InstanceHolder {
+        private static final LanguageCodesTestHelper INSTANCE;
+        static {
+            try {
+                INSTANCE = new LanguageCodesTestHelper();
+            } catch (final Exception e) {
+                throw new ExceptionInInitializerError(e);
+            }
+        }
+    }
+
+    /**
+     * @return the singleton instance of {@link LanguageCodesTestHelper}.
+     */
+    public static LanguageCodesTestHelper getInstance(){
+        return InstanceHolder.INSTANCE;
+    }
+
+    /**
+     * Generates a list of language codes with country codes to test. This tries
+     * to reproduce BCP-47 compliant language tags as specified by
+     * <a href="https://tools.ietf.org/html/bcp47">BCP47</a>.
+     * <p>
+     * Adds all of the following combinations for testing:
+     * <ul>
+     *   <li>Alpha-2 language code only</li>
+     *   <li>Alpha-3 language code only</li>
+     *   <li>Alpha-2 language code + Country code</li>
+     *   <li>Alpha-3 language code + Country code</li>
+     * </ul>
+     * This should produce most common combinations and a lot of unlikely ones
+     * too.
+     * @throws Exception
+     */
+    private void setupLanguageCodes() throws Exception {
+        final List<String> alpha2LangCodes = readCodeFile(ALPHA_2_LANGUAGE_CODE_FILE);
+        final List<String> alpha3LangCodes = readCodeFile(ALPHA_3_LANGUAGE_CODE_FILE);
+        final List<String> countryCodes = readCodeFile(COUNTRY_CODE_FILE);
+
+        // Generate all combinations of language codes and region codes.
+        final List<String> langCodes = new ArrayList<>();
+        langCodes.addAll(alpha2LangCodes);
+        langCodes.addAll(alpha3LangCodes);
+        for (final String languageCode : alpha2LangCodes) {
+            for (final String countryCode : countryCodes) {
+                langCodes.add(languageCode + "-" + countryCode);
+            }
+        }
+        for (final String languageCode : alpha3LangCodes) {
+            for (final String countryCode : countryCodes) {
+                langCodes.add(languageCode + "-" + countryCode);
+            }
+        }
+        languageCodes = ImmutableSet.copyOf(langCodes);
+    }
+
+    private static List<String> readCodeFile(final String fileName) throws IOException {
+        final List<String> codes = new ArrayList<>();
+        // Read each line
+        try (final Stream<String> stream = Files.lines(Paths.get(fileName))) {
+            // Each line might be comma-separated so add multiple codes per line
+            stream.forEach(line -> codes.addAll(Lists.newArrayList(line.split(","))));
+        }
+        return codes;
+    }
+
+    /**
+     * @return the {@link Set} of language codes.
+     */
+    public Set<String> getLanguageCodes() {
+        return languageCodes;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/test/java/org/apache/rya/api/resolver/RdfToRyaConversionsTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/test/java/org/apache/rya/api/resolver/RdfToRyaConversionsTest.java b/common/rya.api/src/test/java/org/apache/rya/api/resolver/RdfToRyaConversionsTest.java
new file mode 100644
index 0000000..5c83537
--- /dev/null
+++ b/common/rya.api/src/test/java/org/apache/rya/api/resolver/RdfToRyaConversionsTest.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.api.resolver;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.List;
+import java.util.Set;
+
+import org.apache.rya.api.domain.RyaType;
+import org.apache.rya.api.utils.LiteralLanguageUtils;
+import org.eclipse.rdf4j.model.Literal;
+import org.eclipse.rdf4j.model.ValueFactory;
+import org.eclipse.rdf4j.model.impl.SimpleLiteral;
+import org.eclipse.rdf4j.model.impl.SimpleValueFactory;
+import org.eclipse.rdf4j.model.util.Literals;
+import org.eclipse.rdf4j.model.vocabulary.RDF;
+import org.eclipse.rdf4j.model.vocabulary.XMLSchema;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Tests the methods of {@link RdfToRyaConversions}.
+ */
+public class RdfToRyaConversionsTest {
+    private static final ValueFactory VF = SimpleValueFactory.getInstance();
+
+    private static final Set<String> LANGUAGE_CODES = LanguageCodesTestHelper.getInstance().getLanguageCodes();
+
+    @Test
+    public void testConvertLiteral_null() {
+        final RyaType ryaType = RdfToRyaConversions.convertLiteral(null);
+        assertNull(ryaType);
+    }
+
+    @Test
+    public void testConvertLiteral_nullDataType() {
+        final Literal literal = mock(SimpleLiteral.class);
+        final String expectedData = "Ice Cream";
+        when(literal.getLabel()).thenReturn(expectedData);
+        when(literal.stringValue()).thenReturn(expectedData);
+        // Don't think this is possible but test anyways. Need to mock to force this null value.
+        when(literal.getDatatype()).thenReturn(null);
+        final RyaType ryaType = RdfToRyaConversions.convertLiteral(literal);
+        final RyaType expected = new RyaType(XMLSchema.STRING, expectedData);
+        assertEquals(expected, ryaType);
+        assertNull(ryaType.getLanguage());
+    }
+
+    @Test
+    public void testConvertLiteral_validLanguage() {
+        final String expectedData = "Hello";
+        for (final String language : LANGUAGE_CODES) {
+            // This only checks the validity of the format. Not that the language tag actually exists.
+            assertTrue(Literals.isValidLanguageTag(language));
+            final Literal literal = VF.createLiteral(expectedData, language);
+            final RyaType ryaType = RdfToRyaConversions.convertLiteral(literal);
+            assertEquals(RDF.LANGSTRING, ryaType.getDataType());
+            assertEquals(expectedData, ryaType.getData());
+            assertEquals(language, ryaType.getLanguage());
+            final RyaType expectedRyaType = new RyaType(RDF.LANGSTRING, expectedData, language);
+            assertEquals(expectedRyaType, ryaType);
+        }
+    }
+
+    @Test
+    public void testConvertLiteral_undeterminedLanguage() {
+        final String expectedData = "Hello";
+        final String language = LiteralLanguageUtils.UNDETERMINED_LANGUAGE;
+        assertTrue(Literals.isValidLanguageTag(language));
+        final Literal literal = VF.createLiteral(expectedData, language);
+        final RyaType ryaType = RdfToRyaConversions.convertLiteral(literal);
+        assertEquals(RDF.LANGSTRING, ryaType.getDataType());
+        assertEquals(expectedData, ryaType.getData());
+        final RyaType expectedRyaType = new RyaType(RDF.LANGSTRING, expectedData, language);
+        assertEquals(expectedRyaType, ryaType);
+        assertEquals(LiteralLanguageUtils.UNDETERMINED_LANGUAGE, ryaType.getLanguage());
+    }
+
+    @Test
+    public void testConvertLiteral_invalidLanguage() {
+        final String expectedData = "Hello";
+        final List<String> badLanguages = Lists.newArrayList(
+                "bad language",
+                "en-",
+                "en-US-"
+        );
+        for (final String badLanguage : badLanguages) {
+            // This only checks the validity of the format. Not that the language tag actually exists.
+            assertFalse(Literals.isValidLanguageTag(badLanguage));
+            final Literal literal = VF.createLiteral(expectedData, badLanguage);
+            final RyaType ryaType = RdfToRyaConversions.convertLiteral(literal);
+            assertEquals(RDF.LANGSTRING, ryaType.getDataType());
+            assertEquals(expectedData, ryaType.getData());
+            // Check that the invalid language is replaced with "und"
+            assertEquals(LiteralLanguageUtils.UNDETERMINED_LANGUAGE, ryaType.getLanguage());
+        }
+    }
+
+    @Test
+    public void testConvertLiteral_normalString() {
+        final String expectedData = "Hello";
+        final Literal literal = VF.createLiteral(expectedData);
+        final RyaType ryaType = RdfToRyaConversions.convertLiteral(literal);
+        assertEquals(XMLSchema.STRING, ryaType.getDataType());
+        assertEquals(expectedData, ryaType.getData());
+        final RyaType expectedRyaType = new RyaType(XMLSchema.STRING, expectedData);
+        assertEquals(expectedRyaType, ryaType);
+        assertNull(ryaType.getLanguage());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/test/java/org/apache/rya/api/resolver/RyaContextTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/test/java/org/apache/rya/api/resolver/RyaContextTest.java b/common/rya.api/src/test/java/org/apache/rya/api/resolver/RyaContextTest.java
index d4b5f5c..954fa85 100644
--- a/common/rya.api/src/test/java/org/apache/rya/api/resolver/RyaContextTest.java
+++ b/common/rya.api/src/test/java/org/apache/rya/api/resolver/RyaContextTest.java
@@ -8,9 +8,9 @@ package org.apache.rya.api.resolver;
  * 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
@@ -22,61 +22,67 @@ package org.apache.rya.api.resolver;
 import java.util.Map;
 
 import org.apache.rya.api.RdfCloudTripleStoreConstants.TABLE_LAYOUT;
+import org.apache.rya.api.domain.RyaIRI;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaType;
-import org.apache.rya.api.domain.RyaIRI;
 import org.apache.rya.api.query.strategy.wholerow.MockRdfConfiguration;
 import org.apache.rya.api.resolver.triple.TripleRow;
 import org.eclipse.rdf4j.model.impl.SimpleValueFactory;
+import org.eclipse.rdf4j.model.vocabulary.RDF;
 
 import junit.framework.TestCase;
 
 /**
  */
 public class RyaContextTest extends TestCase {
- 
+
     public void testDefaultSerialization() throws Exception {
-        RyaContext instance = RyaContext.getInstance();
+        final RyaContext instance = RyaContext.getInstance();
         //plain string
         RyaType ryaType = new RyaType("mydata");
         byte[] serialize = instance.serialize(ryaType);
         assertEquals(ryaType, instance.deserialize(serialize));
 
         //iri
-        RyaIRI ryaIRI = new RyaIRI("urn:test#1234");
+        final RyaIRI ryaIRI = new RyaIRI("urn:test#1234");
         serialize = instance.serialize(ryaIRI);
-        RyaType deserialize = instance.deserialize(serialize);
+        final RyaType deserialize = instance.deserialize(serialize);
         assertEquals(ryaIRI, deserialize);
 
         //custom type
         ryaType = new RyaType(SimpleValueFactory.getInstance().createIRI("urn:test#customDataType"), "mydata");
         serialize = instance.serialize(ryaType);
         assertEquals(ryaType, instance.deserialize(serialize));
+
+        //language type
+        ryaType = new RyaType(RDF.LANGSTRING, "Hello", "en");
+        serialize = instance.serialize(ryaType);
+        assertEquals(ryaType, instance.deserialize(serialize));
     }
 
     public void testTripleRowSerialization() throws Exception {
-        RyaIRI subj = new RyaIRI("urn:test#subj");
-        RyaIRI pred = new RyaIRI("urn:test#pred");
-        RyaType obj = new RyaType("mydata");
-        RyaStatement statement = new RyaStatement(subj, pred, obj);
-        RyaTripleContext instance = RyaTripleContext.getInstance(new MockRdfConfiguration());
+        final RyaIRI subj = new RyaIRI("urn:test#subj");
+        final RyaIRI pred = new RyaIRI("urn:test#pred");
+        final RyaType obj = new RyaType("mydata");
+        final RyaStatement statement = new RyaStatement(subj, pred, obj);
+        final RyaTripleContext instance = RyaTripleContext.getInstance(new MockRdfConfiguration());
 
-        Map<TABLE_LAYOUT, TripleRow> map = instance.serializeTriple(statement);
-        TripleRow tripleRow = map.get(TABLE_LAYOUT.SPO);
+        final Map<TABLE_LAYOUT, TripleRow> map = instance.serializeTriple(statement);
+        final TripleRow tripleRow = map.get(TABLE_LAYOUT.SPO);
         assertEquals(statement, instance.deserializeTriple(TABLE_LAYOUT.SPO, tripleRow));
     }
-    
+
     public void testHashedTripleRowSerialization() throws Exception {
-        RyaIRI subj = new RyaIRI("urn:test#subj");
-        RyaIRI pred = new RyaIRI("urn:test#pred");
-        RyaType obj = new RyaType("mydata");
-        RyaStatement statement = new RyaStatement(subj, pred, obj);
-    	MockRdfConfiguration config = new MockRdfConfiguration();
-    	config.set(MockRdfConfiguration.CONF_PREFIX_ROW_WITH_HASH, Boolean.TRUE.toString());
-       RyaTripleContext instance = RyaTripleContext.getInstance(config);
+        final RyaIRI subj = new RyaIRI("urn:test#subj");
+        final RyaIRI pred = new RyaIRI("urn:test#pred");
+        final RyaType obj = new RyaType("mydata");
+        final RyaStatement statement = new RyaStatement(subj, pred, obj);
+        final MockRdfConfiguration config = new MockRdfConfiguration();
+        config.set(MockRdfConfiguration.CONF_PREFIX_ROW_WITH_HASH, Boolean.TRUE.toString());
+        final RyaTripleContext instance = RyaTripleContext.getInstance(config);
 
-        Map<TABLE_LAYOUT, TripleRow> map = instance.serializeTriple(statement);
-        TripleRow tripleRow = map.get(TABLE_LAYOUT.SPO);
+        final Map<TABLE_LAYOUT, TripleRow> map = instance.serializeTriple(statement);
+        final TripleRow tripleRow = map.get(TABLE_LAYOUT.SPO);
         assertEquals(statement, instance.deserializeTriple(TABLE_LAYOUT.SPO, tripleRow));
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/test/java/org/apache/rya/api/resolver/RyaToRdfConversionsTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/test/java/org/apache/rya/api/resolver/RyaToRdfConversionsTest.java b/common/rya.api/src/test/java/org/apache/rya/api/resolver/RyaToRdfConversionsTest.java
new file mode 100644
index 0000000..242221f
--- /dev/null
+++ b/common/rya.api/src/test/java/org/apache/rya/api/resolver/RyaToRdfConversionsTest.java
@@ -0,0 +1,127 @@
+/*
+ * 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.api.resolver;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+import java.util.Set;
+
+import org.apache.rya.api.domain.RyaType;
+import org.apache.rya.api.utils.LiteralLanguageUtils;
+import org.eclipse.rdf4j.model.Literal;
+import org.eclipse.rdf4j.model.ValueFactory;
+import org.eclipse.rdf4j.model.impl.SimpleValueFactory;
+import org.eclipse.rdf4j.model.util.Literals;
+import org.eclipse.rdf4j.model.vocabulary.RDF;
+import org.eclipse.rdf4j.model.vocabulary.XMLSchema;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Tests the methods of {@link RyaToRdfConversionsTest}.
+ */
+public class RyaToRdfConversionsTest {
+    private static final ValueFactory VF = SimpleValueFactory.getInstance();
+
+    private static final Set<String> LANGUAGE_CODES = LanguageCodesTestHelper.getInstance().getLanguageCodes();
+
+    @Test (expected=NullPointerException.class)
+    public void testConvertLiteral_null() {
+        RyaToRdfConversions.convertLiteral(null);
+    }
+
+    @Test
+    public void testConvertLiteral_nullDataType() {
+        final String expectedData = "Ice Cream";
+        final RyaType ryaType = new RyaType(null, expectedData);
+        final Literal literal = RyaToRdfConversions.convertLiteral(ryaType);
+        final Literal expected = VF.createLiteral(expectedData, XMLSchema.STRING);
+        assertEquals(expected, literal);
+        assertFalse(literal.getLanguage().isPresent());
+    }
+
+    @Test
+    public void testConvertLiteral_validLanguage() {
+        final String expectedData = "Hello";
+        for (final String language : LANGUAGE_CODES) {
+            // This only checks the validity of the format. Not that the language tag actually exists.
+            assertTrue(Literals.isValidLanguageTag(language));
+            final RyaType ryaType = new RyaType(RDF.LANGSTRING, expectedData, language);
+            final Literal literal = RyaToRdfConversions.convertLiteral(ryaType);
+            assertEquals(RDF.LANGSTRING, literal.getDatatype());
+            assertEquals(expectedData, literal.getLabel());
+            assertTrue(literal.getLanguage().isPresent());
+            assertEquals(language, literal.getLanguage().get());
+            final Literal expectedLiteral = VF.createLiteral(expectedData, language);
+            assertEquals(expectedLiteral, literal);
+        }
+    }
+
+    @Test
+    public void testConvertLiteral_undeterminedLanguage() {
+        final String expectedData = "Hello";
+        final String language = LiteralLanguageUtils.UNDETERMINED_LANGUAGE;
+        assertTrue(Literals.isValidLanguageTag(language));
+        final RyaType ryaType = new RyaType(RDF.LANGSTRING, expectedData, language);
+        final Literal literal = RyaToRdfConversions.convertLiteral(ryaType);
+        assertEquals(RDF.LANGSTRING, literal.getDatatype());
+        assertEquals(expectedData, literal.getLabel());
+        assertTrue(literal.getLanguage().isPresent());
+        assertEquals(LiteralLanguageUtils.UNDETERMINED_LANGUAGE, literal.getLanguage().get());
+        final Literal expectedLiteral = VF.createLiteral(expectedData, language);
+        assertEquals(expectedLiteral, literal);
+    }
+
+    @Test
+    public void testConvertLiteral_invalidLanguage() {
+        final String expectedData = "Hello";
+        final List<String> badLanguages = Lists.newArrayList(
+                "bad language",
+                "en-",
+                "en-US-"
+        );
+        for (final String badLanguage : badLanguages) {
+            // This only checks the validity of the format. Not that the language tag actually exists.
+            assertFalse(Literals.isValidLanguageTag(badLanguage));
+            final RyaType ryaType = new RyaType(RDF.LANGSTRING, expectedData, badLanguage);
+            final Literal literal = RyaToRdfConversions.convertLiteral(ryaType);
+            assertEquals(RDF.LANGSTRING, literal.getDatatype());
+            assertEquals(expectedData, literal.getLabel());
+            assertTrue(literal.getLanguage().isPresent());
+            // Check that the invalid language is replaced with "und"
+            assertEquals(LiteralLanguageUtils.UNDETERMINED_LANGUAGE, literal.getLanguage().get());
+        }
+    }
+
+    @Test
+    public void testConvertLiteral_normalString() {
+        final String expectedData = "Hello";
+        final RyaType ryaType = new RyaType(XMLSchema.STRING, expectedData);
+        final Literal literal = RyaToRdfConversions.convertLiteral(ryaType);
+        assertEquals(XMLSchema.STRING, literal.getDatatype());
+        assertEquals(expectedData, literal.getLabel());
+        assertFalse(literal.getLanguage().isPresent());
+        final Literal expectedLiteral = VF.createLiteral(expectedData);
+        assertEquals(expectedLiteral, literal);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/test/resources/ISO-3166-1_Country_Codes.txt
----------------------------------------------------------------------
diff --git a/common/rya.api/src/test/resources/ISO-3166-1_Country_Codes.txt b/common/rya.api/src/test/resources/ISO-3166-1_Country_Codes.txt
new file mode 100644
index 0000000..8c1c997
--- /dev/null
+++ b/common/rya.api/src/test/resources/ISO-3166-1_Country_Codes.txt
@@ -0,0 +1,236 @@
+AF
+AL
+DZ
+AS
+AD
+AO
+AQ
+AG
+AR
+AM
+AW
+AU
+AT
+AZ
+BS
+BH
+BD
+BB
+BY
+BE
+BZ
+BJ
+BM
+BT
+BO
+BA
+BW
+BV
+BR
+IO
+BN
+BG
+BF
+BI
+KH
+CM
+CA
+CV
+KY
+CF
+TD
+CL
+CN
+CX
+CC
+CO
+KM
+CG
+CD
+CK
+CR
+CI
+HR
+CU
+CY
+CZ
+DK
+DJ
+DM
+DO
+EC
+EG
+SV
+GQ
+ER
+EE
+ET
+FK
+FO
+FJ
+FI
+FR
+GF
+PF
+TF
+GA
+GM
+GE
+DE
+GH
+GI
+GR
+GL
+GD
+GP
+GU
+GT
+GN
+GW
+GY
+HT
+HM
+HN
+HK
+HU
+IS
+IN
+ID
+IR
+IQ
+IE
+IL
+IT
+JM
+JP
+JO
+KZ
+KE
+KI
+KP
+KR
+KW
+KG
+LA
+LV
+LB
+LS
+LR
+LY
+LI
+LT
+LU
+MO
+MK
+MG
+MW
+MY
+MV
+ML
+MT
+MH
+MQ
+MR
+MU
+YT
+MX
+FM
+MD
+MD
+MN
+MS
+MA
+MZ
+MM
+NA
+NR
+NP
+NL
+AN
+NC
+NZ
+NI
+NE
+NG
+NU
+NF
+MP
+NO
+OM
+PK
+PW
+PS
+PA
+PG
+PY
+PE
+PH
+PN
+PL
+PR
+QA
+RE
+RO
+RU
+RW
+SH
+KN
+LC
+PM
+VC
+WS
+SM
+ST
+SA
+SN
+CS
+SC
+SL
+SG
+SK
+SI
+SB
+SO
+ZA
+GS
+ES
+LK
+SD
+SR
+SJ
+SZ
+SE
+CH
+SY
+TW
+TJ
+TZ
+TH
+TL
+TG
+TK
+TO
+TT
+TN
+TR
+TM
+TC
+TV
+UG
+UA
+AE
+GB
+US
+UM
+UY
+UZ
+VU
+VE
+VN
+VG
+VI
+WF
+EH
+YE
+ZM
+ZW
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/test/resources/ISO-639-1_Language_Codes.txt
----------------------------------------------------------------------
diff --git a/common/rya.api/src/test/resources/ISO-639-1_Language_Codes.txt b/common/rya.api/src/test/resources/ISO-639-1_Language_Codes.txt
new file mode 100644
index 0000000..1fc94da
--- /dev/null
+++ b/common/rya.api/src/test/resources/ISO-639-1_Language_Codes.txt
@@ -0,0 +1,191 @@
+ab
+aa
+af
+ak
+sq
+am
+ar
+an
+hy
+as
+av
+ae
+ay
+az
+bm
+ba
+eu
+be
+bn
+bh
+bi
+bs
+br
+bg
+my
+ca
+ch
+ce
+ny
+zh
+zh-Hans
+zh-Hant
+cv
+kw
+co
+cr
+hr
+cs
+da
+dv
+nl
+dz
+en
+eo
+et
+ee
+fo
+fj
+fi
+fr
+ff
+gl
+gd
+gv
+ka
+de
+el
+kl
+gn
+gu
+ht
+ha
+he
+hz
+hi
+ho
+hu
+is
+io
+ig
+id,in
+ia
+ie
+iu
+ik
+ga
+it
+ja
+jv
+kl
+kn
+kr
+ks
+kk
+km
+ki
+rw
+rn
+ky
+kv
+kg
+ko
+ku
+kj
+lo
+la
+lv
+li
+ln
+lt
+lu
+lg
+lb
+gv
+mk
+mg
+ms
+ml
+mt
+mi
+mr
+mh
+mo
+mn
+na
+nv
+ng
+nd
+ne
+no
+nb
+nn
+ii
+oc
+oj
+cu
+or
+om
+os
+pi
+ps
+fa
+pl
+pt
+pa
+qu
+rm
+ro
+ru
+se
+sm
+sg
+sa
+sr
+sh
+st
+tn
+sn
+ii
+sd
+si
+ss
+sk
+sl
+so
+nr
+es
+su
+sw
+ss
+sv
+tl
+ty
+tg
+ta
+tt
+te
+th
+bo
+ti
+to
+ts
+tr
+tk
+tw
+ug
+uk
+ur
+uz
+ve
+vi
+vo
+wa
+cy
+wo
+fy
+xh
+yi,ji
+yo
+za
+zu
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2396ebb8/common/rya.api/src/test/resources/ISO-639-2_Language_Codes.txt
----------------------------------------------------------------------
diff --git a/common/rya.api/src/test/resources/ISO-639-2_Language_Codes.txt b/common/rya.api/src/test/resources/ISO-639-2_Language_Codes.txt
new file mode 100644
index 0000000..21f3834
--- /dev/null
+++ b/common/rya.api/src/test/resources/ISO-639-2_Language_Codes.txt
@@ -0,0 +1,504 @@
+aar
+abk
+ace
+ach
+ada
+ady
+afa
+afh
+afr
+ain
+aka
+akk
+alb,sqi
+ale
+alg
+alt
+amh
+ang
+anp
+apa
+ara
+arc
+arg
+arm,hye
+arn
+arp
+art
+arw
+asm
+ast
+ath
+aus
+ava
+ave
+awa
+aym
+aze
+bad
+bai
+bak
+bal
+bam
+ban
+baq,eus
+bas
+bat
+bej
+bel
+bem
+ben
+ber
+bho
+bih
+bik
+bin
+bis
+bla
+bnt
+tib,bod
+bos
+bra
+bre
+btk
+bua
+bug
+bul
+bur,mya
+byn
+cad
+cai
+car
+cat
+cau
+ceb
+cel
+cze,ces
+cha
+chb
+che
+chg
+chi,zho
+chk
+chm
+chn
+cho
+chp
+chr
+chu
+chv
+chy
+cmc
+cnr
+cop
+cor
+cos
+cpe
+cpf
+cpp
+cre
+crh
+crp
+csb
+cus
+wel,cym
+cze,ces
+dak
+dan
+dar
+day
+del
+den
+ger,deu
+dgr
+din
+div
+doi
+dra
+dsb
+dua
+dum
+dut,nld
+dyu
+dzo
+efi
+egy
+eka
+gre,ell
+elx
+eng
+enm
+epo
+est
+baq,eus
+ewe
+ewo
+fan
+fao
+per,fas
+fat
+fij
+fil
+fin
+fiu
+fon
+fre,fra
+frm
+fro
+frr
+frs
+fry
+ful
+fur
+gaa
+gay
+gba
+gem
+geo,kat
+ger,deu
+gez
+gil
+gla
+gle
+glg
+glv
+gmh
+goh
+gon
+gor
+got
+grb
+grc
+gre,ell
+grn
+gsw
+guj
+gwi
+hai
+hat
+hau
+haw
+heb
+her
+hil
+him
+hin
+hit
+hmn
+hmo
+hrv
+hsb
+hun
+hup
+arm,hye
+iba
+ibo
+ice,isl
+ido
+iii
+ijo
+iku
+ile
+ilo
+ina
+inc
+ind
+ine
+inh
+ipk
+ira
+iro
+ice,isl
+ita
+jav
+jbo
+jpn
+jpr
+jrb
+kaa
+kab
+kac
+kal
+kam
+kan
+kar
+kas
+geo,kat
+kau
+kaw
+kaz
+kbd
+kha
+khi
+khm
+kho
+kik
+kin
+kir
+kmb
+kok
+kom
+kon
+kor
+kos
+kpe
+krc
+krl
+kro
+kru
+kua
+kum
+kur
+kut
+lad
+lah
+lam
+lao
+lat
+lav
+lez
+lim
+lin
+lit
+lol
+loz
+ltz
+lua
+lub
+lug
+lui
+lun
+luo
+lus
+mac,mkd
+mad
+mag
+mah
+mai
+mak
+mal
+man
+mao,mri
+map
+mar
+mas
+may,msa
+mdf
+mdr
+men
+mga
+mic
+min
+mis
+mac,mkd
+mkh
+mlg
+mlt
+mnc
+mni
+mno
+moh
+mon
+mos
+mao,mri
+may,msa
+mul
+mun
+mus
+mwl
+mwr
+bur,mya
+myn
+myv
+nah
+nai
+nap
+nau
+nav
+nbl
+nde
+ndo
+nds
+nep
+new
+nia
+nic
+niu
+dut,nld
+nno
+nob
+nog
+non
+nor
+nqo
+nso
+nub
+nwc
+nya
+nym
+nyn
+nyo
+nzi
+oci
+oji
+ori
+orm
+osa
+oss
+ota
+oto
+paa
+pag
+pal
+pam
+pan
+pap
+pau
+peo
+per,fas
+phi
+phn
+pli
+pol
+pon
+por
+pra
+pro
+pus
+qaa-qtz
+que
+raj
+rap
+rar
+roa
+roh
+rom
+rum,ron
+run
+rup
+rus
+sad
+sag
+sah
+sai
+sal
+sam
+san
+sas
+sat
+scn
+sco
+sel
+sem
+sga
+sgn
+shn
+sid
+sin
+sio
+sit
+sla
+slo,slk
+slv
+sma
+sme
+smi
+smj
+smn
+smo
+sms
+sna
+snd
+snk
+sog
+som
+son
+sot
+spa
+alb,sqi
+srd
+srn
+srp
+srr
+ssa
+ssw
+suk
+sun
+sus
+sux
+swa
+swe
+syc
+syr
+tah
+tai
+tam
+tat
+tel
+tem
+ter
+tet
+tgk
+tgl
+tha
+tib,bod
+tig
+tir
+tiv
+tkl
+tlh
+tli
+tmh
+tog
+ton
+tpi
+tsi
+tsn
+tso
+tuk
+tum
+tup
+tur
+tut
+tvl
+twi
+tyv
+udm
+uga
+uig
+ukr
+umb
+und
+urd
+uzb
+vai
+ven
+vie
+vol
+vot
+wak
+wal
+war
+was
+wel,cym
+wen
+wln
+wol
+xal
+xho
+yao
+yap
+yid
+yor
+ypk
+zap
+zbl
+zen
+zgh
+zha
+chi,zho
+znd
+zul
+zun
+zxx
+zza