You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2011/10/13 21:14:00 UTC

svn commit: r1183037 [3/4] - in /incubator/accumulo/site/trunk: ./ cgi-bin/ content/ content/accumulo/ content/accumulo/css/ content/accumulo/images/ content/accumulo/user_manual_1.3-incubating/ content/accumulo/user_manual_1.4-incubating/ lib/ templates/

Added: incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Analytics.mdtext
URL: http://svn.apache.org/viewvc/incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Analytics.mdtext?rev=1183037&view=auto
==============================================================================
--- incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Analytics.mdtext (added)
+++ incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Analytics.mdtext Thu Oct 13 19:13:58 2011
@@ -0,0 +1,140 @@
+Title: Accumulo User Manual: Analytics
+<a id=CHILD_LINKS></a>**Subsections**
+
+* [MapReduce][1]
+* [Aggregating Iterators][2]
+* [Statistical Modeling][3]
+
+* * *
+
+## <a id=SECTION00900000000000000000></a> Analytics
+
+Accumulo supports more advanced data processing than simply keeping keys sorted and performing efficient lookups. Analytics can be developed by using MapReduce and Iterators in conjunction with Accumulo tables. 
+
+## <a id=SECTION00910000000000000000></a> MapReduce
+
+Accumulo tables can be used as the source and destination of MapReduce jobs. To use a Accumulo table with a MapReduce job (specifically with the new Hadoop API as of version 0.20), 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: 
+
+* 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 
+
+### <a id=SECTION00911000000000000000></a> Mapper and Reducer classes
+
+To read from a Accumulo table create a Mapper with the following class parameterization and be sure to configure the AccumuloInputFormat. 
+    
+    
+    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 a 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. 
+    
+    
+    class MyReducer extends Reducer<WritableComparable, Writable, Text, Mutation> {
+    
+        public void reduce(WritableComparable key, Iterator<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. 
+
+### <a id=SECTION00912000000000000000></a> AccumuloInputFormat options
+    
+    
+    Job job = new Job(getConf());
+    AccumuloInputFormat.setInputInfo(job,
+            "user",
+            "passwd".getBytes(),
+            "table",
+            new Authorizations());
+    
+    AccumuloInputFormat.setZooKeeperInstance(job, "myinstance",
+            "zooserver-one,zooserver-two");
+    
+
+**Optional settings:**
+
+To restrict Accumulo to a set of row ranges: 
+    
+    
+    ArrayList<Range> ranges = new ArrayList<Range>();
+    // populate array list of row ranges ...
+    AccumuloInputFormat.setRanges(job, ranges);
+    
+
+To restrict accumulo to a list of columns: 
+    
+    
+    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: 
+    
+    
+    AccumuloInputFormat.setRegex(job, RegexType.ROW, "^.*");
+    
+
+### <a id=SECTION00913000000000000000></a> AccumuloOutputFormat options
+    
+    
+    boolean createTables = true;
+    String defaultTable = "mytable";
+    
+    AccumuloOutputFormat.setOutputInfo(job,
+            "user",
+            "passwd".getBytes(),
+            createTables,
+            defaultTable);
+    
+    AccumuloOutputFormat.setZooKeeperInstance(job, "myinstance",
+            "zooserver-one,zooserver-two");
+    
+
+**Optional Settings:**
+    
+    
+    AccumuloOutputFormat.setMaxLatency(job, 300); // milliseconds
+    AccumuloOutputFormat.setMaxMutationBufferSize(job, 5000000); // bytes
+    
+
+An example of using MapReduce with Accumulo can be found at   
+accumulo/docs/examples/README.mapred 
+
+## <a id=SECTION00920000000000000000></a> Aggregating Iterators
+
+Many applications can benefit from the ability to aggregate values across common keys. This can be done via aggregating iterators and is similar to the Reduce step in MapReduce. This provides the ability to define online, incrementally updated analytics without the overhead or latency associated with batch-oriented MapReduce jobs. 
+
+All that is needed to aggregate values of a table is to identify the fields over which values will be grouped, insert mutations with those fields as the key, and configure the table with an aggregating iterator that supports the summarization operation desired. 
+
+The only restriction on an aggregating iterator is that the aggregator developer should not assume that all values for a given key have been seen, since new mutations can be inserted at anytime. This precludes using the total number of values in the aggregation such as when calculating an average, for example. 
+
+### <a id=SECTION00921000000000000000></a> Feature Vectors
+
+An interesting use of aggregating iterators within a Accumulo table is to store feature vectors for use in machine learning algorithms. For example, many algorithms such as k-means clustering, support vector machines, anomaly detection, etc. use the concept of a feature vector and the calculation of distance metrics to learn a particular model. The columns in a Accumulo table can be used to efficiently store sparse features and their weights to be incrementally updated via the use of an aggregating iterator. 
+
+## <a id=SECTION00930000000000000000></a> Statistical Modeling
+
+Statistical models that need to be updated by many machines in parallel could be similarly stored within a Accumulo table. For example, a MapReduce job that is iteratively updating a global statistical model could have each map or reduce worker reference the parts of the model to be read and updated through an embedded Accumulo client. 
+
+Using Accumulo this way enables efficient and fast lookups and updates of small pieces of information in a random access pattern, which is complementary to MapReduce's sequential access model.   
+
+
+* * *
+
+   [1]: Analytics.html#SECTION00910000000000000000
+   [2]: Analytics.html#SECTION00920000000000000000
+   [3]: Analytics.html#SECTION00930000000000000000
+

Added: incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Contents.mdtext
URL: http://svn.apache.org/viewvc/incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Contents.mdtext?rev=1183037&view=auto
==============================================================================
--- incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Contents.mdtext (added)
+++ incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Contents.mdtext Thu Oct 13 19:13:58 2011
@@ -0,0 +1,238 @@
+Title: Accumulo User Manual: Contents
+  
+
+
+### <a id=SECTION00100000000000000000></a> Contents
+
+* [Introduction][1]
+* [Accumulo Design][2]
+
+    * [Data Model][3]
+    * [Architecture][4]
+    * [Components][5]
+
+        * [Tablet Server][6]
+        * [Loggers][7]
+        * [Garbage Collector][8]
+        * [Master][9]
+        * [Client][10]
+
+    * [Data Management][11]
+    * [Tablet Service][12]
+    * [Compactions][13]
+    * [Fault-Tolerance][14]
+
+  
+
+* [Accumulo Shell][15]
+
+    * [Basic Administration][16]
+    * [Table Maintenance][17]
+    * [User Administration][18]
+
+  
+
+* [Writing Accumulo Clients][19]
+
+    * [Writing Data][20]
+
+        * [BatchWriter][21]
+
+    * [Reading Data][22]
+
+        * [Scanner][23]
+        * [BatchScanner][24]
+
+  
+
+* [Table Configuration][25]
+
+    * [Locality Groups][26]
+
+        * [Managing Locality Groups via the Shell][27]
+        * [Managing Locality Groups via the Client API][28]
+
+    * [Constraints][29]
+    * [Bloom Filters][30]
+    * [Iterators][31]
+
+        * [Setting Iterators via the Shell][32]
+        * [Setting Iterators Programmatically][33]
+
+    * [Versioning Iterators and Timestamps][34]
+
+        * [Logical Time][35]
+        * [Deletes][36]
+
+    * [Filters][37]
+    * [Aggregating Iterators][38]
+    * [Pre-splitting tables][39]
+    * [Merging tables][40]
+    * [Delete Range][41]
+
+  
+
+* [Table Design][42]
+
+    * [Basic Table][43]
+    * [RowID Design][44]
+    * [Indexing][45]
+    * [Entity-Attribute and Graph Tables][46]
+    * [Document-Partitioned Indexing][47]
+
+  
+
+* [High-Speed Ingest][48]
+
+    * [Pre-Splitting New Tables][49]
+    * [Multiple Ingester Clients][50]
+    * [Bulk Ingest][51]
+    * [MapReduce Ingest][52]
+
+  
+
+* [Analytics][53]
+
+    * [MapReduce][54]
+
+        * [Mapper and Reducer classes][55]
+        * [AccumuloInputFormat options][56]
+        * [AccumuloOutputFormat options][57]
+
+    * [Aggregating Iterators][58]
+
+        * [Feature Vectors][59]
+
+    * [Statistical Modeling][60]
+
+  
+
+* [Security][61]
+
+    * [Security Label Expressions][62]
+    * [Security Label Expression Syntax][63]
+    * [Authorization][64]
+    * [User Authorizations][65]
+    * [Secure Authorizations Handling][66]
+    * [Query Services Layer][67]
+
+  
+
+* [Administration][68]
+
+    * [Hardware][69]
+    * [Network][70]
+    * [Installation][71]
+    * [Dependencies][72]
+    * [Configuration][73]
+
+        * [Edit conf/accumulo-env.sh][74]
+        * [Cluster Specification][75]
+        * [Accumulo Settings][76]
+        * [Deploy Configuration][77]
+
+    * [Initialization][78]
+    * [Running][79]
+
+        * [Starting Accumulo][80]
+        * [Stopping Accumulo][81]
+
+    * [Monitoring][82]
+    * [Logging][83]
+    * [Recovery][84]
+
+  
+
+* [Shell Commands][85]
+
+  
+
+
+* * *
+
+   [1]: Introduction.html
+   [2]: Accumulo_Design.html
+   [3]: Accumulo_Design.html#SECTION00310000000000000000
+   [4]: Accumulo_Design.html#SECTION00320000000000000000
+   [5]: Accumulo_Design.html#SECTION00330000000000000000
+   [6]: Accumulo_Design.html#SECTION00331000000000000000
+   [7]: Accumulo_Design.html#SECTION00332000000000000000
+   [8]: Accumulo_Design.html#SECTION00333000000000000000
+   [9]: Accumulo_Design.html#SECTION00334000000000000000
+   [10]: Accumulo_Design.html#SECTION00335000000000000000
+   [11]: Accumulo_Design.html#SECTION00340000000000000000
+   [12]: Accumulo_Design.html#SECTION00350000000000000000
+   [13]: Accumulo_Design.html#SECTION00360000000000000000
+   [14]: Accumulo_Design.html#SECTION00370000000000000000
+   [15]: Accumulo_Shell.html
+   [16]: Accumulo_Shell.html#SECTION00410000000000000000
+   [17]: Accumulo_Shell.html#SECTION00420000000000000000
+   [18]: Accumulo_Shell.html#SECTION00430000000000000000
+   [19]: Writing_Accumulo_Clients.html
+   [20]: Writing_Accumulo_Clients.html#SECTION00510000000000000000
+   [21]: Writing_Accumulo_Clients.html#SECTION00511000000000000000
+   [22]: Writing_Accumulo_Clients.html#SECTION00520000000000000000
+   [23]: Writing_Accumulo_Clients.html#SECTION00521000000000000000
+   [24]: Writing_Accumulo_Clients.html#SECTION00522000000000000000
+   [25]: Table_Configuration.html
+   [26]: Table_Configuration.html#SECTION00610000000000000000
+   [27]: Table_Configuration.html#SECTION00611000000000000000
+   [28]: Table_Configuration.html#SECTION00612000000000000000
+   [29]: Table_Configuration.html#SECTION00620000000000000000
+   [30]: Table_Configuration.html#SECTION00630000000000000000
+   [31]: Table_Configuration.html#SECTION00640000000000000000
+   [32]: Table_Configuration.html#SECTION00641000000000000000
+   [33]: Table_Configuration.html#SECTION00642000000000000000
+   [34]: Table_Configuration.html#SECTION00650000000000000000
+   [35]: Table_Configuration.html#SECTION00651000000000000000
+   [36]: Table_Configuration.html#SECTION00652000000000000000
+   [37]: Table_Configuration.html#SECTION00660000000000000000
+   [38]: Table_Configuration.html#SECTION00670000000000000000
+   [39]: Table_Configuration.html#SECTION00680000000000000000
+   [40]: Table_Configuration.html#SECTION00690000000000000000
+   [41]: Table_Configuration.html#SECTION006100000000000000000
+   [42]: Table_Design.html
+   [43]: Table_Design.html#SECTION00710000000000000000
+   [44]: Table_Design.html#SECTION00720000000000000000
+   [45]: Table_Design.html#SECTION00730000000000000000
+   [46]: Table_Design.html#SECTION00740000000000000000
+   [47]: Table_Design.html#SECTION00750000000000000000
+   [48]: High_Speed_Ingest.html
+   [49]: High_Speed_Ingest.html#SECTION00810000000000000000
+   [50]: High_Speed_Ingest.html#SECTION00820000000000000000
+   [51]: High_Speed_Ingest.html#SECTION00830000000000000000
+   [52]: High_Speed_Ingest.html#SECTION00840000000000000000
+   [53]: Analytics.html
+   [54]: Analytics.html#SECTION00910000000000000000
+   [55]: Analytics.html#SECTION00911000000000000000
+   [56]: Analytics.html#SECTION00912000000000000000
+   [57]: Analytics.html#SECTION00913000000000000000
+   [58]: Analytics.html#SECTION00920000000000000000
+   [59]: Analytics.html#SECTION00921000000000000000
+   [60]: Analytics.html#SECTION00930000000000000000
+   [61]: Security.html
+   [62]: Security.html#SECTION001010000000000000000
+   [63]: Security.html#SECTION001020000000000000000
+   [64]: Security.html#SECTION001030000000000000000
+   [65]: Security.html#SECTION001040000000000000000
+   [66]: Security.html#SECTION001050000000000000000
+   [67]: Security.html#SECTION001060000000000000000
+   [68]: Administration.html
+   [69]: Administration.html#SECTION001110000000000000000
+   [70]: Administration.html#SECTION001120000000000000000
+   [71]: Administration.html#SECTION001130000000000000000
+   [72]: Administration.html#SECTION001140000000000000000
+   [73]: Administration.html#SECTION001150000000000000000
+   [74]: Administration.html#SECTION001151000000000000000
+   [75]: Administration.html#SECTION001152000000000000000
+   [76]: Administration.html#SECTION001153000000000000000
+   [77]: Administration.html#SECTION001154000000000000000
+   [78]: Administration.html#SECTION001160000000000000000
+   [79]: Administration.html#SECTION001170000000000000000
+   [80]: Administration.html#SECTION001171000000000000000
+   [81]: Administration.html#SECTION001172000000000000000
+   [82]: Administration.html#SECTION001180000000000000000
+   [83]: Administration.html#SECTION001190000000000000000
+   [84]: Administration.html#SECTION0011100000000000000000
+   [85]: Shell_Commands.html
+

Added: incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/High_Speed_Ingest.mdtext
URL: http://svn.apache.org/viewvc/incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/High_Speed_Ingest.mdtext?rev=1183037&view=auto
==============================================================================
--- incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/High_Speed_Ingest.mdtext (added)
+++ incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/High_Speed_Ingest.mdtext Thu Oct 13 19:13:58 2011
@@ -0,0 +1,75 @@
+Title: Accumulo User Manual: High Speed Ingest
+<a id=CHILD_LINKS></a>**Subsections**
+
+* [Pre-Splitting New Tables][1]
+* [Multiple Ingester Clients][2]
+* [Bulk Ingest][3]
+* [MapReduce Ingest][4]
+
+* * *
+
+## <a id=SECTION00800000000000000000></a> High-Speed Ingest
+
+Accumulo is often used as part of a larger data processing and storage system. To maximize the performance of a parallel system involving Accumulo, the ingestion and query components should be designed to provide enough parallelism and concurrency to avoid creating bottlenecks for users and other systems writing to and reading from Accumulo. There are several ways to achieve high ingest performance. 
+
+## <a id=SECTION00810000000000000000></a> Pre-Splitting New Tables
+
+New tables consist of a single tablet by default. As mutations are applied, the table grows and splits into multiple tablets which are balanced by the Master across TabletServers. This implies that the aggregate ingest rate will be limited to fewer servers than are available within the cluster until the table has reached the point where there are tablets on every TabletServer. 
+
+Pre-splitting a table ensures that there are as many tablets as desired available before ingest begins to take advantage of all the parallelism possible with the cluster hardware. Tables can be split anytime by using the shell: 
+    
+    
+    user@myinstance mytable> addsplits -sf /local_splitfile -t mytable
+    
+
+For the purposes of providing parallelism to ingest it is not necessary to create more tablets than there are physical machines within the cluster as the aggregate ingest rate is a function of the number of physical machines. Note that the aggregate ingest rate is still subject to the number of machines running ingest clients, and the distribution of rowIDs across the table. The aggregation ingest rate will be suboptimal if there are many inserts into a small number of rowIDs. 
+
+## <a id=SECTION00820000000000000000></a> Multiple Ingester Clients
+
+Accumulo is capable of scaling to very high rates of ingest, which is dependent upon not just the number of TabletServers in operation but also the number of ingest clients. This is because a single client, while capable of batching mutations and sending them to all TabletServers, is ultimately limited by the amount of data that can be processed on a single machine. The aggregate ingest rate will scale linearly with the number of clients up to the point at which either the aggregate I/O of TabletServers or total network bandwidth capacity is reached. 
+
+In operational settings where high rates of ingest are paramount, clusters are often configured to dedicate some number of machines solely to running Ingester Clients. The exact ratio of clients to TabletServers necessary for optimum ingestion rates will vary according to the distribution of resources per machine and by data type. 
+
+## <a id=SECTION00830000000000000000></a> Bulk Ingest
+
+Accumulo supports the ability to import files produced by an external process such as MapReduce into an existing table. In some cases it may be faster to load data this way rather than via ingesting through clients using BatchWriters. This allows a large number of machines to format data the way Accumulo expects. The new files can then simply be introduced to Accumulo via a shell command. 
+
+To configure MapReduce to format data in preparation for bulk loading, the job should be set to use a range partitioner instead of the default hash partitioner. The range partitioner uses the split points of the Accumulo table that will receive the data. The split points can be obtained from the shell and used by the MapReduce RangePartitioner. Note that this is only useful if the existing table is already split into multiple tablets. 
+    
+    
+    user@myinstance mytable> getsplits
+    aa
+    ab
+    ac
+    ...
+    zx
+    zy
+    zz
+    
+
+Run the MapReduce job, using the AccumuloFileOutputFormat to create the files to be introduced to Accumulo. Once this is complete, the files can be added to Accumulo via the shell: 
+    
+    
+    user@myinstance mytable> importdirectory /files_dir /failures
+    
+
+Note that the paths referenced are directories within the same HDFS instance over which Accumulo is running. Accumulo places any files that failed to be added to the second directory specified. 
+
+A complete example of using Bulk Ingest can be found at   
+accumulo/docs/examples/README.bulkIngest 
+
+## <a id=SECTION00840000000000000000></a> MapReduce Ingest
+
+It is possible to efficiently write many mutations to Accumulo in parallel via a MapReduce job. In this scenario the MapReduce is written to process data that lives in HDFS and write mutations to Accumulo using the AccumuloOutputFormat. See the MapReduce section under Analytics for details. 
+
+An example of using MapReduce can be found under   
+accumulo/docs/examples/README.mapred   
+
+
+* * *
+
+   [1]: High_Speed_Ingest.html#SECTION00810000000000000000
+   [2]: High_Speed_Ingest.html#SECTION00820000000000000000
+   [3]: High_Speed_Ingest.html#SECTION00830000000000000000
+   [4]: High_Speed_Ingest.html#SECTION00840000000000000000
+

Added: incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Introduction.mdtext
URL: http://svn.apache.org/viewvc/incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Introduction.mdtext?rev=1183037&view=auto
==============================================================================
--- incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Introduction.mdtext (added)
+++ incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Introduction.mdtext Thu Oct 13 19:13:58 2011
@@ -0,0 +1,9 @@
+Title: Accumulo User Manual: Introduction
+## <a id=SECTION00200000000000000000></a> Introduction
+
+Accumulo is a highly scalable structured store based on Google's BigTable. Accumulo is written in Java and operates over the Hadoop Distributed File System (HDFS), which is part of the popular Apache Hadoop project. Accumulo supports efficient storage and retrieval of structured data, including queries for ranges, and provides support for using Accumulo tables as input and output for MapReduce jobs. 
+
+Accumulo features automatic load-balancing and partitioning, data compression and fine-grained security labels.   
+
+
+* * *

Added: incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Security.mdtext
URL: http://svn.apache.org/viewvc/incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Security.mdtext?rev=1183037&view=auto
==============================================================================
--- incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Security.mdtext (added)
+++ incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Security.mdtext Thu Oct 13 19:13:58 2011
@@ -0,0 +1,114 @@
+Title: Accumulo User Manual: Security
+<a id=CHILD_LINKS></a>**Subsections**
+
+* [Security Label Expressions][1]
+* [Security Label Expression Syntax][2]
+* [Authorization][3]
+* [User Authorizations][4]
+* [Secure Authorizations Handling][5]
+* [Query Services Layer][6]
+
+* * *
+
+## <a id=SECTION001000000000000000000></a> Security
+
+Accumulo extends the BigTable data model to implement a security mechanism known as cell-level security. Every key-value pair has its own security label, stored under the column visibility element of the key, which is used to determine whether a given user meets the security requirements to read the value. This enables data of various security levels to be stored within the same row, and users of varying degrees of access to query the same table, while preserving data confidentiality. 
+
+## <a id=SECTION001010000000000000000></a> Security Label Expressions
+
+When mutations are applied, users can specify a security label for each value. This is done as the Mutation is created by passing a ColumnVisibility object to the put() method: 
+    
+    
+    Text rowID = new Text("row1");
+    Text colFam = new Text("myColFam");
+    Text colQual = new Text("myColQual");
+    ColumnVisibility colVis = new ColumnVisibility("public");
+    long timestamp = System.currentTimeMillis();
+    
+    Value value = new Value("myValue");
+    
+    Mutation mutation = new Mutation(rowID);
+    mutation.put(colFam, colQual, colVis, timestamp, value);
+    
+
+## <a id=SECTION001020000000000000000></a> Security Label Expression Syntax
+
+Security labels consist of a set of user-defined tokens that are required to read the value the label is associated with. The set of tokens required can be specified using syntax that supports logical AND and OR combinations of tokens, as well as nesting groups of tokens together. 
+
+For example, suppose within our organization we want to label our data values with security labels defined in terms of user roles. We might have tokens such as: 
+    
+    
+    admin
+    audit
+    system
+    
+
+These can be specified alone or combined using logical operators: 
+    
+    
+    // Users must have admin privileges:
+    admin
+    
+    // Users must have admin and audit privileges
+    admin&audit
+    
+    // Users with either admin or audit privileges
+    admin|audit
+    
+    // Users must have audit and one or both of admin or system
+    (admin|system)&audit
+    
+
+When both `|` and `&` operators are used, parentheses must be used to specify precedence of the operators. 
+
+## <a id=SECTION001030000000000000000></a> Authorization
+
+When clients attempt to read data from Accumulo, any security labels present are examined against the set of authorizations passed by the client code when the Scanner or BatchScanner are created. If the authorizations are determined to be insufficient to satisfy the security label, the value is suppressed from the set of results sent back to the client. 
+
+Authorizations are specified as a comma-separated list of tokens the user possesses: 
+    
+    
+    // user possess both admin and system level access
+    Authorization auths = new Authorization("admin","system");
+    
+    Scanner s = connector.createScanner("table", auths);
+    
+
+## <a id=SECTION001040000000000000000></a> User Authorizations
+
+Each accumulo user has a set of associated security labels. To manipulate these in the shell use the setuaths and getauths commands. These may also be modified using the java security operations API. 
+
+When a user creates a scanner a set of Authorizations is passed. If the authorizations passed to the scanner are not a subset of the users authorizations, then an exception will be thrown. 
+
+To prevent users from writing data they can not read, add the visibility constraint to a table. Use the -evc option in the createtable shell command to enable this constraint. For existing tables use the following shell command to enable the visibility constraint. Ensure the constraint number does not conflict with any existing constraints. 
+    
+    
+    config -t table -s table.constraint.1=org.apache.accumulo.core.security.VisibilityConstraint
+    
+
+Any user with the alter table permission can add or remove this constraint. This constraint is not applied to bulk imported data, if this a concern then disable the bulk import pesmission. 
+
+## <a id=SECTION001050000000000000000></a> Secure Authorizations Handling
+
+For applications serving many users, it is not expected that a accumulo user will be created for each application user. In this case a accumulo user with all authorizations needed by any of the applications users must be created. To service queries, the application should create a scanner with the application users authorizations. These authorizations could be obtined from a trusted 3rd party. 
+
+Often production systems will integrate with Public-Key Infrastructure (PKI) and designate client code within the query layer to negotiate with PKI servers in order to authenticate users and retrieve their authorization tokens (credentials). This requires users to specify only the information necessary to authenticate themselves to the system. Once user identity is established, their credentials can be accessed by the client code and passed to Accumulo outside of the reach of the user. 
+
+## <a id=SECTION001060000000000000000></a> Query Services Layer
+
+Since the primary method of interaction with Accumulo is through the Java API, production environments often call for the implementation of a Query layer. This can be done using web services in containers such as Apache Tomcat, but is not a requirement. The Query Services Layer provides a mechanism for providing a platform on which user facing applications can be built. This allows the application designers to isolate potentially complex query logic, and enables a convenient point at which to perform essential security functions. 
+
+Several production environments choose to implement authentication at this layer, where users identifiers are used to retrieve their access credentials which are then cached within the query layer and presented to Accumulo through the Authorizations mechanism. 
+
+Typically, the query services layer sits between Accumulo and user workstations.   
+
+
+* * *
+
+   [1]: Security.html#SECTION001010000000000000000
+   [2]: Security.html#SECTION001020000000000000000
+   [3]: Security.html#SECTION001030000000000000000
+   [4]: Security.html#SECTION001040000000000000000
+   [5]: Security.html#SECTION001050000000000000000
+   [6]: Security.html#SECTION001060000000000000000
+

Added: incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Shell_Commands.mdtext
URL: http://svn.apache.org/viewvc/incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Shell_Commands.mdtext?rev=1183037&view=auto
==============================================================================
--- incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Shell_Commands.mdtext (added)
+++ incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Shell_Commands.mdtext Thu Oct 13 19:13:58 2011
@@ -0,0 +1,628 @@
+Title: Accumulo User Manual: Shell Commands
+## <a id=SECTION001200000000000000000></a> Shell Commands
+
+**?**   
+  
+    usage: ? [ <command> <command> ] [-?] [-np]   
+    description: provides information about the available commands   
+      -?,-help  display this help   
+      -np,-no-pagination  disables pagination of output   
+  
+**about**   
+  
+    usage: about [-?] [-v]   
+    description: displays information about this program   
+      -?,-help  display this help   
+      -v,-verbose displays details session information   
+  
+**addsplits**   
+  
+    usage: addsplits [<split> <split> ] [-?] [-b64] [-sf <filename>] [-t   
+           <tableName>]   
+    description: add split points to an existing table   
+      -?,-help  display this help   
+      -b64,-base64encoded decode encoded split points   
+      -sf,-splits-file <filename> file with newline separated list of rows to add   
+           to table   
+      -t,-table <tableName>  name of a table to add split points to   
+  
+**authenticate**   
+  
+    usage: authenticate <username> [-?]   
+    description: verifies a user's credentials   
+      -?,-help  display this help   
+  
+**bye**   
+  
+    usage: bye [-?]   
+    description: exits the shell   
+      -?,-help  display this help   
+  
+**classpath**   
+  
+    usage: classpath [-?]   
+    description: lists the current files on the classpath   
+      -?,-help  display this help   
+  
+**clear**   
+  
+    usage: clear [-?]   
+    description: clears the screen   
+      -?,-help  display this help   
+  
+**clonetable**   
+  
+    usage: clonetable <current table name> <new table name> [-?] [-e <arg>] [-nf] [-s   
+           <arg>]   
+    description: clone a table   
+      -?,-help  display this help   
+      -e,-exclude <arg>  properties that should not be copied from source table.   
+           Expects <prop>,<prop>   
+      -nf,-noFlush  do not flush table data in memory before cloning.   
+      -s,-set <arg>  set initial properties before the table comes online. Expects   
+           <prop>=<value>,<prop>=<value>   
+  
+**cls**   
+  
+    usage: cls [-?]   
+    description: clears the screen   
+      -?,-help  display this help   
+  
+**compact**   
+  
+    usage: compact [-?] [-b <arg>] [-e <arg>] [-nf] [-p <pattern> | -t <tableName>]   
+           [-w]   
+    description: sets all tablets for a table to major compact as soon as possible   
+           (based on current time)   
+      -?,-help  display this help   
+      -b,-begin-row <arg> begin row   
+      -e,-end-row <arg>  end row   
+      -nf,-noFlush  do not flush table data in memory before compacting.   
+      -p,-pattern <pattern>  regex pattern of table names to flush   
+      -t,-table <tableName>  name of a table to flush   
+      -w,-wait  wait for compact to finish   
+  
+**config**   
+  
+    usage: config [-?] [-d <property> | -f <string> | -s <property=value>] [-np]   
+           [-t <table>]   
+    description: prints system properties and table specific properties   
+      -?,-help  display this help   
+      -d,-delete <property>  delete a per-table property   
+      -f,-filter <string> show only properties that contain this string   
+      -np,-no-pagination  disables pagination of output   
+      -s,-set <property=value>  set a per-table property   
+      -t,-table <table>  display/set/delete properties for specified table   
+  
+**createtable**   
+  
+    usage: createtable <tableName> [-?] [-a   
+           <<columnfamily>[:<columnqualifier>]=<aggregation_class>>] [-b64]   
+           [-cc <table>] [-cs <table> | -sf <filename>] [-evc] [-ndi]  [-tl |   
+           -tm]   
+    description: creates a new table, with optional aggregators and optionally   
+           pre-split   
+      -?,-help  display this help   
+      -a,-aggregator <<columnfamily>[:<columnqualifier>]=<aggregation_class>>   
+           comma separated column=aggregator   
+      -b64,-base64encoded decode encoded split points   
+      -cc,-copy-config <table>  table to copy configuration from   
+      -cs,-copy-splits <table>  table to copy current splits from   
+      -evc,-enable-visibility-constraint  prevents users from writing data they can   
+           not read.  When enabling this may want to consider disabling bulk   
+           import and alter table   
+      -ndi,-no-default-iterators  prevents creation of the normal default iterator   
+           set   
+      -sf,-splits-file <filename> file with newline separated list of rows to   
+           create a pre-split table   
+      -tl,-time-logical  use logical time   
+      -tm,-time-millis  use time in milliseconds   
+  
+**createuser**   
+  
+    usage: createuser <username> [-?] [-s <comma-separated-authorizations>]   
+    description: creates a new user   
+      -?,-help  display this help   
+      -s,-scan-authorizations <comma-separated-authorizations>  scan authorizations   
+  
+**debug**   
+  
+    usage: debug [ on | off ] [-?]   
+    description: turns debug logging on or off   
+      -?,-help  display this help   
+  
+**delete**   
+  
+    usage: delete <row> <colfamily> <colqualifier> [-?] [-l <expression>] [-t   
+           <timestamp>]   
+    description: deletes a record from a table   
+      -?,-help  display this help   
+      -l,-authorization-label <expression>  formatted authorization label expression   
+      -t,-timestamp <timestamp>  timestamp to use for insert   
+  
+**deleteiter**   
+  
+    usage: deleteiter [-?] [-majc] [-minc] -n <itername> [-scan] [-t <table>]   
+    description: deletes a table-specific iterator   
+      -?,-help  display this help   
+      -majc,-major-compaction  applied at major compaction   
+      -minc,-minor-compaction  applied at minor compaction   
+      -n,-name <itername> iterator to delete   
+      -scan,-scan-time  applied at scan time   
+      -t,-table <table>  tableName   
+  
+**deletemany**   
+  
+    usage: deletemany [-?] [-b <start-row>] [-c   
+           <<columnfamily>[:<columnqualifier>]>] [-e <end-row>] [-f] [-np]   
+           [-s <comma-separated-authorizations>] [-st] [-t <table>]   
+    description: scans a table and deletes the resulting records   
+      -?,-help  display this help   
+      -b,-begin-row <start-row>  begin row (inclusive)   
+      -c,-columns <<columnfamily>[:<columnqualifier>]>  comma-separated columns   
+      -e,-end-row <end-row>  end row (inclusive)   
+      -f,-force  forces deletion without prompting   
+      -np,-no-pagination  disables pagination of output   
+      -s,-scan-authorizations <comma-separated-authorizations>  scan authorizations   
+           (all user auths are used if this argument is not specified)   
+      -st,-show-timestamps  enables displaying timestamps   
+      -t,-table <table>  table to be created   
+  
+**deleterows**   
+  
+    usage: deleterows [-?] [-b <arg>] [-e <arg>] [-f] [-t <table>]   
+    description: delete a range of rows in a table   
+    Note that rows matching the start row ARE NOT deleted, but rows matching the end   
+           row ARE deleted.   
+      -?,-help  display this help   
+      -b,-begin-row <arg> begin row   
+      -e,-end-row <arg>  end row   
+      -f,-force  delete data even if start or end are not specified   
+      -t,-tableName <table>  table to delete row range   
+  
+**Note**   
+  
+  
+**deletescaniter**   
+  
+    usage: deletescaniter [-?] [-a] [-n <itername>] [-t <table>]   
+    description: deletes a table-specific scan iterator so it is no longer used   
+           during this shell session   
+      -?,-help  display this help   
+      -a,-all  delete all for tableName   
+      -n,-name <itername> iterator to delete   
+      -t,-table <table>  tableName   
+  
+**deletetable**   
+  
+    usage: deletetable <tableName> [-?] [-t <arg>]   
+    description: deletes a table   
+      -?,-help  display this help   
+      -t,-tableName <arg> deletes a table   
+  
+**deleteuser**   
+  
+    usage: deleteuser <username> [-?]   
+    description: deletes a user   
+      -?,-help  display this help   
+  
+**droptable**   
+  
+    usage: droptable <tableName> [-?] [-t <arg>]   
+    description: deletes a table   
+      -?,-help  display this help   
+      -t,-tableName <arg> deletes a table   
+  
+**dropuser**   
+  
+    usage: dropuser <username> [-?]   
+    description: deletes a user   
+      -?,-help  display this help   
+  
+**du**   
+  
+    usage: du <table> <table> [-?] [-p <pattern>]   
+    description: Prints how much space is used by files referenced by a table.  When   
+           multiple tables are specified it prints how much space is used by   
+           files shared between tables, if any.   
+      -?,-help  display this help   
+      -p,-pattern <pattern>  regex pattern of table names   
+  
+**egrep**   
+  
+    usage: egrep <regex> <regex> [-?] [-b <start-row>] [-c   
+           <<columnfamily>[:<columnqualifier>]>] [-e <end-row>] [-f <int>]   
+           [-np] [-s <comma-separated-authorizations>] [-st] [-t <arg>]   
+    description: searches each row, column family, column qualifier and value, in   
+           parallel, on the server side (using a java regular expression)   
+      -?,-help  display this help   
+      -b,-begin-row <start-row>  begin row (inclusive)   
+      -c,-columns <<columnfamily>[:<columnqualifier>]>  comma-separated columns   
+      -e,-end-row <end-row>  end row (inclusive)   
+      -f,-show few <int>  Only shows certain amount of characters   
+      -np,-no-pagination  disables pagination of output   
+      -s,-scan-authorizations <comma-separated-authorizations>  scan authorizations   
+           (all user auths are used if this argument is not specified)   
+      -st,-show-timestamps  enables displaying timestamps   
+      -t,-num-threads <arg>  num threads   
+  
+**execfile**   
+  
+    usage: execfile [-?] [-v]   
+    description: specifies a file containing accumulo commands to execute   
+      -?,-help  display this help   
+      -v,-verbose displays command prompt as commands are executed   
+  
+**exit**   
+  
+    usage: exit [-?]   
+    description: exits the shell   
+      -?,-help  display this help   
+  
+**flush**   
+  
+    usage: flush [-?] [-b <arg>] [-e <arg>] [-p <pattern> | -t <tableName>]  [-w]   
+    description: flushes a tables data that is currently in memory to disk   
+      -?,-help  display this help   
+      -b,-begin-row <arg> begin row   
+      -e,-end-row <arg>  end row   
+      -p,-pattern <pattern>  regex pattern of table names to flush   
+      -t,-table <tableName>  name of a table to flush   
+      -w,-wait  wait for flush to finish   
+  
+**formatter**   
+  
+    usage: formatter [-?] -f <className> | -l | -r   
+    description: specifies a formatter to use for displaying database entries   
+      -?,-help  display this help   
+      -f,-formatter <className>  fully qualified name of formatter class to use   
+      -l,-list  display the current formatter   
+      -r,-reset  reset to default formatter   
+  
+**getauths**   
+  
+    usage: getauths [-?] [-u <user>]   
+    description: displays the maximum scan authorizations for a user   
+      -?,-help  display this help   
+      -u,-user <user>  user to operate on   
+  
+**getgroups**   
+  
+    usage: getgroups [-?] [-t <table>]   
+    description: gets the locality groups for a given table   
+      -?,-help  display this help   
+      -t,-table <table>  get locality groups for specified table   
+  
+**getsplits**   
+  
+    usage: getsplits [-?] [-b64] [-m <num>] [-o <file>] [-t <table>] [-v]   
+    description: retrieves the current split points for tablets in the current table   
+      -?,-help  display this help   
+      -b64,-base64encoded encode the split points   
+      -m,-max <num>  specifies the maximum number of splits to create   
+      -o,-output <file>  specifies a local file to write the splits to   
+      -t,-tableName <table>  table to get splits on   
+      -v,-verbose print out the tablet information with start/end rows   
+  
+**grant**   
+  
+    usage: grant <permission> [-?] -p <pattern> | -s | -t <table>  -u <username>   
+    description: grants system or table permissions for a user   
+      -?,-help  display this help   
+      -p,-pattern <pattern>  regex pattern of tables to grant permissions on   
+      -s,-system  grant a system permission   
+      -t,-table <table>  grant a table permission on this table   
+      -u,-user <username> user to operate on   
+  
+**grep**   
+  
+    usage: grep <term> <term> [-?] [-b <start-row>] [-c   
+           <<columnfamily>[:<columnqualifier>]>] [-e <end-row>] [-f <int>]   
+           [-np] [-s <comma-separated-authorizations>] [-st] [-t <arg>]   
+    description: searches each row, column family, column qualifier and value in a   
+           table for a substring (not a regular expression), in parallel, on   
+           the server side   
+      -?,-help  display this help   
+      -b,-begin-row <start-row>  begin row (inclusive)   
+      -c,-columns <<columnfamily>[:<columnqualifier>]>  comma-separated columns   
+      -e,-end-row <end-row>  end row (inclusive)   
+      -f,-show few <int>  Only shows certain amount of characters   
+      -np,-no-pagination  disables pagination of output   
+      -s,-scan-authorizations <comma-separated-authorizations>  scan authorizations   
+           (all user auths are used if this argument is not specified)   
+      -st,-show-timestamps  enables displaying timestamps   
+      -t,-num-threads <arg>  num threads   
+  
+**help**   
+  
+    usage: help [ <command> <command> ] [-?] [-np]   
+    description: provides information about the available commands   
+      -?,-help  display this help   
+      -np,-no-pagination  disables pagination of output   
+  
+**history**   
+  
+    usage: history [-?] [-c]   
+    description: Generates a list of commands previously executed   
+      -?,-help  display this help   
+      -c,-Clears History, takes no arguments.  Clears History File   
+  
+**importdirectory**   
+  
+    usage: importdirectory <directory> <failureDirectory> true|false [-?]   
+    description: bulk imports an entire directory of data files to the current table.   
+            The boolean argument determines if accumulo sets the time.   
+      -?,-help  display this help   
+  
+**info**   
+  
+    usage: info [-?] [-v]   
+    description: displays information about this program   
+      -?,-help  display this help   
+      -v,-verbose displays details session information   
+  
+**insert**   
+  
+    usage: insert <row> <colfamily> <colqualifier> <value> [-?] [-l <expression>] [-t   
+           <timestamp>]   
+    description: inserts a record   
+      -?,-help  display this help   
+      -l,-authorization-label <expression>  formatted authorization label expression   
+      -t,-timestamp <timestamp>  timestamp to use for insert   
+  
+**listscans**   
+  
+    usage: listscans [-?] [-np] [-ts <tablet server>]   
+    description: list what scans are currently running in accumulo. See the   
+           accumulo.core.client.admin.ActiveScan javadoc for more information   
+           about columns.   
+      -?,-help  display this help   
+      -np,-no-pagination  disables pagination of output   
+      -ts,-tabletServer <tablet server>  list scans for a specific tablet server   
+  
+**masterstate**   
+  
+    usage: use the command line utility instead [-?]   
+    description: DEPRECATED: use the command line utility instead   
+      -?,-help  display this help   
+  
+**maxrow**   
+  
+    usage: maxrow [-?] [-b <begin-row>] [-be] [-e <end-row>] [-ee] [-s   
+           <comma-separated-authorizations>] [-t <table>]   
+    description: find the max row in a table within a given range   
+      -?,-help  display this help   
+      -b,-begin-row <begin-row>  begin row   
+      -be,-begin-exclusive  make start row exclusive, by defaults it inclusive   
+      -e,-end-row <end-row>  end row   
+      -ee,-end-exclusive  make end row exclusive, by defaults it inclusive   
+      -s,-scan-authorizations <comma-separated-authorizations>  scan authorizations   
+           (all user auths are used if this argument is not specified)   
+      -t,-table <table>  table to be created   
+  
+**merge**   
+  
+    usage: merge [-?] [-b <arg>] [-e <arg>] [-f] [-s <arg>] [-t <table>] [-v]   
+    description: merge tablets in a table   
+      -?,-help  display this help   
+      -b,-begin-row <arg> begin row   
+      -e,-end-row <arg>  end row   
+      -f,-force  merge small tablets to large tablets, even if it goes over the   
+           given size   
+      -s,-size <arg>  merge tablets to the given size over the entire table   
+      -t,-tableName <table>  table to be merged   
+      -v,-verbose verbose output during merge   
+  
+**notable**   
+  
+    usage: notable [-?] [-t <arg>]   
+    description: returns to a tableless shell state   
+      -?,-help  display this help   
+      -t,-tableName <arg> Returns to a no table state   
+  
+**offline**   
+  
+    usage: offline [-?] [-p <pattern> | -t <tableName>]   
+    description: starts the process of taking table offline   
+      -?,-help  display this help   
+      -p,-pattern <pattern>  regex pattern of table names to flush   
+      -t,-table <tableName>  name of a table to flush   
+  
+**online**   
+  
+    usage: online [-?] [-p <pattern> | -t <tableName>]   
+    description: starts the process of putting a table online   
+      -?,-help  display this help   
+      -p,-pattern <pattern>  regex pattern of table names to flush   
+      -t,-table <tableName>  name of a table to flush   
+  
+**passwd**   
+  
+    usage: passwd [-?] [-u <user>]   
+    description: changes a user's password   
+      -?,-help  display this help   
+      -u,-user <user>  user to operate on   
+  
+**quit**   
+  
+    usage: quit [-?]   
+    description: exits the shell   
+      -?,-help  display this help   
+  
+**renametable**   
+  
+    usage: renametable <current table name> <new table name> [-?]   
+    description: rename a table   
+      -?,-help  display this help   
+  
+**revoke**   
+  
+    usage: revoke <permission> [-?] -s | -t <table>  -u <username>   
+    description: revokes system or table permissions from a user   
+      -?,-help  display this help   
+      -s,-system  revoke a system permission   
+      -t,-table <table>  revoke a table permission on this table   
+      -u,-user <username> user to operate on   
+  
+**scan**   
+  
+    usage: scan [-?] [-b <start-row>] [-c <<columnfamily>[:<columnqualifier>]>] [-e   
+           <end-row>] [-f <int>] [-np] [-s <comma-separated-authorizations>]   
+           [-st] [-t <table>]   
+    description: scans the table, and displays the resulting records   
+      -?,-help  display this help   
+      -b,-begin-row <start-row>  begin row (inclusive)   
+      -c,-columns <<columnfamily>[:<columnqualifier>]>  comma-separated columns   
+      -e,-end-row <end-row>  end row (inclusive)   
+      -f,-show few <int>  Only shows certain amount of characters   
+      -np,-no-pagination  disables pagination of output   
+      -s,-scan-authorizations <comma-separated-authorizations>  scan authorizations   
+           (all user auths are used if this argument is not specified)   
+      -st,-show-timestamps  enables displaying timestamps   
+      -t,-tableName <table>  table to be scanned   
+  
+**select**   
+  
+    usage: select <row> <columnfamily> <columnqualifier> [-?] [-np] [-s   
+           <comma-separated-authorizations>] [-st] [-t <table>]   
+    description: scans for and displays a single record   
+      -?,-help  display this help   
+      -np,-no-pagination  disables pagination of output   
+      -s,-scan-authorizations <comma-separated-authorizations>  scan authorizations   
+      -st,-show-timestamps  enables displaying timestamps   
+      -t,-tableName <table>  table   
+  
+**selectrow**   
+  
+    usage: selectrow <row> [-?] [-np] [-s <comma-separated-authorizations>] [-st] [-t   
+           <table>]   
+    description: scans a single row and displays all resulting records   
+      -?,-help  display this help   
+      -np,-no-pagination  disables pagination of output   
+      -s,-scan-authorizations <comma-separated-authorizations>  scan authorizations   
+      -st,-show-timestamps  enables displaying timestamps   
+      -t,-tableName <table>  table to row select   
+  
+**setauths**   
+  
+    usage: setauths [-?] -c | -s <comma-separated-authorizations>  [-u <user>]   
+    description: sets the maximum scan authorizations for a user   
+      -?,-help  display this help   
+      -c,-clear-authorizations  clears the scan authorizations   
+      -s,-scan-authorizations <comma-separated-authorizations>  set the scan   
+           authorizations   
+      -u,-user <user>  user to operate on   
+  
+**setgroups**   
+  
+    usage: setgroups <group>=<col fam>,<col fam> <group>=<col fam>,<col fam>   
+           [-?] [-t <table>]   
+    description: sets the locality groups for a given table (for binary or commas,   
+           use Java API)   
+      -?,-help  display this help   
+      -t,-table <table>  get locality groups for specified table   
+  
+**setiter**   
+  
+    usage: setiter [-?] -ageoff | -agg | -class <name> | -nolabel | -regex | -vers   
+           [-majc] [-minc] [-n <itername>]  -p <pri>  [-scan] [-t <table>]   
+    description: sets a table-specific iterator   
+      -?,-help  display this help   
+      -ageoff,-ageoff  an aging off type   
+      -agg,-aggregator  an aggregating type   
+      -class,-class-name <name>  a java class type   
+      -majc,-major-compaction  applied at major compaction   
+      -minc,-minor-compaction  applied at minor compaction   
+      -n,-name <itername> iterator to set   
+      -nolabel,-no-label  a no-labeling type   
+      -p,-priority <pri>  the order in which the iterator is applied   
+      -regex,-regular-expression  a regex matching type   
+      -scan,-scan-time  applied at scan time   
+      -t,-table <table>  tableName   
+      -vers,-version  a versioning type   
+  
+**setscaniter**   
+  
+    usage: setscaniter [-?] -ageoff | -agg | -class <name> | -nolabel | -regex |   
+           -vers  [-n <itername>]  -p <pri> [-t <table>]   
+    description: sets a table-specific scan iterator for this shell session   
+      -?,-help  display this help   
+      -ageoff,-ageoff  an aging off type   
+      -agg,-aggregator  an aggregating type   
+      -class,-class-name <name>  a java class type   
+      -n,-name <itername> iterator to set   
+      -nolabel,-no-label  a no-labeling type   
+      -p,-priority <pri>  the order in which the iterator is applied   
+      -regex,-regular-expression  a regex matching type   
+      -t,-table <table>  tableName   
+      -vers,-version  a versioning type   
+  
+**sleep**   
+  
+    usage: sleep [-?]   
+    description: sleep for the given number of seconds   
+      -?,-help  display this help   
+  
+**systempermissions**   
+  
+    usage: systempermissions [-?]   
+    description: displays a list of valid system permissions   
+      -?,-help  display this help   
+  
+**table**   
+  
+    usage: table <tableName> [-?]   
+    description: switches to the specified table   
+      -?,-help  display this help   
+  
+**tablepermissions**   
+  
+    usage: tablepermissions [-?]   
+    description: displays a list of valid table permissions   
+      -?,-help  display this help   
+  
+**tables**   
+  
+    usage: tables [-?] [-l]   
+    description: displays a list of all existing tables   
+      -?,-help  display this help   
+      -l,-list-ids  display internal table ids along with the table name   
+  
+**trace**   
+  
+    usage: trace [ on | off ] [-?]   
+    description: turns trace logging on or off   
+      -?,-help  display this help   
+  
+**user**   
+  
+    usage: user <username> [-?]   
+    description: switches to the specified user   
+      -?,-help  display this help   
+  
+**userpermissions**   
+  
+    usage: userpermissions [-?] [-u <user>]   
+    description: displays a user's system and table permissions   
+      -?,-help  display this help   
+      -u,-user <user>  user to operate on   
+  
+**users**   
+  
+    usage: users [-?]   
+    description: displays a list of existing users   
+      -?,-help  display this help   
+  
+**whoami**   
+  
+    usage: whoami [-?]   
+    description: reports the current user name   
+      -?,-help  display this help   
+  
+  
+
+
+  
+
+
+* * *

Added: incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Table_Configuration.mdtext
URL: http://svn.apache.org/viewvc/incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Table_Configuration.mdtext?rev=1183037&view=auto
==============================================================================
--- incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Table_Configuration.mdtext (added)
+++ incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Table_Configuration.mdtext Thu Oct 13 19:13:58 2011
@@ -0,0 +1,353 @@
+Title: Accumulo User Manual: Table Configuration
+<a id=CHILD_LINKS></a>**Subsections**
+
+* [Locality Groups][1]
+* [Constraints][2]
+* [Bloom Filters][3]
+* [Iterators][4]
+* [Versioning Iterators and Timestamps][5]
+* [Filters][6]
+* [Aggregating Iterators][7]
+* [Pre-splitting tables][8]
+* [Merging tables][9]
+* [Delete Range][10]
+
+* * *
+
+## <a id=SECTION00600000000000000000></a> Table Configuration
+
+Accumulo tables have a few options that can be configured to alter the default behavior of Accumulo as well as improve performance based on the data stored. These include locality groups, constraints, and iterators. 
+
+## <a id=SECTION00610000000000000000></a> Locality Groups
+
+Accumulo supports storing of sets of column families separately on disk to allow clients to scan over columns that are frequently used together efficient and to avoid scanning over column families that are not requested. After a locality group is set Scanner and BatchScanner operations will automatically take advantage of them whenever the fetchColumnFamilies() method is used. 
+
+By default tables place all column families into the same ``default" locality group. Additional locality groups can be configured anytime via the shell or programmatically as follows: 
+
+### <a id=SECTION00611000000000000000></a> Managing Locality Groups via the Shell
+    
+    
+    usage: setgroups <group>=<col fam>{,<col fam>}{ <group>=<col fam>{,<col
+    fam>}} [-?] -t <table>
+    
+    user@myinstance mytable> setgroups -t mytable group_one=colf1,colf2
+    
+    user@myinstance mytable> getgroups -t mytable
+    group_one=colf1,colf2
+    
+
+### <a id=SECTION00612000000000000000></a> Managing Locality Groups via the Client API
+    
+    
+    Connector conn;
+    
+    HashMap<String,Set<Text>> localityGroups =
+        new HashMap<String, Set<Text>>();
+    
+    HashSet<Text> metadataColumns = new HashSet<Text>();
+    metadataColumns.add(new Text("domain"));
+    metadataColumns.add(new Text("link"));
+    
+    HashSet<Text> contentColumns = new HashSet<Text>();
+    contentColumns.add(new Text("body"));
+    contentColumns.add(new Text("images"));
+    
+    localityGroups.put("metadata", metadataColumns);
+    localityGroups.put("content", contentColumns);
+    
+    conn.tableOperations().setLocalityGroups("mytable", localityGroups);
+    
+    // existing locality groups can be obtained as follows
+    Map<String, Set<Text>> groups =
+        conn.tableOperations().getLocalityGroups("mytable");
+    
+
+The assignment of Column Families to Locality Groups can be changed anytime. The physical movement of column families into their new locality groups takes place via the periodic Major Compaction process that takes place continuously in the background. Major Compaction can also be scheduled to take place immediately through the shell: 
+    
+    
+    user@myinstance mytable> compact -t mytable
+    
+
+## <a id=SECTION00620000000000000000></a> Constraints
+
+Accumulo supports constraints applied on mutations at insert time. This can be used to disallow certain inserts according to a user defined policy. Any mutation that fails to meet the requirements of the constraint is rejected and sent back to the client. 
+
+Constraints can be enabled by setting a table property as follows: 
+    
+    
+    user@myinstance mytable> config -t mytable -s table.constraint.1=com.test.ExampleConstraint
+    user@myinstance mytable> config -t mytable -s table.constraint.2=com.test.AnotherConstraint
+    user@myinstance mytable> config -t mytable -f constraint
+    ---------+--------------------------------+----------------------------
+    SCOPE    | NAME                           | VALUE
+    ---------+--------------------------------+----------------------------
+    table    | table.constraint.1............ | com.test.ExampleConstraint
+    table    | table.constraint.2............ | com.test.AnotherConstraint
+    ---------+--------------------------------+----------------------------
+    
+
+Currently there are no general-purpose constraints provided with the Accumulo distribution. New constraints can be created by writing a Java class that implements the org.apache.accumulo.core.constraints.Constraint interface. 
+
+To deploy a new constraint, create a jar file containing the class implementing the new constraint and place it in the lib directory of the Accumulo installation. New constraint jars can be added to Accumulo and enabled without restarting but any change to an existing constraint class requires Accumulo to be restarted. 
+
+An example of constraints can be found in   
+accumulo/docs/examples/README.constraints with corresponding code under   
+accumulo/src/examples/main/java/accumulo/examples/constraints . 
+
+## <a id=SECTION00630000000000000000></a> Bloom Filters
+
+As mutations are applied to a Accumulo table, several files are created per tablet. If bloom filters are enabled, Accumulo will create and load a small data structure into memory to determine whether a file contains a given key before opening the file. This can speed up lookups considerably. 
+
+To enable bloom filters, enter the following command in the Shell: 
+    
+    
+    user@myinstance> config -t mytable -s table.bloom.enabled=true
+    
+
+An extensive example of using Bloom Filters can be found at   
+accumulo/docs/examples/README.bloom . 
+
+## <a id=SECTION00640000000000000000></a> Iterators
+
+Iterators provide a modular mechanism for adding functionality to be executed by TabletServers when scanning or compacting data. This allows users to efficiently summarize, filter, and aggregate data. In fact, the built-in features of cell-level security and column fetching are implemented using Iterators. Some useful Iterators are provided with Accumulo and can be found in the org.apache.accumulo.core.iterators.user package. 
+
+### <a id=SECTION00641000000000000000></a> Setting Iterators via the Shell
+    
+    
+    usage: setiter [-?] -agg | -class <name> | -filter | -nolabel | 
+    -regex | -vers [-majc] [-minc] [-n <itername>] -p <pri> [-scan] 
+    [-t <table>]
+    
+    user@myinstance mytable> setiter -t mytable -scan -p 10 -n myiter
+    
+
+### <a id=SECTION00642000000000000000></a> Setting Iterators Programmatically
+    
+    
+    scanner.addIterator(new IteratorSetting(
+        15, // priority
+        "com.company.MyIterator", // class name
+        "myiter" // name this iterator
+    ));
+    
+
+Some iterators take additional parameters from client code, as in the following example: 
+    
+    
+    IteratorSetting iter = new IteratorSetting(...);
+    iter.addOption("myoptionname", "myoptionvalue");
+    scanner.addIterator(iter)
+    
+
+Tables support separate Iterator settings to be applied at scan time, upon minor compaction and upon major compaction. For most uses, tables will have identical iterator settings for all three to avoid inconsistent results. 
+
+## <a id=SECTION00650000000000000000></a> Versioning Iterators and Timestamps
+
+Accumulo provides the capability to manage versioned data through the use of timestamps within the Key. If a timestamp is not specified in the key created by the client then the system will set the timestamp to the current time. Two keys with identical rowIDs and columns but different timestamps are considered two versions of the same key. If two inserts are made into accumulo with the same rowID, column, and timestamp, then the behavior is non-deterministic. 
+
+Timestamps are sorted in descending order, so the most recent data comes first. Accumulo can be configured to return the top k versions, or versions later than a given date. The default is to return the one most recent version. 
+
+The version policy can be changed by changing the VersioningIterator options for a table as follows: 
+    
+    
+    user@myinstance mytable> config -t mytable -s
+    table.iterator.scan.vers.opt.maxVersions=3
+    
+    user@myinstance mytable> config -t mytable -s
+    table.iterator.minc.vers.opt.maxVersions=3
+    
+    user@myinstance mytable> config -t mytable -s
+    table.iterator.majc.vers.opt.maxVersions=3
+    
+
+### <a id=SECTION00651000000000000000></a> Logical Time
+
+Accumulo 1.2 introduces the concept of logical time. This ensures that timestamps set by accumulo always move forward. This helps avoid problems caused by TabletServers that have different time settings. The per tablet counter gives unique one up time stamps on a per mutation basis. When using time in milliseconds, if two things arrive within the same millisecond then both receive the same timestamp. 
+
+A table can be configured to use logical timestamps at creation time as follows: 
+    
+    
+    user@myinstance> createtable -tl logical
+    
+
+### <a id=SECTION00652000000000000000></a> Deletes
+
+Deletes are special keys in accumulo that get sorted along will all the other data. When a delete key is inserted, accumulo will not show anything that has a timestamp less than or equal to the delete key. During major compaction, any keys older than a delete key are omitted from the new file created, and the omitted keys are removed from disk as part of the regular garbage collection process. 
+
+## <a id=SECTION00660000000000000000></a> Filters
+
+When scanning over a set of key-value pairs it is possible to apply an arbitrary filtering policy through the use of a Filter. Filters are types of iterators that return only key-value pairs that satisfy the filter logic. Accumulo has a few built-in filters that can be configured on any table: AgeOff, ColumnAgeOff, Timestamp, NoVis, and RegEx. More can be added by writing a Java class that extends the   
+org.apache.accumulo.core.iterators.Filter class. 
+
+The AgeOff filter can be configured to remove data older than a certain date or a fixed amount of time from the present. The following example sets a table to delete everything inserted over 30 seconds ago: 
+    
+    
+    user@myinstance> createtable filtertest
+    user@myinstance filtertest> setiter -t filtertest -scan -minc -majc -p 10 -n myfilter -ageoff
+    AgeOffFilter removes entries with timestamps more than <ttl> milliseconds old
+    ----------> set org.apache.accumulo.core.iterators.user.AgeOffFilter parameter negate, default false keeps k/v that pass accept method, true rejects k/v that pass accept method: 
+    ----------> set org.apache.accumulo.core.iterators.user.AgeOffFilter parameter ttl, time to live (milliseconds): 3000
+    ----------> set org.apache.accumulo.core.iterators.user.AgeOffFilter parameter currentTime, if set, use the given value as the absolute time in milliseconds as the current time of day: 
+    user@myinstance filtertest> 
+    user@myinstance filtertest> scan
+    user@myinstance filtertest> insert foo a b c
+    user@myinstance filtertest> scan
+    foo a:b [] c
+    user@myinstance filtertest> sleep 4
+    user@myinstance filtertest> scan
+    user@myinstance filtertest>
+    
+
+To see the iterator settings for a table, use: 
+    
+    
+    user@example filtertest> config -t filtertest -f iterator
+    ---------+---------------------------------------------+------------------
+    SCOPE    | NAME                                        | VALUE
+    ---------+---------------------------------------------+------------------
+    table    | table.iterator.majc.myfilter .............. | 10,org.apache.accumulo.core.iterators.user.AgeOffFilter
+    table    | table.iterator.majc.myfilter.opt.ttl ...... | 3000
+    table    | table.iterator.majc.vers .................. | 20,org.apache.accumulo.core.iterators.VersioningIterator
+    table    | table.iterator.majc.vers.opt.maxVersions .. | 1
+    table    | table.iterator.minc.myfilter .............. | 10,org.apache.accumulo.core.iterators.user.AgeOffFilter
+    table    | table.iterator.minc.myfilter.opt.ttl ...... | 3000
+    table    | table.iterator.minc.vers .................. | 20,org.apache.accumulo.core.iterators.VersioningIterator
+    table    | table.iterator.minc.vers.opt.maxVersions .. | 1
+    table    | table.iterator.scan.myfilter .............. | 10,org.apache.accumulo.core.iterators.user.AgeOffFilter
+    table    | table.iterator.scan.myfilter.opt.ttl ...... | 3000
+    table    | table.iterator.scan.vers .................. | 20,org.apache.accumulo.core.iterators.VersioningIterator
+    table    | table.iterator.scan.vers.opt.maxVersions .. | 1
+    ---------+------------------------------------------+------------------
+    
+
+## <a id=SECTION00670000000000000000></a> Aggregating Iterators
+
+Accumulo allows aggregating iterators to be configured on tables and column families. When an aggregating iterator is set, the iterator is applied across the values associated with any keys that share rowID, column family, and column qualifier. This is similar to the reduce step in MapReduce, which applied some function to all the values associated with a particular key. 
+
+For example, if an aggregating iterator were configured on a table and the following mutations were inserted: 
+    
+    
+    Row     Family Qualifier Timestamp  Value
+    rowID1  colfA  colqA     20100101   1
+    rowID1  colfA  colqA     20100102   1
+    
+
+The table would reflect only one aggregate value: 
+    
+    
+    rowID1  colfA  colqA     -          2
+    
+
+Aggregating iterators can be enabled for a table as follows: 
+    
+    
+    user@myinstance> createtable perDayCounts -a
+    day=org.apache.accumulo.core.iterators.aggregation.StringSummation
+    
+    user@myinstance perDayCounts> insert row1 day 20080101 1
+    user@myinstance perDayCounts> insert row1 day 20080101 1
+    user@myinstance perDayCounts> insert row1 day 20080103 1
+    user@myinstance perDayCounts> insert row2 day 20080101 1
+    user@myinstance perDayCounts> insert row3 day 20080101 1
+    
+    user@myinstance perDayCounts> scan
+    row1 day:20080101 [] 2
+    row1 day:20080103 [] 1
+    row2 day:20080101 [] 2
+    
+
+Accumulo includes the following aggregators: 
+
+* **LongSummation**: expects values of type long and adds them. 
+* **StringSummation**: expects numbers represented as strings and adds them. 
+* **StringMax**: expects numbers as strings and retains the maximum number inserted. 
+* **StringMin**: expects numbers as strings and retains the minimum number inserted. 
+
+Additional Aggregators can be added by creating a Java class that implements   
+**org.apache.accumulo.core.iterators.aggregation.Aggregator** and adding a jar containing that class to Accumulo's lib directory. 
+
+An example of an aggregator can be found under   
+accumulo/src/examples/main/java/accumulo/examples/aggregation/SortedSetAggregator.java 
+
+## <a id=SECTION00680000000000000000></a> Pre-splitting tables
+
+Accumulo will balance and distribute tables accross servers. Before a table gets large, it will be maintained as a single tablet on a single server. This limits the speed at which data can be added or queried to the speed of a single node. To improve performance when the a table is new, or small, you can add split points and generate new tablets. 
+
+In the shell: 
+    
+    
+    root@myinstance> createtable newTable
+    root@myinstance> addsplits -t newTable g n t
+    
+
+This will create a new table with 4 tablets. The table will be split on the letters ``g'', ``n'', and ``t'' which will work nicely if the row data start with lower-case alphabetic characters. If your row data includes binary information or numeric information, or if the distribution of the row information is not flat, then you would pick different split points. Now ingest and query can proceed on 4 nodes which can improve performance. 
+
+## <a id=SECTION00690000000000000000></a> Merging tables
+
+Over time, a table can get very large, so large that it has hundreds of thousands of split points. Once there are enough tablets to spread a table across the entire cluster, additional splits may not improve performance, and may create unnecessary bookkeeping. The distribution of data may change over time. For example, if row data contains date information, and data is continually added and removed to maintain a window of current information, tablets for older rows may be empty. 
+
+Accumulo supports tablet merging, which can be used to reduce delete the number of split points. The following command will merge all rows from ``A'' to ``Z'' into a single tablet: 
+    
+    
+    root@myinstance> merge -t myTable -s A -e Z
+    
+
+If the result of a merge produces a tablet that is larger than the configured split size, the tablet may be split by the tablet server. Be sure to increase your tablet size prior to any merges if the goal is to have larger tablets: 
+    
+    
+    root@myinstance> config -t myTable -s table.split.threshold=2G
+    
+
+In order to merge small tablets, you can ask accumulo to merge sections of a table smaller than a given size. 
+    
+    
+    root@myinstance> merge -t myTable -s 100M
+    
+
+By default, small tablets will not be merged into tablets that are already larger than the given size. This can leave isolated small tablets. To force small tablets to be merged into larger tablets use the ``-force'' option: 
+    
+    
+    root@myinstance> merge -t myTable -s 100M --force
+    
+
+Merging away small tablets works on one section at a time. If your table contains many sections of small split points, or you are attempting to change the split size of the entire table, it will be faster to set the split point and merge the entire table: 
+    
+    
+    root@myinstance> config -t myTable -s table.split.threshold=256M
+    root@myinstance> merge -t myTable
+    
+
+## <a id=SECTION006100000000000000000></a> Delete Range
+
+Consider an indexing scheme that uses date information in each row. For example ``20110823-15:20:25.013'' might be a row that specifies a date and time. In some cases, we might like to delete rows based on this date, say to remove all the data older than the current year. Accumulo supports a delete range operation which can efficiently remove data between two rows. For example: 
+    
+    
+    root@myinstance> deleterange -t myTable -s 2010 -e 2011
+    
+
+This will delete all rows starting with ``2010'' and it will stop at any row starting ``2011''. You can delete any data prior to 2011 with: 
+    
+    
+    root@myinstance> deleterange -t myTable -e 2011 --force
+    
+
+The shell will not allow you to delete an unbounded range (no start) unless you provide the ``-force'' option. 
+
+Range deletion is implemented using splits at the given start/end positions, and will affect the number of splits in the table. 
+
+  
+
+
+* * *
+
+   [1]: Table_Configuration.html#SECTION00610000000000000000
+   [2]: Table_Configuration.html#SECTION00620000000000000000
+   [3]: Table_Configuration.html#SECTION00630000000000000000
+   [4]: Table_Configuration.html#SECTION00640000000000000000
+   [5]: Table_Configuration.html#SECTION00650000000000000000
+   [6]: Table_Configuration.html#SECTION00660000000000000000
+   [7]: Table_Configuration.html#SECTION00670000000000000000
+   [8]: Table_Configuration.html#SECTION00680000000000000000
+   [9]: Table_Configuration.html#SECTION00690000000000000000
+   [10]: Table_Configuration.html#SECTION006100000000000000000
+

Added: incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Table_Design.mdtext
URL: http://svn.apache.org/viewvc/incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Table_Design.mdtext?rev=1183037&view=auto
==============================================================================
--- incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Table_Design.mdtext (added)
+++ incubator/accumulo/site/trunk/content/accumulo/user_manual_1.4-incubating/Table_Design.mdtext Thu Oct 13 19:13:58 2011
@@ -0,0 +1,183 @@
+Title: Accumulo User Manual: Table Design
+<a id=CHILD_LINKS></a>**Subsections**
+
+* [Basic Table][1]
+* [RowID Design][2]
+* [Indexing][3]
+* [Entity-Attribute and Graph Tables][4]
+* [Document-Partitioned Indexing][5]
+
+* * *
+
+## <a id=SECTION00700000000000000000></a> Table Design
+
+## <a id=SECTION00710000000000000000></a> Basic Table
+
+Since Accumulo tables are sorted by row ID, each table can be thought of as being indexed by the row ID. Lookups performed row ID can be executed quickly, by doing a binary search, first across the tablets, and then within a tablet. Clients should choose a row ID carefully in order to support their desired application. A simple rule is to select a unique identifier as the row ID for each entity to be stored and assign all the other attributes to be tracked to be columns under this row ID. For example, if we have the following data in a comma-separated file: 
+    
+    
+        userid,age,address,account-balance
+    
+
+We might choose to store this data using the userid as the rowID and the rest of the data in column families: 
+    
+    
+    Mutation m = new Mutation(new Text(userid));
+    m.put(new Text("age"), age);
+    m.put(new Text("address"), address);
+    m.put(new Text("balance"), account_balance);
+    
+    writer.add(m);
+    
+
+We could then retrieve any of the columns for a specific userid by specifying the userid as the range of a scanner and fetching specific columns: 
+    
+    
+    Range r = new Range(userid, userid); // single row
+    Scanner s = conn.createScanner("userdata", auths);
+    s.setRange(r);
+    s.fetchColumnFamily(new Text("age"));
+    
+    for(Entry<Key,Value> entry : s)
+        System.out.println(entry.getValue().toString());
+    
+
+## <a id=SECTION00720000000000000000></a> RowID Design
+
+Often it is necessary to transform the rowID in order to have rows ordered in a way that is optimal for anticipated access patterns. A good example of this is reversing the order of components of internet domain names in order to group rows of the same parent domain together: 
+    
+    
+    com.google.code
+    com.google.labs
+    com.google.mail
+    com.yahoo.mail
+    com.yahoo.research
+    
+
+Some data may result in the creation of very large rows - rows with many columns. In this case the table designer may wish to split up these rows for better load balancing while keeping them sorted together for scanning purposes. This can be done by appending a random substring at the end of the row: 
+    
+    
+    com.google.code_00
+    com.google.code_01
+    com.google.code_02
+    com.google.labs_00
+    com.google.mail_00
+    com.google.mail_01
+    
+
+It could also be done by adding a string representation of some period of time such as date to the week or month: 
+    
+    
+    com.google.code_201003
+    com.google.code_201004
+    com.google.code_201005
+    com.google.labs_201003
+    com.google.mail_201003
+    com.google.mail_201004
+    
+
+Appending dates provides the additional capability of restricting a scan to a given date range. 
+
+## <a id=SECTION00730000000000000000></a> Indexing
+
+In order to support lookups via more than one attribute of an entity, additional indexes can be built. However, because Accumulo tables can support any number of columns without specifying them beforehand, a single additional index will often suffice for supporting lookups of records in the main table. Here, the index has, as the rowID, the Value or Term from the main table, the column families are the same, and the column qualifier of the index table contains the rowID from the main table. 
+
+![converted table][6]
+
+Note: We store rowIDs in the column qualifier rather than the Value so that we can have more than one rowID associated with a particular term within the index. If we stored this in the Value we would only see one of the rows in which the value appears since Accumulo is configured by default to return the one most recent value associated with a key. 
+
+Lookups can then be done by scanning the Index Table first for occurrences of the desired values in the columns specified, which returns a list of row ID from the main table. These can then be used to retrieve each matching record, in their entirety, or a subset of their columns, from the Main Table. 
+
+To support efficient lookups of multiple rowIDs from the same table, the Accumulo client library provides a BatchScanner. Users specify a set of Ranges to the BatchScanner, which performs the lookups in multiple threads to multiple servers and returns an Iterator over all the rows retrieved. The rows returned are NOT in sorted order, as is the case with the basic Scanner interface. 
+    
+    
+    // first we scan the index for IDs of rows matching our query
+    
+    Text term = new Text("mySearchTerm");
+    
+    HashSet<Text> matchingRows = new HashSet<Text>();
+    
+    Scanner indexScanner = createScanner("index", auths);
+    indexScanner.setRange(new Range(term, term));
+    
+    // we retrieve the matching rowIDs and create a set of ranges
+    for(Entry<Key,Value> entry : indexScanner)
+        matchingRows.add(new Text(entry.getValue()));
+    
+    // now we pass the set of rowIDs to the batch scanner to retrieve them
+    BatchScanner bscan = conn.createBatchScanner("table", auths, 10);
+    
+    bscan.setRanges(matchingRows);
+    bscan.fetchFamily("attributes");
+    
+    for(Entry<Key,Value> entry : scan)
+        System.out.println(e.getValue());
+    
+
+One advantage of the dynamic schema capabilities of Accumulo is that different fields may be indexed into the same physical table. However, it may be necessary to create different index tables if the terms must be formatted differently in order to maintain proper sort order. For example, real numbers must be formatted differently than their usual notation in order to be sorted correctly. In these cases, usually one index per unique data type will suffice. 
+
+## <a id=SECTION00740000000000000000></a> Entity-Attribute and Graph Tables
+
+Accumulo is ideal for storing entities and their attributes, especially of the attributes are sparse. It is often useful to join several datasets together on common entities within the same table. This can allow for the representation of graphs, including nodes, their attributes, and connections to other nodes. 
+
+Rather than storing individual events, Entity-Attribute or Graph tables store aggregate information about the entities involved in the events and the relationships between entities. This is often preferrable when single events aren't very useful and when a continuously updated summarization is desired. 
+
+The physical schema for an entity-attribute or graph table is as follows: 
+
+![converted table][7]
+
+For example, to keep track of employees, managers and products the following entity-attribute table could be used. Note that the weights are not always necessary and are set to 0 when not used. 
+
+![converted table][8]   
+  
+
+
+To allow efficient updating of edge weights, an aggregating iterator can be configured to add the value of all mutations applied with the same key. These types of tables can easily be created from raw events by simply extracting the entities, attributes, and relationships from individual events and inserting the keys into Accumulo each with a count of 1. The aggregating iterator will take care of maintaining the edge weights. 
+
+## <a id=SECTION00750000000000000000></a> Document-Partitioned Indexing
+
+Using a simple index as described above works well when looking for records that match one of a set of given criteria. When looking for records that match more than one criterion simultaneously, such as when looking for documents that contain all of the words `the' and `white' and `house', there are several issues. 
+
+First is that the set of all records matching any one of the search terms must be sent to the client, which incurs a lot of network traffic. The second problem is that the client is responsible for performing set intersection on the sets of records returned to eliminate all but the records matching all search terms. The memory of the client may easily be overwhelmed during this operation. 
+
+For these reasons Accumulo includes support for a scheme known as sharded indexing, in which these set operations can be performed at the TabletServers and decisions about which records to include in the result set can be made without incurring network traffic. 
+
+This is accomplished via partitioning records into bins that each reside on at most one TabletServer, and then creating an index of terms per record within each bin as follows: 
+
+![converted table][9]
+
+Documents or records are mapped into bins by a user-defined ingest application. By storing the BinID as the RowID we ensure that all the information for a particular bin is contained in a single tablet and hosted on a single TabletServer since Accumulo never splits rows across tablets. Storing the Terms as column families serves to enable fast lookups of all the documents within this bin that contain the given term. 
+
+Finally, we perform set intersection operations on the TabletServer via a special iterator called the Intersecting Iterator. Since documents are partitioned into many bins, a search of all documents must search every bin. We can use the BatchScanner to scan all bins in parallel. The Intersecting Iterator should be enabled on a BatchScanner within user query code as follows: 
+    
+    
+    Text[] terms = {new Text("the"), new Text("white"), new Text("house")};
+    
+    BatchScanner bs = conn.createBatchScanner(table, auths, 20);
+    IteratorSetting iter = new IteratorSetting(20, "ii", IntersectingIterator.class);
+    IntersectingIterator.setColumnFamilies(iter, terms);
+    bs.addScanIterator(iter);
+    bs.setRanges(Collections.singleton(new Range()));
+    
+    for(Entry<Key,Value> entry : bs) {
+        System.out.println(" " + entry.getKey().getColumnQualifier());
+    }
+    
+
+This code effectively has the BatchScanner scan all tablets of a table, looking for documents that match all the given terms. Because all tablets are being scanned for every query, each query is more expensive than other Accumulo scans, which typically involve a small number of TabletServers. This reduces the number of concurrent queries supported and is subject to what is known as the `straggler' problem in which every query runs as slow as the slowest server participating. 
+
+Of course, fast servers will return their results to the client which can display them to the user immediately while they wait for the rest of the results to arrive. If the results are unordered this is quite effective as the first results to arrive are as good as any others to the user.   
+
+
+* * *
+
+   [1]: Table_Design.html#SECTION00710000000000000000
+   [2]: Table_Design.html#SECTION00720000000000000000
+   [3]: Table_Design.html#SECTION00730000000000000000
+   [4]: Table_Design.html#SECTION00740000000000000000
+   [5]: Table_Design.html#SECTION00750000000000000000
+   [6]: img2.png
+   [7]: img3.png
+   [8]: img4.png
+   [9]: img5.png
+