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/10/01 13:09:56 UTC

[1/4] incubator-rya git commit: Make RdfFileInputTool to accept multiple input paths. Doc improvements

Repository: incubator-rya
Updated Branches:
  refs/heads/master 3c3ab0dfd -> 5463da23c


Make RdfFileInputTool to accept multiple input paths. Doc improvements


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

Branch: refs/heads/master
Commit: 639b980ce80677ec4703ba39e19cfd9e7943c506
Parents: 3c3ab0d
Author: Maxim Kolchin <ko...@gmail.com>
Authored: Wed Jul 4 13:04:30 2018 +0300
Committer: Maxim Kolchin <ko...@gmail.com>
Committed: Wed Jul 4 13:04:30 2018 +0300

----------------------------------------------------------------------
 extras/rya.manual/src/site/markdown/loaddata.md | 48 +++++++++++++----
 .../rya.manual/src/site/markdown/quickstart.md  |  4 +-
 mapreduce/pom.xml                               | 56 ++++++++++----------
 .../rya/accumulo/mr/AbstractAccumuloMRTool.java |  6 +--
 .../rya/accumulo/mr/tools/RdfFileInputTool.java |  2 +-
 .../accumulo/mr/tools/RdfFileInputToolTest.java | 40 +++++++++++---
 mapreduce/src/test/resources/test2.ntriples     |  3 ++
 7 files changed, 108 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/639b980c/extras/rya.manual/src/site/markdown/loaddata.md
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/markdown/loaddata.md b/extras/rya.manual/src/site/markdown/loaddata.md
index e5c7bd2..9d43edd 100644
--- a/extras/rya.manual/src/site/markdown/loaddata.md
+++ b/extras/rya.manual/src/site/markdown/loaddata.md
@@ -21,7 +21,7 @@
 -->
 # Load Data
 
-There are a few mechanisms to load data
+There are a few mechanisms to load data.
 
 ## Web REST endpoint
 
@@ -92,29 +92,55 @@ The default "format" is RDF/XML, but these formats are supported : RDFXML, NTRIP
 
 ## Bulk Loading data
 
-Bulk loading data is done through Map Reduce jobs
+Bulk loading data is done through Map Reduce jobs.
 
 ### Bulk Load RDF data
 
-This Map Reduce job will read files into memory and parse them into statements. The statements are saved into the store. Here is an example for storing in Accumulo:
+This Map Reduce job will read files into memory and parse them into statements. The statements are saved into the triplestore. 
+Here are the steps to prepare and run the job:
+
+  * Load the RDF data to HDFS. It can be single of multiple volumes and directories in them.
+  * Also load the `mapreduce/target/rya.mapreduce-<version>-shaded.jar` executable jar file to HDFS.
+  * Run the following sample command:
 
 ```
-hadoop jar target/rya.mapreduce-3.2.10-SNAPSHOT-shaded.jar org.apache.rya.accumulo.mr.RdfFileInputTool -Dac.zk=localhost:2181 -Dac.instance=accumulo -Dac.username=root -Dac.pwd=secret -Drdf.tablePrefix=triplestore_ -Drdf.format=N-Triples /tmp/temp.ntrips
+hadoop hdfs://volume/rya.mapreduce-<version>-shaded.jar org.apache.rya.accumulo.mr.tools.RdfFileInputTool -Dac.zk=localhost:2181 -Dac.instance=accumulo -Dac.username=root -Dac.pwd=secret -Drdf.tablePrefix=triplestore_ -Drdf.format=N-Triples hdfs://volume/dir1,hdfs://volume/dir2,hdfs://volume/file1.nt
 ```
 
 Options:
 
-- rdf.tablePrefix : The tables (spo, po, osp) are prefixed with this qualifier. The tables become: (rdf.tablePrefix)spo,(rdf.tablePrefix)po,(rdf.tablePrefix)osp
-- ac.* : Accumulo connection parameters
-- rdf.format : See RDFFormat from RDF4J, samples include (Trig, N-Triples, RDF/XML)
-- sc.use_freetext, sc.use_geo, sc.use_temporal, sc.use_entity : If any of these are set to true, statements will also be
+- **rdf.tablePrefix** - The tables (spo, po, osp) are prefixed with this qualifier.
+    The tables become: (rdf.tablePrefix)spo,(rdf.tablePrefix)po,(rdf.tablePrefix)osp
+- **ac.*** - Accumulo connection parameters
+- **rdf.format** - See RDFFormat from RDF4J, samples include (Trig, N-Triples, RDF/XML)
+- **sc.use_freetext, sc.use_geo, sc.use_temporal, sc.use_entity** - If any of these are set to true, statements will also be
     added to the enabled secondary indices.
-- sc.freetext.predicates, sc.geo.predicates, sc.temporal.predicates: If the associated indexer is enabled, these options specify
+- **sc.freetext.predicates, sc.geo.predicates, sc.temporal.predicates** - If the associated indexer is enabled, these options specify
     which statements should be sent to that indexer (based on the predicate). If not given, all indexers will attempt to index
     all statements.
 
-The argument is the directory/file to load. This file needs to be loaded into HDFS before running. If loading a directory, all files should have the same RDF
-format.
+The positional argument is a comma separated list of directories/files to load.
+They need to be loaded into HDFS before running. If loading a directory,
+all files should have the same RDF format.
+
+Once the data is loaded, it is actually a good practice to compact your tables.
+You can do this by opening the accumulo shell shell and running the compact
+command on the generated tables. Remember the generated tables will be
+prefixed by the rdf.tablePrefix property you assigned above.
+The default tablePrefix is `rts`.
+Here is a sample Accumulo Shell command:
+
+```
+compact -p triplestore_(.*)
+```
+
+### Generate Prospects table
+
+For the best query performance, it is recommended to run the job that
+creates the Prospects table. This job will read through your data and
+gather statistics on the distribution of the dataset. This table is then
+queried before query execution to reorder queries based on the data
+distribution. See the [Prospects Table](eval.md) section on how to do this.
 
 ## Direct RDF4J API
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/639b980c/extras/rya.manual/src/site/markdown/quickstart.md
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/markdown/quickstart.md b/extras/rya.manual/src/site/markdown/quickstart.md
index f0d76a8..7a93cda 100644
--- a/extras/rya.manual/src/site/markdown/quickstart.md
+++ b/extras/rya.manual/src/site/markdown/quickstart.md
@@ -56,7 +56,7 @@ Start the Tomcat server. `./bin/startup.sh`
 
 ## Usage
 
-First, we need to load data. See the [Load Data Section] (loaddata.md)
+First, we need to load data. See the [Load Data](loaddata.md) section.
 
-Second, we need to query that data. See the [Query Data Section](querydata.md)
+Second, we need to query that data. See the [Query Data](querydata.md) section.
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/639b980c/mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/mapreduce/pom.xml b/mapreduce/pom.xml
index dc3cec4..bc019da 100644
--- a/mapreduce/pom.xml
+++ b/mapreduce/pom.xml
@@ -88,6 +88,35 @@ under the License.
     </dependencies>
 
     <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <filters>
+                                <filter>
+                                    <artifact>*:*</artifact>
+                                    <excludes>
+                                        <exclude>META-INF/*.SF</exclude>
+                                        <exclude>META-INF/*.DSA</exclude>
+                                        <exclude>META-INF/*.RSA</exclude>
+                                    </excludes>
+                                </filter>
+                            </filters>
+                            <transformers>
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                            </transformers>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
         <pluginManagement>
             <plugins>
                 <plugin>
@@ -101,33 +130,6 @@ under the License.
                         </excludes>
                     </configuration>
                 </plugin>
-                <plugin>
-                    <groupId>org.apache.maven.plugins</groupId>
-                    <artifactId>maven-shade-plugin</artifactId>
-                    <executions>
-                        <execution>
-                            <phase>package</phase>
-                            <goals>
-                                <goal>shade</goal>
-                            </goals>
-                            <configuration>
-                                <filters>
-                                    <filter>
-                                        <artifact>*:*</artifact>
-                                        <excludes>
-                                            <exclude>META-INF/*.SF</exclude>
-                                            <exclude>META-INF/*.DSA</exclude>
-                                            <exclude>META-INF/*.RSA</exclude>
-                                        </excludes>
-                                    </filter>
-                                </filters>
-                                <transformers>
-                                    <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
-                                </transformers>
-                            </configuration>
-                        </execution>
-                    </executions>
-                </plugin>
             </plugins>
         </pluginManagement>
     </build>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/639b980c/mapreduce/src/main/java/org/apache/rya/accumulo/mr/AbstractAccumuloMRTool.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/rya/accumulo/mr/AbstractAccumuloMRTool.java b/mapreduce/src/main/java/org/apache/rya/accumulo/mr/AbstractAccumuloMRTool.java
index 7489391..cd29e1e 100644
--- a/mapreduce/src/main/java/org/apache/rya/accumulo/mr/AbstractAccumuloMRTool.java
+++ b/mapreduce/src/main/java/org/apache/rya/accumulo/mr/AbstractAccumuloMRTool.java
@@ -209,18 +209,18 @@ public abstract class AbstractAccumuloMRTool implements Tool {
      * ({@link org.apache.hadoop.io.LongWritable}, {@link RyaStatementWritable})
      * pairs from RDF file(s) found at the specified path.
      * @param   job   Job to configure
-     * @param   inputPath     File or directory name
+     * @param   commaSeparatedPaths a comma separated list of files or directories
      * @param   defaultFormat  Default RDF serialization format, can be
      *                         overridden by {@link MRUtils#FORMAT_PROP}
      * @throws  IOException if there's an error interacting with the
      *          {@link org.apache.hadoop.fs.FileSystem}.
      */
-    protected void setupFileInput(Job job, String inputPath, RDFFormat defaultFormat) throws IOException {
+    protected void setupFileInputs(Job job, String commaSeparatedPaths, RDFFormat defaultFormat) throws IOException {
         RDFFormat format = MRUtils.getRDFFormat(conf);
         if (format == null) {
             format = defaultFormat;
         }
-        RdfFileInputFormat.addInputPath(job, new Path(inputPath));
+        RdfFileInputFormat.addInputPaths(job, commaSeparatedPaths);
         RdfFileInputFormat.setRDFFormat(job, format);
         job.setInputFormatClass(RdfFileInputFormat.class);
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/639b980c/mapreduce/src/main/java/org/apache/rya/accumulo/mr/tools/RdfFileInputTool.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/main/java/org/apache/rya/accumulo/mr/tools/RdfFileInputTool.java b/mapreduce/src/main/java/org/apache/rya/accumulo/mr/tools/RdfFileInputTool.java
index c004f4e..5d7209a 100644
--- a/mapreduce/src/main/java/org/apache/rya/accumulo/mr/tools/RdfFileInputTool.java
+++ b/mapreduce/src/main/java/org/apache/rya/accumulo/mr/tools/RdfFileInputTool.java
@@ -65,7 +65,7 @@ public class RdfFileInputTool extends AbstractAccumuloMRTool implements Tool {
         job.setJarByClass(RdfFileInputTool.class);
 
         String inputPath = conf.get(MRUtils.INPUT_PATH, args[0]);
-        setupFileInput(job, inputPath, RDFFormat.RDFXML);
+        setupFileInputs(job, inputPath, RDFFormat.RDFXML);
         setupRyaOutput(job);
         job.setNumReduceTasks(0);
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/639b980c/mapreduce/src/test/java/org/apache/rya/accumulo/mr/tools/RdfFileInputToolTest.java
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/java/org/apache/rya/accumulo/mr/tools/RdfFileInputToolTest.java b/mapreduce/src/test/java/org/apache/rya/accumulo/mr/tools/RdfFileInputToolTest.java
index 8f92cf1..020122b 100644
--- a/mapreduce/src/test/java/org/apache/rya/accumulo/mr/tools/RdfFileInputToolTest.java
+++ b/mapreduce/src/test/java/org/apache/rya/accumulo/mr/tools/RdfFileInputToolTest.java
@@ -19,7 +19,6 @@ package org.apache.rya.accumulo.mr.tools;
  * under the License.
  */
 
-import junit.framework.TestCase;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
 import org.apache.accumulo.core.client.mock.MockInstance;
@@ -29,10 +28,12 @@ import org.apache.accumulo.core.security.TablePermission;
 import org.apache.rya.accumulo.AccumuloRdfConfiguration;
 import org.apache.rya.accumulo.mr.TestUtils;
 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.rio.RDFFormat;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 /**
@@ -41,7 +42,7 @@ import org.junit.Test;
  * Time: 10:51 AM
  * To change this template use File | Settings | File Templates.
  */
-public class RdfFileInputToolTest extends TestCase {
+public class RdfFileInputToolTest {
 
     private String user = "user";
     private String pwd = "pwd";
@@ -50,9 +51,8 @@ public class RdfFileInputToolTest extends TestCase {
     private Authorizations auths = new Authorizations("test_auths");
     private Connector connector;
 
-    @Override
+    @Before
     public void setUp() throws Exception {
-        super.setUp();
         connector = new MockInstance(instance).getConnector(user, new PasswordToken(pwd));
         connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
         connector.tableOperations().create(tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
@@ -70,9 +70,8 @@ public class RdfFileInputToolTest extends TestCase {
         secOps.grantTablePermission(user, tablePrefix + RdfCloudTripleStoreConstants.TBL_EVAL_SUFFIX, TablePermission.WRITE);
     }
 
-    @Override
+    @After
     public void tearDown() throws Exception {
-        super.tearDown();
         connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_SPO_SUFFIX);
         connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_PO_SUFFIX);
         connector.tableOperations().delete(tablePrefix + RdfCloudTripleStoreConstants.TBL_OSP_SUFFIX);
@@ -104,6 +103,33 @@ public class RdfFileInputToolTest extends TestCase {
     }
 
     @Test
+    public void testMultipleNTriplesInputs() throws Exception {
+        RdfFileInputTool.main(new String[]{
+                "-Dac.mock=true",
+                "-Dac.instance=" + instance,
+                "-Dac.username=" + user,
+                "-Dac.pwd=" + pwd,
+                "-Dac.auth=" + auths.toString(),
+                "-Dac.cv=" + auths.toString(),
+                "-Drdf.tablePrefix=" + tablePrefix,
+                "-Drdf.format=" + RDFFormat.NTRIPLES.getName(),
+                "src/test/resources/test.ntriples,src/test/resources/test2.ntriples",
+        });
+        RyaStatement rs1 = new RyaStatement(new RyaIRI("urn:lubm:rdfts#GraduateStudent01"),
+                new RyaIRI("urn:lubm:rdfts#hasFriend"),
+                new RyaIRI("urn:lubm:rdfts#GraduateStudent02"));
+        RyaStatement rs2 = new RyaStatement(new RyaIRI("urn:lubm:rdfts#GraduateStudent05"),
+                new RyaIRI("urn:lubm:rdfts#hasFriend"),
+                new RyaIRI("urn:lubm:rdfts#GraduateStudent07"));
+        rs1.setColumnVisibility(auths.toString().getBytes());
+        rs2.setColumnVisibility(auths.toString().getBytes());
+        AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
+        conf.setTablePrefix(tablePrefix);
+        conf.setAuths(auths.toString());
+        TestUtils.verify(connector, conf, rs1, rs2);
+    }
+
+    @Test
     public void testInputContext() throws Exception {
         RdfFileInputTool.main(new String[]{
                 "-Dac.mock=true",

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/639b980c/mapreduce/src/test/resources/test2.ntriples
----------------------------------------------------------------------
diff --git a/mapreduce/src/test/resources/test2.ntriples b/mapreduce/src/test/resources/test2.ntriples
new file mode 100644
index 0000000..692f66a
--- /dev/null
+++ b/mapreduce/src/test/resources/test2.ntriples
@@ -0,0 +1,3 @@
+<urn:lubm:rdfts#GraduateStudent05> <urn:lubm:rdfts#hasFriend> <urn:lubm:rdfts#GraduateStudent07> .
+<urn:lubm:rdfts#GraduateStudent06> <urn:lubm:rdfts#hasFriend> <urn:lubm:rdfts#GraduateStudent06> .
+<urn:lubm:rdfts#GraduateStudent07> <urn:lubm:rdfts#hasFriend> <urn:lubm:rdfts#GraduateStudent05> .


[3/4] incubator-rya git commit: Remove the Typical First Steps section

Posted by pu...@apache.org.
Remove the Typical First Steps section


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

Branch: refs/heads/master
Commit: 6dd6882877c817dfd2308563b6cb479c8e7e52ba
Parents: 0018afd
Author: Maxim Kolchin <ko...@gmail.com>
Authored: Thu Jul 5 12:07:42 2018 +0300
Committer: Maxim Kolchin <ko...@gmail.com>
Committed: Thu Jul 5 12:07:42 2018 +0300

----------------------------------------------------------------------
 extras/rya.manual/src/site/markdown/_index.md   |  3 +-
 extras/rya.manual/src/site/markdown/index.md    |  3 +-
 .../src/site/markdown/sm-firststeps.md          | 80 --------------------
 3 files changed, 4 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/6dd68828/extras/rya.manual/src/site/markdown/_index.md
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/markdown/_index.md b/extras/rya.manual/src/site/markdown/_index.md
index 07dfe50..7a3aed9 100644
--- a/extras/rya.manual/src/site/markdown/_index.md
+++ b/extras/rya.manual/src/site/markdown/_index.md
@@ -36,7 +36,7 @@ This project contains documentation about Apache Rya, a scalable RDF triple stor
 - [Kafka Connect Integration](kafka-connect-integration.md)
 
 # Samples
-- [Typical First Steps](sm-firststeps.md)
+
 - [Simple Add/Query/Remove Statements](sm-simpleaqr.md)
 - [Sparql query](sm-sparqlquery.md)
 - [Adding Authentication](sm-addauth.md)
@@ -46,4 +46,5 @@ This project contains documentation about Apache Rya, a scalable RDF triple stor
 - [Alx](alx.md)
 
 # Development
+
 - [Building From Source](build-source.md)

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/6dd68828/extras/rya.manual/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/markdown/index.md b/extras/rya.manual/src/site/markdown/index.md
index e686736..54f30e6 100644
--- a/extras/rya.manual/src/site/markdown/index.md
+++ b/extras/rya.manual/src/site/markdown/index.md
@@ -38,7 +38,7 @@ This project contains documentation about Apache Rya, a scalable RDF triple stor
 - [Kafka Connect Integration](kafka-connect-integration.md)
 
 # Samples
-- [Typical First Steps](sm-firststeps.md)
+
 - [Simple Add/Query/Remove Statements](sm-simpleaqr.md)
 - [Sparql query](sm-sparqlquery.md)
 - [Adding Authentication](sm-addauth.md)
@@ -48,4 +48,5 @@ This project contains documentation about Apache Rya, a scalable RDF triple stor
 - [Alx](alx.md)
 
 # Development
+
 - [Building From Source](build-source.md)

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/6dd68828/extras/rya.manual/src/site/markdown/sm-firststeps.md
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/markdown/sm-firststeps.md b/extras/rya.manual/src/site/markdown/sm-firststeps.md
deleted file mode 100644
index 228bfb5..0000000
--- a/extras/rya.manual/src/site/markdown/sm-firststeps.md
+++ /dev/null
@@ -1,80 +0,0 @@
-
-<!--
-
-[comment]: # Licensed to the Apache Software Foundation (ASF) under one
-[comment]: # or more contributor license agreements.  See the NOTICE file
-[comment]: # distributed with this work for additional information
-[comment]: # regarding copyright ownership.  The ASF licenses this file
-[comment]: # to you under the Apache License, Version 2.0 (the
-[comment]: # "License"); you may not use this file except in compliance
-[comment]: # with the License.  You may obtain a copy of the License at
-[comment]: # 
-[comment]: #   http://www.apache.org/licenses/LICENSE-2.0
-[comment]: # 
-[comment]: # Unless required by applicable law or agreed to in writing,
-[comment]: # software distributed under the License is distributed on an
-[comment]: # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-[comment]: # KIND, either express or implied.  See the License for the
-[comment]: # specific language governing permissions and limitations
-[comment]: # under the License.
-
--->
-# Typical First Steps
-
-In this tutorial, I will give you a quick overview of some of the first steps I perform to get data loaded and read for query.
-
-## Prerequisites
-
- We are assuming Accumulo 1.5+ usage here.
-
- * Apache Rya Source Code (`web.rya.war`)
- * Accumulo on top of Hadoop 0.20+
- * RDF Data (in N-Triples format, this format is the easiest to bulk load)
-
-## Building Source
-
-Skip this section if you already have the Map Reduce artifact and the WAR
-
-See the [Build From Source Section](build-source.md) to get the appropriate artifacts built
-
-## Load Data
-
-I find that the best way to load the data is through the Bulk Load Map Reduce job.
-
-* Save the RDF Data above onto HDFS. From now on we will refer to this location as `<RDF_HDFS_LOCATION>`
-* Move the `rya.mapreduce-<version>-job.jar` onto the hadoop cluster
-* Bulk load the data. Here is a sample command line:
-
-```
-hadoop jar ../rya.mapreduce-3.2.10-SNAPSHOT-job.jar org.apache.rya.accumulo.mr.RdfFileInputTool -Drdf.tablePrefix=lubm_ -Dcb.username=user -Dcb.pwd=cbpwd -Dcb.instance=instance -Dcb.zk=zookeeperLocation -Drdf.format=N-Triples <RDF_HDFS_LOCATION>
-```
-
-Once the data is loaded, it is actually a good practice to compact your tables. You can do this by opening the accumulo shell `shell` and running the `compact` command on the generated tables. Remember the generated tables will be prefixed by the `rdf.tablePrefix` property you assigned above. The default tablePrefix is `rts`.
-
-Here is a sample accumulo shell command:
-
-```
-compact -p lubm_(.*)
-```
-
-See the [Load Data Section](loaddata.md) for more options on loading rdf data
-
-## Run the Statistics Optimizer
-
-For the best query performance, it is recommended to run the Statistics Optimizer to create the Evaluation Statistics table. This job will read through your data and gather statistics on the distribution of the dataset. This table is then queried before query execution to reorder queries based on the data distribution.
-
-See the [Evaluation Statistics Table Section](eval.md) on how to do this.
-
-## Query data
-
-I find the easiest way to query is just to use the WAR. Load the WAR into your favorite web application container and go to the sparqlQuery.jsp page. Example:
-
-```
-http://localhost:8080/web.rya/sparqlQuery.jsp
-```
-
-This page provides a very simple text box for running queries against the store and getting data back. (SPARQL queries)
-
-Remember to update the connection information in the WAR: `WEB-INF/spring/spring-accumulo.xml`
-
-See the [Query data section](querydata.md) for more information.


[2/4] incubator-rya git commit: Minor fixes in the docs

Posted by pu...@apache.org.
Minor fixes in the docs


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

Branch: refs/heads/master
Commit: 0018afd5368e8cc1315ea447ff2666fa2444aa16
Parents: 639b980
Author: Maxim Kolchin <ko...@gmail.com>
Authored: Thu Jul 5 12:03:48 2018 +0300
Committer: Maxim Kolchin <ko...@gmail.com>
Committed: Thu Jul 5 12:03:48 2018 +0300

----------------------------------------------------------------------
 extras/rya.manual/src/site/markdown/loaddata.md   | 9 +++++----
 extras/rya.manual/src/site/markdown/quickstart.md | 2 +-
 2 files changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0018afd5/extras/rya.manual/src/site/markdown/loaddata.md
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/markdown/loaddata.md b/extras/rya.manual/src/site/markdown/loaddata.md
index 9d43edd..8894cbd 100644
--- a/extras/rya.manual/src/site/markdown/loaddata.md
+++ b/extras/rya.manual/src/site/markdown/loaddata.md
@@ -104,7 +104,7 @@ Here are the steps to prepare and run the job:
   * Run the following sample command:
 
 ```
-hadoop hdfs://volume/rya.mapreduce-<version>-shaded.jar org.apache.rya.accumulo.mr.tools.RdfFileInputTool -Dac.zk=localhost:2181 -Dac.instance=accumulo -Dac.username=root -Dac.pwd=secret -Drdf.tablePrefix=triplestore_ -Drdf.format=N-Triples hdfs://volume/dir1,hdfs://volume/dir2,hdfs://volume/file1.nt
+hadoop hdfs://volume/rya.mapreduce-<version>-shaded.jar org.apache.rya.accumulo.mr.tools.RdfFileInputTool -Dac.zk=localhost:2181 -Dac.instance=accumulo -Dac.username=root -Dac.pwd=secret -Drdf.tablePrefix=rya_ -Drdf.format=N-Triples hdfs://volume/dir1,hdfs://volume/dir2,hdfs://volume/file1.nt
 ```
 
 Options:
@@ -124,10 +124,11 @@ They need to be loaded into HDFS before running. If loading a directory,
 all files should have the same RDF format.
 
 Once the data is loaded, it is actually a good practice to compact your tables.
-You can do this by opening the accumulo shell shell and running the compact
+You can do this by opening the accumulo shell and running the compact
 command on the generated tables. Remember the generated tables will be
-prefixed by the rdf.tablePrefix property you assigned above.
-The default tablePrefix is `rts`.
+prefixed by the `rdf.tablePrefix` property you assigned above.
+The default tablePrefix is `rya_`.
+
 Here is a sample Accumulo Shell command:
 
 ```

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0018afd5/extras/rya.manual/src/site/markdown/quickstart.md
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/markdown/quickstart.md b/extras/rya.manual/src/site/markdown/quickstart.md
index 7a93cda..4483aa8 100644
--- a/extras/rya.manual/src/site/markdown/quickstart.md
+++ b/extras/rya.manual/src/site/markdown/quickstart.md
@@ -48,7 +48,7 @@ instance.name=accumulo  #Accumulo instance name
 instance.zk=localhost:2181  #Accumulo Zookeepers
 instance.username=root  #Accumulo username
 instance.password=secret  #Accumulo pwd
-rya.tableprefix=triplestore_  #Rya Table Prefix
+rya.tableprefix=rya_  #Rya Table Prefix
 rya.displayqueryplan=true  #To display the query plan
 ```
 


[4/4] incubator-rya git commit: Minor fix in loaddata.md; Closes #299

Posted by pu...@apache.org.
Minor fix in loaddata.md; Closes #299


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

Branch: refs/heads/master
Commit: 5463da23c7cd5eaddf2ab13f0b42141211ab59f2
Parents: 6dd6882
Author: Maxim Kolchin <ko...@gmail.com>
Authored: Thu Jul 5 17:04:47 2018 +0300
Committer: Valiyil <Pu...@parsons.com>
Committed: Mon Oct 1 09:07:45 2018 -0400

----------------------------------------------------------------------
 extras/rya.manual/src/site/markdown/loaddata.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5463da23/extras/rya.manual/src/site/markdown/loaddata.md
----------------------------------------------------------------------
diff --git a/extras/rya.manual/src/site/markdown/loaddata.md b/extras/rya.manual/src/site/markdown/loaddata.md
index 8894cbd..0770eed 100644
--- a/extras/rya.manual/src/site/markdown/loaddata.md
+++ b/extras/rya.manual/src/site/markdown/loaddata.md
@@ -99,7 +99,7 @@ Bulk loading data is done through Map Reduce jobs.
 This Map Reduce job will read files into memory and parse them into statements. The statements are saved into the triplestore. 
 Here are the steps to prepare and run the job:
 
-  * Load the RDF data to HDFS. It can be single of multiple volumes and directories in them.
+  * Load the RDF data to HDFS. It can be single of multiple volumes and can have directories in them.
   * Also load the `mapreduce/target/rya.mapreduce-<version>-shaded.jar` executable jar file to HDFS.
   * Run the following sample command: