This is an automated email from the ASF dual-hosted git repository.

mwalch pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo-website.git


The following commit(s) were added to refs/heads/master by this push:
     new 36b89f4  Updated MapReduce docs with 2.0 changes (#140)
36b89f4 is described below

commit 36b89f42ae4b9d7f2a110c98d4cff78c52aaecee
Author: Mike Walch <mwa...@apache.org>
AuthorDate: Fri Jan 4 09:23:08 2019 -0500

    Updated MapReduce docs with 2.0 changes (#140)
---
 _docs-2/development/high_speed_ingest.md |   4 +-
 _docs-2/development/mapreduce.md         | 245 +++++++++++--------------------
 _docs-2/development/sampling.md          |  10 +-
 _docs-2/development/summaries.md         |   5 +-
 _docs-2/security/kerberos.md             |   8 +-
 _docs-2/security/on-disk-encryption.md   |   6 +-
 _plugins/links.rb                        |   4 +-
 7 files changed, 107 insertions(+), 175 deletions(-)

diff --git a/_docs-2/development/high_speed_ingest.md 
b/_docs-2/development/high_speed_ingest.md
index ecf458b..46fee58 100644
--- a/_docs-2/development/high_speed_ingest.md
+++ b/_docs-2/development/high_speed_ingest.md
@@ -112,7 +112,7 @@ on how use to use MapReduce with Accumulo, see the 
[MapReduce documentation][map
 and the [MapReduce example code][mapred-code].
 
 [bulk-example]: 
https://github.com/apache/accumulo-examples/blob/master/docs/bulkIngest.md
-[AccumuloOutputFormat]: {% jurl 
org.apache.accumulo.core.client.mapred.AccumuloOutputFormat %}
-[AccumuloFileOutputFormat]: {% jurl 
org.apache.accumulo.core.client.mapred.AccumuloFileOutputFormat %}
+[AccumuloOutputFormat]: {% jurl 
org.apache.accumulo.hadoop.mapreduce.AccumuloOutputFormat %}
+[AccumuloFileOutputFormat]: {% jurl 
org.apache.accumulo.hadoop.mapreduce.AccumuloFileOutputFormat %}
 [mapred-docs]: {% durl development/mapreduce %}
 [mapred-code]: 
https://github.com/apache/accumulo-examples/blob/master/docs/mapred.md
diff --git a/_docs-2/development/mapreduce.md b/_docs-2/development/mapreduce.md
index ee6a93a..7687ae8 100644
--- a/_docs-2/development/mapreduce.md
+++ b/_docs-2/development/mapreduce.md
@@ -4,18 +4,11 @@ category: development
 order: 2
 ---
 
-Accumulo tables can be used as the source and destination of MapReduce jobs. To
-use an Accumulo table with a MapReduce job, configure the job parameters to use
-the [AccumuloInputFormat] and [AccumuloOutputFormat]. Accumulo specific 
parameters
-can be set via these two format classes to do the following:
+Accumulo tables can be used as the source and destination of MapReduce jobs.
 
-* Authenticate and provide user credentials for the input
-* Restrict the scan to a range of rows
-* Restrict the input to a subset of available columns
+## General MapReduce configuration
 
-## Configuration
-
-Since 2.0.0, Accumulo no longer has the same versions of dependencies (i.e 
Guava, etc) as Hadoop.
+Since 2.0.0, Accumulo no longer has the same dependency versions (i.e Guava, 
etc) as Hadoop.
 When launching a MapReduce job that reads or writes to Accumulo, you should 
build a shaded jar
 with all of your dependencies and complete the following steps so YARN only 
includes Hadoop code
 (and not all of Hadoop dependencies) when running your MapReduce job:
@@ -28,163 +21,101 @@ with all of your dependencies and complete the following 
steps so YARN only incl
     job.getConfiguration().set("mapreduce.job.classloader", "true");
     ```
 
-## Mapper and Reducer classes
+## Read input from an Accumulo table
 
-To read from an Accumulo table create a Mapper with the following class
-parameterization and be sure to configure the [AccumuloInputFormat].
+Follow the steps below to create a MapReduce job that reads from an Accumulo 
table:
 
-```java
-class MyMapper extends Mapper<Key,Value,WritableComparable,Writable> {
-    public void map(Key k, Value v, Context c) {
-        // transform key and value data here
-    }
-}
-```
-
-To write to an Accumulo table, create a Reducer with the following class
-parameterization and be sure to configure the [AccumuloOutputFormat]. The key
-emitted from the Reducer identifies the table to which the mutation is sent. 
This
-allows a single Reducer to write to more than one table if desired. A default 
table
-can be configured using the AccumuloOutputFormat, in which case the output 
table
-name does not have to be passed to the Context object within the Reducer.
-
-```java
-class MyReducer extends Reducer<WritableComparable, Writable, Text, Mutation> {
-    public void reduce(WritableComparable key, Iterable<Text> values, Context 
c) {
-        Mutation m;
-        // create the mutation based on input key and value
-        c.write(new Text("output-table"), m);
+1. Create a Mapper with the following class parameterization.
+
+    ```java
+    class MyMapper extends Mapper<Key,Value,WritableComparable,Writable> {
+        public void map(Key k, Value v, Context c) {
+            // transform key and value data here
+        }
     }
-}
-```
+    ```
 
-The Text object passed as the output should contain the name of the table to 
which
-this mutation should be applied. The Text can be null in which case the 
mutation
-will be applied to the default table name specified in the 
[AccumuloOutputFormat]
-options.
-
-## AccumuloInputFormat options
-
-The following code shows how to set up Accumulo
-
-```java
-Job job = new Job(getConf());
-ClientInfo info = Accumulo.newClient().to("myinstance","zoo1,zoo2")
-                        .as("user", "passwd").info()
-AccumuloInputFormat.setClientInfo(job, info);
-AccumuloInputFormat.setInputTableName(job, table);
-AccumuloInputFormat.setScanAuthorizations(job, new Authorizations());
-```
-
-**Optional Settings:**
-
-To restrict Accumulo to a set of row ranges:
-
-```java
-ArrayList<Range> ranges = new ArrayList<Range>();
-// populate array list of row ranges ...
-AccumuloInputFormat.setRanges(job, ranges);
-```
-
-To restrict Accumulo to a list of columns:
-
-```java
-ArrayList<Pair<Text,Text>> columns = new ArrayList<Pair<Text,Text>>();
-// populate list of columns
-AccumuloInputFormat.fetchColumns(job, columns);
-```
-
-To use a regular expression to match row IDs:
-
-```java
-IteratorSetting is = new IteratorSetting(30, RexExFilter.class);
-RegExFilter.setRegexs(is, ".*suffix", null, null, null, true);
-AccumuloInputFormat.addIterator(job, is);
-```
-
-## AccumuloMultiTableInputFormat options
-
-The [AccumuloMultiTableInputFormat] allows the scanning over multiple tables
-in a single MapReduce job. Separate ranges, columns, and iterators can be
-used for each table.
-
-```java
-InputTableConfig tableOneConfig = new InputTableConfig();
-InputTableConfig tableTwoConfig = new InputTableConfig();
-```
-
-To set the configuration objects on the job:
-
-```java
-Map<String, InputTableConfig> configs = new HashMap<String,InputTableConfig>();
-configs.put("table1", tableOneConfig);
-configs.put("table2", tableTwoConfig);
-AccumuloMultiTableInputFormat.setInputTableConfigs(job, configs);
-```
-
-**Optional settings:**
-
-To restrict to a set of ranges:
-
-```java
-ArrayList<Range> tableOneRanges = new ArrayList<Range>();
-ArrayList<Range> tableTwoRanges = new ArrayList<Range>();
-// populate array lists of row ranges for tables...
-tableOneConfig.setRanges(tableOneRanges);
-tableTwoConfig.setRanges(tableTwoRanges);
-```
-
-To restrict Accumulo to a list of columns:
-
-```java
-ArrayList<Pair<Text,Text>> tableOneColumns = new ArrayList<Pair<Text,Text>>();
-ArrayList<Pair<Text,Text>> tableTwoColumns = new ArrayList<Pair<Text,Text>>();
-// populate lists of columns for each of the tables ...
-tableOneConfig.fetchColumns(tableOneColumns);
-tableTwoConfig.fetchColumns(tableTwoColumns);
-```
-
-To set scan iterators:
-
-```java
-List<IteratorSetting> tableOneIterators = new ArrayList<IteratorSetting>();
-List<IteratorSetting> tableTwoIterators = new ArrayList<IteratorSetting>();
-// populate the lists of iterator settings for each of the tables ...
-tableOneConfig.setIterators(tableOneIterators);
-tableTwoConfig.setIterators(tableTwoIterators);
-```
-
-The name of the table can be retrieved from the input split:
-
-```java
-class MyMapper extends Mapper<Key,Value,WritableComparable,Writable> {
-    public void map(Key k, Value v, Context c) {
-        RangeInputSplit split = (RangeInputSplit)c.getInputSplit();
-        String tableName = split.getTableName();
-        // do something with table name
+2. Configure your MapReduce job to use [AccumuloInputFormat].
+
+    ```java
+    Job job = Job.getInstance(getConf());
+    job.setInputFormatClass(AccumuloInputFormat.class);
+    Properties props = 
Accumulo.newClientProperties().to("myinstance","zoo1,zoo2")
+                            .as("user", "passwd").build();
+    
AccumuloInputFormat.configure().clientProperties(props).table(table).store(job);
+    ```
+    [AccumuloInputFormat] has optional settings.
+    ```java
+    List<Range> ranges = new ArrayList<Range>();
+    List<Pair<Text,Text>> columns = new ArrayList<Pair<Text,Text>>();
+    // populate ranges & columns
+    IteratorSetting is = new IteratorSetting(30, RexExFilter.class);
+    RegExFilter.setRegexs(is, ".*suffix", null, null, null, true);
+
+    AccumuloInputFormat.configure().clientProperties(props).table(table)
+        .auths(Authorizations.EMPTY) // optional: default to user's auths if 
not set
+        .ranges(ranges)              // optional: only read specified ranges
+        .fetchColumns(columns)       // optional: only read specified columns
+        .addIterator(is)             // optional: add iterator that matches 
row IDs
+        .store(job);
+    ```
+    [AccumuloInputFormat] can also be configured to read from multiple 
Accumulo tables.
+    ```java
+    Job job = Job.getInstance(getConf());
+    job.setInputFormatClass(AccumuloInputFormat.class);
+    Properties props = 
Accumulo.newClientProperties().to("myinstance","zoo1,zoo2")
+                            .as("user", "passwd").build();
+    AccumuloInputFormat.configure().clientProperties(props)
+        .table("table1").auths(Authorizations.EMPTY).ranges(tableOneRanges)
+        .table("table2").auths(Authorizations.EMPTY).ranges(tableTwoRanges)
+        .store(job);
+    ```
+    If reading from multiple tables, the table name can be retrieved from the 
input split:
+    ```java
+    class MyMapper extends Mapper<Key,Value,WritableComparable,Writable> {
+        public void map(Key k, Value v, Context c) {
+            RangeInputSplit split = (RangeInputSplit)c.getInputSplit();
+            String tableName = split.getTableName();
+            // do something with table name
+        }
     }
-}
-```
+    ```
 
-## AccumuloOutputFormat options
+## Write output to an Accumulo table
 
-```java
-ClientInfo info = Accumulo.newClient().to("myinstance","zoo1,zoo2")
-                        .as("user", "passwd").info()
-AccumuloOutputFormat.setClientInfo(job, info);
-AccumuloOutputFormat.setDefaultTableName(job, "mytable");
-```
+Follow the steps below to write to an Accumulo table from a MapReduce job.
 
-**Optional Settings:**
+1. Create a Reducer with the following class parameterization. The key emitted 
from
+    the Reducer identifies the table to which the mutation is sent. This 
allows a single
+    Reducer to write to more than one table if desired. A default table can be 
configured
+    using the [AccumuloOutputFormat], in which case the output table name does 
not have to
+    be passed to the Context object within the Reducer.
+    ```java
+    class MyReducer extends Reducer<WritableComparable, Writable, Text, 
Mutation> {
+        public void reduce(WritableComparable key, Iterable<Text> values, 
Context c) {
+            Mutation m;
+            // create the mutation based on input key and value
+            c.write(new Text("output-table"), m);
+        }
+    }
+    ```
+    The Text object passed as the output should contain the name of the table 
to which
+    this mutation should be applied. The Text can be null in which case the 
mutation
+    will be applied to the default table name specified in the 
[AccumuloOutputFormat]
+    options.
 
-```java
-AccumuloOutputFormat.setMaxLatency(job, 300000); // milliseconds
-AccumuloOutputFormat.setMaxMutationBufferSize(job, 50000000); // bytes
-```
+2. Configure your MapReduce job to use [AccumuloOutputFormat].
+    ```java
+    Job job = Job.getInstance(getConf());
+    job.setOutputFormatClass(AccumuloOutputFormat.class);
+    Properties props = 
Accumulo.newClientProperties().to("myinstance","zoo1,zoo2")
+                            .as("user", "passwd").build();
+    AccumuloOutputFormat.configure().clientProperties(props)
+        .defaultTable("mytable").store(job);
+    ```
 
 The [MapReduce example][mapred-example] contains a complete example of using 
MapReduce with Accumulo.
 
 [mapred-example]: 
https://github.com/apache/accumulo-examples/blob/master/docs/mapred.md
-[AccumuloInputFormat]: {% jurl 
org.apache.accumulo.core.client.mapred.AccumuloInputFormat %}
-[AccumuloMultiTableInputFormat]: {% jurl 
org.apache.accumulo.core.client.mapred.AccumuloMultiTableInputFormat %}
-[AccumuloOutputFormat]: {% jurl 
org.apache.accumulo.core.client.mapred.AccumuloOutputFormat %}
+[AccumuloInputFormat]: {% jurl 
org.apache.accumulo.hadoop.mapreduce.AccumuloInputFormat %}
+[AccumuloOutputFormat]: {% jurl 
org.apache.accumulo.hadoop.mapreduce.AccumuloOutputFormat %}
diff --git a/_docs-2/development/sampling.md b/_docs-2/development/sampling.md
index cde4642..4d586d3 100644
--- a/_docs-2/development/sampling.md
+++ b/_docs-2/development/sampling.md
@@ -52,8 +52,8 @@ Sample data can also be scanned from within an Accumulo 
[SortedKeyValueIterator]
 To see how to do this, look at the example iterator referenced in the 
[sampling example][example].
 Also, consult the javadoc on 
[IteratorEnvironment.cloneWithSamplingEnabled()][clone-sampling].
 
-Map reduce jobs using the [AccumuloInputFormat] can also read sample data.  See
-the javadoc for the `setSamplerConfiguration()` method of 
[AccumuloInputFormat].
+MapReduce jobs using the [AccumuloInputFormat] can also read sample data.  See 
the javadoc
+for `samplerConfiguration()` in the `configure()` method of 
[AccumuloInputFormat].
 
 Scans over sample data will throw a [SampleNotPresentException] in the 
following cases :
 
@@ -67,7 +67,7 @@ generated with the same configuration.
 ## Bulk import
 
 When generating rfiles to bulk import into Accumulo, those rfiles can contain
-sample data.  To use this feature, look at the javadoc of the `setSampler(...)`
+sample data.  To use this feature, look at the javadoc of `sampler()` in the 
`configure()`
 method of [AccumuloFileOutputFormat].
 
 [example]: 
https://github.com/apache/accumulo-examples/blob/master/docs/sample.md
@@ -75,8 +75,8 @@ method of [AccumuloFileOutputFormat].
 [sample-package]: {% jurl org.apache.accumulo.core.client.sample %}
 [skv-iterator]: {% jurl 
org.apache.accumulo.core.iterators.SortedKeyValueIterator %}
 [clone-sampling]: {% jurl 
org.apache.accumulo.core.iterators.IteratorEnvironment#cloneWithSamplingEnabled--
 %}
-[AccumuloInputFormat]: {% jurl 
org.apache.accumulo.core.client.mapred.AccumuloInputFormat %}
-[AccumuloFileOutputFormat]: {% jurl 
org.apache.accumulo.core.client.mapred.AccumuloFileOutputFormat %}
+[AccumuloInputFormat]: {% jurl 
org.apache.accumulo.hadoop.mapreduce.AccumuloInputFormat %}
+[AccumuloFileOutputFormat]: {% jurl 
org.apache.accumulo.hadoop.mapreduce.AccumuloFileOutputFormat %}
 [SampleNotPresentException]: {% jurl 
org.apache.accumulo.core.client.SampleNotPresentException %}
 [BatchScanner]: {% jurl org.apache.accumulo.core.client.BatchScanner %}
 [Scanner]: {% jurl org.apache.accumulo.core.client.Scanner %}
diff --git a/_docs-2/development/summaries.md b/_docs-2/development/summaries.md
index d68a570..40f6c1e 100644
--- a/_docs-2/development/summaries.md
+++ b/_docs-2/development/summaries.md
@@ -63,8 +63,8 @@ requires a special permission.  User must have the table 
permission
 ## Bulk import
 
 When generating RFiles to bulk import into Accumulo, those RFiles can contain
-summary data.  To use this feature, look at the javadoc on the
-`AccumuloFileOutputFormat.setSummarizers(...)` method.  Also, the {% jlink 
org.apache.accumulo.core.client.rfile.RFile %}
+summary data.  To use this feature, look at the javadoc of `summarizers()` in 
the `configure()` method
+of AccumuloFileOutputFormat.  Also, the {% jlink 
org.apache.accumulo.core.client.rfile.RFile %}
 class has options for creating RFiles with embedded summary data.
 
 ## Examples
@@ -218,3 +218,4 @@ root@uno summary_test> summaries
 root@uno summary_test>   
 ```
 
+[AccumuloFileOutputFormat]: {% jurl 
org.apache.accumulo.hadoop.mapreduce.AccumuloFileOutputFormat %}
diff --git a/_docs-2/security/kerberos.md b/_docs-2/security/kerberos.md
index 716f630..2535935 100644
--- a/_docs-2/security/kerberos.md
+++ b/_docs-2/security/kerberos.md
@@ -390,14 +390,14 @@ KerberosToken kt = new KerberosToken();
 AccumuloClient client = Accumulo.newClient().to("myinstance", "zoo1,zoo2")
                           .as(principal, kt).build();
 DelegationToken dt = client.securityOperations().getDelegationToken();
-AccumuloClient client2 = client.changeUser(principal, dt);
-ClientInfo info2 = client2.info();
+Properties props = Accumulo.newClientProperties().from(client.properties())
+                          .as(principal, dt).build();
 
 // Reading from Accumulo
-AccumuloInputFormat.setClientInfo(job, info2);
+AccumuloInputFormat.configure().clientProperties(props).store(job);
 
 // Writing to Accumulo
-AccumuloOutputFormat.setClientInfo(job, info2);
+AccumuloOutputFormat.configure().clientProperties(props).store(job);
 ```
 
 Users must have the `DELEGATION_TOKEN` system permission to call the 
`getDelegationToken`
diff --git a/_docs-2/security/on-disk-encryption.md 
b/_docs-2/security/on-disk-encryption.md
index e7be37b..7046767 100644
--- a/_docs-2/security/on-disk-encryption.md
+++ b/_docs-2/security/on-disk-encryption.md
@@ -78,8 +78,8 @@ its the additional data that gets encrypted on disk that 
could be exposed in a l
 
 ### Bulk Import
 
-There are 2 ways to create RFiles for bulk ingest: with the [RFile API][rfile] 
and during Map Reduce using [AccumuloOutputFormat].  
-The [RFile API][rfile] allows passing in the configuration properties for 
encryption mentioned above.  The [AccumuloOutputFormat] does 
+There are 2 ways to create RFiles for bulk ingest: with the [RFile API][rfile] 
and during Map Reduce using [AccumuloFileOutputFormat].  
+The [RFile API][rfile] allows passing in the configuration properties for 
encryption mentioned above.  The [AccumuloFileOutputFormat] does 
 not allow for encryption of RFiles so any data bulk imported through this 
process will be unencrypted.
 
 ### Zookeeper
@@ -104,4 +104,4 @@ As you can see, there is a significant performance hit when 
running without the
 [Kerberos]: {% durl security/kerberos %}
 [design]: {% durl getting-started/design#rfile %}
 [rfile]: {% jurl org.apache.accumulo.core.client.rfile.RFile %}
-[AccumuloOutputFormat]: {% jurl 
org.apache.accumulo.core.client.mapred.AccumuloOutputFormat %}
+[AccumuloFileOutputFormat]: {% jurl 
org.apache.accumulo.hadoop.mapreduce.AccumuloFileOutputFormat %}
diff --git a/_plugins/links.rb b/_plugins/links.rb
index 2f9dc3f..f227890 100755
--- a/_plugins/links.rb
+++ b/_plugins/links.rb
@@ -43,8 +43,8 @@ def render_javadoc(context, text, url_only)
   jmodule = 'accumulo-' + clz.split('.')[3]
   if clz.start_with?('org.apache.accumulo.server')
     jmodule = 'accumulo-server-base'
-  elsif clz.start_with?('org.apache.accumulo.core.client.mapred')
-    jmodule = 'accumulo-client-mapreduce'
+  elsif clz.start_with?('org.apache.accumulo.hadoop.mapred')
+    jmodule = 'accumulo-hadoop-mapreduce'
   elsif clz.start_with?('org.apache.accumulo.iteratortest')
     jmodule = 'accumulo-iterator-test-harness'
   end

Reply via email to