You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2012/03/22 20:32:11 UTC

svn commit: r1304000 - in /incubator/accumulo/trunk: ./ assemble/ conf/ conf/examples/3GB/native-standalone/ conf/examples/3GB/standalone/ core/src/main/java/org/apache/accumulo/core/data/ examples/wikisearch/ server/src/main/java/org/apache/accumulo/s...

Author: ecn
Date: Thu Mar 22 19:32:10 2012
New Revision: 1304000

URL: http://svn.apache.org/viewvc?rev=1304000&view=rev
Log:
ACCUMULO-485 ACCUMULO-486 ACCUMULO-484 ACCUMULO-411: merge to trunk

Modified:
    incubator/accumulo/trunk/   (props changed)
    incubator/accumulo/trunk/.gitignore
    incubator/accumulo/trunk/assemble/build.sh
    incubator/accumulo/trunk/conf/examples/3GB/native-standalone/accumulo-site.xml
    incubator/accumulo/trunk/conf/examples/3GB/standalone/accumulo-site.xml
    incubator/accumulo/trunk/conf/generic_logger.xml
    incubator/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/data/Key.java
    incubator/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/data/Range.java
    incubator/accumulo/trunk/examples/wikisearch/README
    incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
    incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
    incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/log/RemoteLogger.java

Propchange: incubator/accumulo/trunk/
------------------------------------------------------------------------------
  Merged /incubator/accumulo/branches/1.4:r1303841,1303845,1303847,1303928,1303976,1303995
  Merged /incubator/accumulo/branches/1.4/src:r1303841,1303845,1303847,1303928,1303976,1303995

Modified: incubator/accumulo/trunk/.gitignore
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/.gitignore?rev=1304000&r1=1303999&r2=1304000&view=diff
==============================================================================
--- incubator/accumulo/trunk/.gitignore (original)
+++ incubator/accumulo/trunk/.gitignore Thu Mar 22 19:32:10 2012
@@ -1,3 +1,17 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
 
 # /
 /logs

Modified: incubator/accumulo/trunk/assemble/build.sh
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/assemble/build.sh?rev=1304000&r1=1303999&r2=1304000&view=diff
==============================================================================
--- incubator/accumulo/trunk/assemble/build.sh (original)
+++ incubator/accumulo/trunk/assemble/build.sh Thu Mar 22 19:32:10 2012
@@ -41,7 +41,7 @@ runAt() {
 run mvn -U clean 
 mvn rat:check 
 COUNT=`grep '!????' target/rat.txt | wc -l`
-EXPECTED=56
+EXPECTED=55
 if [ "$COUNT" -ne $EXPECTED ]
 then
    fail expected $EXPECTED files missing licenses, but saw "$COUNT"

Modified: incubator/accumulo/trunk/conf/examples/3GB/native-standalone/accumulo-site.xml
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/conf/examples/3GB/native-standalone/accumulo-site.xml?rev=1304000&r1=1303999&r2=1304000&view=diff
==============================================================================
--- incubator/accumulo/trunk/conf/examples/3GB/native-standalone/accumulo-site.xml (original)
+++ incubator/accumulo/trunk/conf/examples/3GB/native-standalone/accumulo-site.xml Thu Mar 22 19:32:10 2012
@@ -61,7 +61,7 @@
     
     <property>
       <name>tserver.cache.index.size</name>
-      <value>512M</value>
+      <value>100M</value>
     </property>
     
     <property>

Modified: incubator/accumulo/trunk/conf/examples/3GB/standalone/accumulo-site.xml
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/conf/examples/3GB/standalone/accumulo-site.xml?rev=1304000&r1=1303999&r2=1304000&view=diff
==============================================================================
--- incubator/accumulo/trunk/conf/examples/3GB/standalone/accumulo-site.xml (original)
+++ incubator/accumulo/trunk/conf/examples/3GB/standalone/accumulo-site.xml Thu Mar 22 19:32:10 2012
@@ -61,7 +61,7 @@
     
     <property>
       <name>tserver.cache.index.size</name>
-      <value>512M</value>
+      <value>100M</value>
     </property>
     
     <property>

Modified: incubator/accumulo/trunk/conf/generic_logger.xml
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/conf/generic_logger.xml?rev=1304000&r1=1303999&r2=1304000&view=diff
==============================================================================
--- incubator/accumulo/trunk/conf/generic_logger.xml (original)
+++ incubator/accumulo/trunk/conf/generic_logger.xml Thu Mar 22 19:32:10 2012
@@ -69,6 +69,10 @@
      <level value="INFO"/>
   </logger>
 
+  <logger name="org.apache.accumulo.examples.wikisearch">
+     <level value="INFO"/>
+  </logger>
+
   <logger name="org.mortbay.log">
      <level value="WARN"/>
   </logger>

Modified: incubator/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/data/Key.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/data/Key.java?rev=1304000&r1=1303999&r2=1304000&view=diff
==============================================================================
--- incubator/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/data/Key.java (original)
+++ incubator/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/data/Key.java Thu Mar 22 19:32:10 2012
@@ -20,7 +20,7 @@ package org.apache.accumulo.core.data;
  * This is the Key used to store and access individual values in Accumulo.  A Key is a tuple composed of a row, column family, column qualifier, 
  * column visibility, timestamp, and delete marker.
  * 
- * Keys are comparable and therefore have a sorted order.  
+ * Keys are comparable and therefore have a sorted order defined by {@link #compareTo(Key)}.
  * 
  */
 
@@ -82,6 +82,10 @@ public class Key implements WritableComp
     deleted = del;
   }
   
+  /**
+   * Creates a key with empty row, empty column family, empty column qualifier, empty column visibility, timestamp {@link Long#MAX_VALUE}, and delete marker
+   * false.
+   */
   public Key() {
     row = EMPTY_BYTES;
     colFamily = EMPTY_BYTES;
@@ -91,10 +95,18 @@ public class Key implements WritableComp
     deleted = false;
   }
   
+  /**
+   * Creates a key with the specified row, empty column family, empty column qualifier, empty column visibility, timestamp {@link Long#MAX_VALUE}, and delete
+   * marker false.
+   */
   public Key(Text row) {
     init(row.getBytes(), 0, row.getLength(), EMPTY_BYTES, 0, 0, EMPTY_BYTES, 0, 0, EMPTY_BYTES, 0, 0, Long.MAX_VALUE, false, true);
   }
   
+  /**
+   * Creates a key with the specified row, empty column family, empty column qualifier, empty column visibility, the specified timestamp, and delete marker
+   * false.
+   */
   public Key(Text row, long ts) {
     this(row);
     timestamp = ts;
@@ -116,57 +128,102 @@ public class Key implements WritableComp
     init(row, 0, row.length, cf, 0, cf.length, cq, 0, cq.length, cv, 0, cv.length, ts, deleted, copy);
   }
   
+  /**
+   * Creates a key with the specified row, the specified column family, empty column qualifier, empty column visibility, timestamp {@link Long#MAX_VALUE}, and
+   * delete marker false.
+   */
   public Key(Text row, Text cf) {
     init(row.getBytes(), 0, row.getLength(), cf.getBytes(), 0, cf.getLength(), EMPTY_BYTES, 0, 0, EMPTY_BYTES, 0, 0, Long.MAX_VALUE, false, true);
   }
   
+  /**
+   * Creates a key with the specified row, the specified column family, the specified column qualifier, empty column visibility, timestamp
+   * {@link Long#MAX_VALUE}, and delete marker false.
+   */
   public Key(Text row, Text cf, Text cq) {
     init(row.getBytes(), 0, row.getLength(), cf.getBytes(), 0, cf.getLength(), cq.getBytes(), 0, cq.getLength(), EMPTY_BYTES, 0, 0, Long.MAX_VALUE, false, true);
   }
   
+  /**
+   * Creates a key with the specified row, the specified column family, the specified column qualifier, the specified column visibility, timestamp
+   * {@link Long#MAX_VALUE}, and delete marker false.
+   */
   public Key(Text row, Text cf, Text cq, Text cv) {
     init(row.getBytes(), 0, row.getLength(), cf.getBytes(), 0, cf.getLength(), cq.getBytes(), 0, cq.getLength(), cv.getBytes(), 0, cv.getLength(),
         Long.MAX_VALUE, false, true);
   }
   
+  /**
+   * Creates a key with the specified row, the specified column family, the specified column qualifier, empty column visibility, the specified timestamp, and
+   * delete marker false.
+   */
   public Key(Text row, Text cf, Text cq, long ts) {
     init(row.getBytes(), 0, row.getLength(), cf.getBytes(), 0, cf.getLength(), cq.getBytes(), 0, cq.getLength(), EMPTY_BYTES, 0, 0, ts, false, true);
   }
   
+  /**
+   * Creates a key with the specified row, the specified column family, the specified column qualifier, the specified column visibility, the specified
+   * timestamp, and delete marker false.
+   */
   public Key(Text row, Text cf, Text cq, Text cv, long ts) {
     init(row.getBytes(), 0, row.getLength(), cf.getBytes(), 0, cf.getLength(), cq.getBytes(), 0, cq.getLength(), cv.getBytes(), 0, cv.getLength(), ts, false,
         true);
   }
   
+  /**
+   * Creates a key with the specified row, the specified column family, the specified column qualifier, the specified column visibility, the specified
+   * timestamp, and delete marker false.
+   */
   public Key(Text row, Text cf, Text cq, ColumnVisibility cv, long ts) {
     byte[] expr = cv.getExpression();
     init(row.getBytes(), 0, row.getLength(), cf.getBytes(), 0, cf.getLength(), cq.getBytes(), 0, cq.getLength(), expr, 0, expr.length, ts, false, true);
   }
   
+  /**
+   * Converts CharSequence to Text and creates a Key using {@link #Key(Text)}.
+   */
   public Key(CharSequence row) {
     this(new Text(row.toString()));
   }
   
+  /**
+   * Converts CharSequence to Text and creates a Key using {@link #Key(Text,Text)}.
+   */
   public Key(CharSequence row, CharSequence cf) {
     this(new Text(row.toString()), new Text(cf.toString()));
   }
   
+  /**
+   * Converts CharSequence to Text and creates a Key using {@link #Key(Text,Text,Text)}.
+   */
   public Key(CharSequence row, CharSequence cf, CharSequence cq) {
     this(new Text(row.toString()), new Text(cf.toString()), new Text(cq.toString()));
   }
   
+  /**
+   * Converts CharSequence to Text and creates a Key using {@link #Key(Text,Text,Text,Text)}.
+   */
   public Key(CharSequence row, CharSequence cf, CharSequence cq, CharSequence cv) {
     this(new Text(row.toString()), new Text(cf.toString()), new Text(cq.toString()), new Text(cv.toString()));
   }
   
+  /**
+   * Converts CharSequence to Text and creates a Key using {@link #Key(Text,Text,Text,long)}.
+   */
   public Key(CharSequence row, CharSequence cf, CharSequence cq, long ts) {
     this(new Text(row.toString()), new Text(cf.toString()), new Text(cq.toString()), ts);
   }
   
+  /**
+   * Converts CharSequence to Text and creates a Key using {@link #Key(Text,Text,Text,Text,long)}.
+   */
   public Key(CharSequence row, CharSequence cf, CharSequence cq, CharSequence cv, long ts) {
     this(new Text(row.toString()), new Text(cf.toString()), new Text(cq.toString()), new Text(cv.toString()), ts);
   }
   
+  /**
+   * Converts CharSequence to Text and creates a Key using {@link #Key(Text,Text,Text,ColumnVisibility,long)}.
+   */
   public Key(CharSequence row, CharSequence cf, CharSequence cq, ColumnVisibility cv, long ts) {
     this(new Text(row.toString()), new Text(cf.toString()), new Text(cq.toString()), new Text(cv.getExpression()), ts);
   }
@@ -220,6 +277,9 @@ public class Key implements WritableComp
     return returnKey;
   }
   
+  /**
+   * Creates a key with the same row, column family, column qualifier, column visibility, timestamp, and delete marker as the given key.
+   */
   public Key(Key other) {
     set(other);
   }
@@ -240,7 +300,7 @@ public class Key implements WritableComp
    *          the key's row will be copied into this Text
    * @return the Text that was passed in
    */
-
+  
   public Text getRow(Text r) {
     r.set(row, 0, row.length);
     return r;
@@ -251,7 +311,7 @@ public class Key implements WritableComp
    * 
    * @return ByteSequence that points to the internal key row data.
    */
-
+  
   public ByteSequence getRowData() {
     return new ArrayByteSequence(row);
   }
@@ -261,7 +321,7 @@ public class Key implements WritableComp
    * 
    * @return Text containing the row field
    */
-
+  
   public Text getRow() {
     return getRow(new Text());
   }
@@ -271,9 +331,9 @@ public class Key implements WritableComp
    * 
    * @param r
    *          row to compare to keys row
-   * @return same as getRow().compareTo(r)
+   * @return same as {@link getRow()}.compareTo(r)
    */
-
+  
   public int compareRow(Text r) {
     return WritableComparator.compareBytes(row, 0, row.length, r.getBytes(), 0, r.getLength());
   }
@@ -283,7 +343,7 @@ public class Key implements WritableComp
    * 
    * @return ByteSequence that points to the internal key column family data.
    */
-
+  
   public ByteSequence getColumnFamilyData() {
     return new ArrayByteSequence(colFamily);
   }
@@ -295,7 +355,7 @@ public class Key implements WritableComp
    *          the key's column family will be copied into this Text
    * @return the Text that was passed in
    */
-
+  
   public Text getColumnFamily(Text cf) {
     cf.set(colFamily, 0, colFamily.length);
     return cf;
@@ -306,7 +366,7 @@ public class Key implements WritableComp
    * 
    * @return Text containing the column family field
    */
-
+  
   public Text getColumnFamily() {
     return getColumnFamily(new Text());
   }
@@ -316,9 +376,9 @@ public class Key implements WritableComp
    * 
    * @param cf
    *          column family to compare to keys column family
-   * @return same as getColumnFamily().compareTo(cf)
+   * @return same as {@link #getColumnFamily()}.compareTo(cf)
    */
-
+  
   public int compareColumnFamily(Text cf) {
     return WritableComparator.compareBytes(colFamily, 0, colFamily.length, cf.getBytes(), 0, cf.getLength());
   }
@@ -328,7 +388,7 @@ public class Key implements WritableComp
    * 
    * @return ByteSequence that points to the internal key column qualifier data.
    */
-
+  
   public ByteSequence getColumnQualifierData() {
     return new ArrayByteSequence(colQualifier);
   }
@@ -340,7 +400,7 @@ public class Key implements WritableComp
    *          the key's column qualifier will be copied into this Text
    * @return the Text that was passed in
    */
-
+  
   public Text getColumnQualifier(Text cq) {
     cq.set(colQualifier, 0, colQualifier.length);
     return cq;
@@ -351,7 +411,7 @@ public class Key implements WritableComp
    * 
    * @return Text containing the column qualifier field
    */
-
+  
   public Text getColumnQualifier() {
     return getColumnQualifier(new Text());
   }
@@ -361,9 +421,9 @@ public class Key implements WritableComp
    * 
    * @param cq
    *          column family to compare to keys column qualifier
-   * @return same as compareColumnQualifier().compareTo(cq)
+   * @return same as {@link #getColumnQualifier()}.compareTo(cq)
    */
-
+  
   public int compareColumnQualifier(Text cq) {
     return WritableComparator.compareBytes(colQualifier, 0, colQualifier.length, cq.getBytes(), 0, cq.getLength());
   }
@@ -389,7 +449,7 @@ public class Key implements WritableComp
    * 
    * @return ByteSequence that points to the internal key column visibility data.
    */
-
+  
   public ByteSequence getColumnVisibilityData() {
     return new ArrayByteSequence(colVisibility);
   }
@@ -399,7 +459,7 @@ public class Key implements WritableComp
    * 
    * @return Text containing the column visibility field
    */
-
+  
   public final Text getColumnVisibility() {
     return getColumnVisibility(new Text());
   }
@@ -411,12 +471,15 @@ public class Key implements WritableComp
    *          the key's column visibility will be copied into this Text
    * @return the Text that was passed in
    */
-
+  
   public final Text getColumnVisibility(Text cv) {
     cv.set(colVisibility, 0, colVisibility.length);
     return cv;
   }
   
+  /**
+   * Sets this key's row, column family, column qualifier, column visibility, timestamp, and delete marker to be the same as another key's.
+   */
   public void set(Key k) {
     row = k.row;
     colFamily = k.colFamily;
@@ -476,7 +539,7 @@ public class Key implements WritableComp
    * Compare part of a key. For example compare just the row and column family, and if those are equal then return true.
    * 
    */
-
+  
   public boolean equals(Key other, PartialKey part) {
     switch (part) {
       case ROW:
@@ -500,10 +563,12 @@ public class Key implements WritableComp
   }
   
   /**
-   * Compare part of a key. For example compare just the row and column family, and if those are equal then return 0.
+   * Compare elements of a key given by a {@link PartialKey}. For example, for {@link PartialKey#ROW_COLFAM}, compare just the row and column family. If the
+   * rows are not equal, return the result of the row comparison; otherwise, return the result of the column family comparison.
    * 
+   * @see #compareTo(Key)
    */
-
+  
   public int compareTo(Key other, PartialKey part) {
     // check for matching row
     int result = WritableComparator.compareBytes(row, 0, row.length, other.row, 0, other.row.length);
@@ -546,9 +611,10 @@ public class Key implements WritableComp
   }
   
   /**
-   * Compare the elements of a key starting with the row. If the row is equal, then compare the column family, etc. The row, column family, column qualifier,
-   * and column visibility are compared lexographically and sorted ascending. The timestamps are compared numerically and sorted descending so that the most
-   * recent data comes first. Last when delete is compared, true come first and false after.
+   * Compare all elements of a key. The elements (row, column family, column qualifier, column visibility, timestamp, and delete marker) are compared in order
+   * until an unequal element is found. If the row is equal, then compare the column family, etc. The row, column family, column qualifier, and column
+   * visibility are compared lexographically and sorted ascending. The timestamps are compared numerically and sorted descending so that the most recent data
+   * comes first. Lastly, a delete marker of true sorts before a delete marker of false.
    */
   
   public int compareTo(Key other) {
@@ -609,10 +675,18 @@ public class Key implements WritableComp
     return rowColumnStringBuilder().toString();
   }
   
+  /**
+   * Returns the sums of the lengths of the row, column family, column qualifier, and visibility.
+   * 
+   * @return row.length + colFamily.length + colQualifier.length + colVisibility.length;
+   */
   public int getLength() {
     return row.length + colFamily.length + colQualifier.length + colVisibility.length;
   }
   
+  /**
+   * Same as {@link #getLength()}.
+   */
   public int getSize() {
     return getLength();
   }
@@ -657,6 +731,7 @@ public class Key implements WritableComp
    * Use this to compress a list of keys before sending them via thrift.
    * 
    * @param param
+   *          a list of key/value pairs
    */
   public static List<TKeyValue> compress(List<? extends KeyValue> param) {
     

Modified: incubator/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/data/Range.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/data/Range.java?rev=1304000&r1=1303999&r2=1304000&view=diff
==============================================================================
--- incubator/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/data/Range.java (original)
+++ incubator/accumulo/trunk/core/src/main/java/org/apache/accumulo/core/data/Range.java Thu Mar 22 19:32:10 2012
@@ -107,7 +107,7 @@ public class Range implements WritableCo
   }
   
   /**
-   * Creates a range from startRow inclusive to endRow inclusive
+   * Creates a range from startRow to endRow
    * 
    * @param startRow
    *          set this to null when negative infinity is needed
@@ -125,7 +125,7 @@ public class Range implements WritableCo
   }
   
   /**
-   * Creates a range from startRow inclusive to endRow inclusive
+   * Creates a range from startRow to endRow
    * 
    * @param startRow
    *          set this to null when negative infinity is needed
@@ -142,6 +142,8 @@ public class Range implements WritableCo
   }
   
   /**
+   * Creates a range from startKey to endKey
+   * 
    * @param startKey
    *          set this to null when negative infinity is needed
    * @param startKeyInclusive
@@ -165,7 +167,7 @@ public class Range implements WritableCo
   }
   
   /**
-   * Copy constructor
+   * Copies a range
    */
   public Range(Range range) {
     this(range.start, range.stop, range.startKeyInclusive, range.stopKeyInclusive, range.infiniteStartKey, range.infiniteStopKey);
@@ -191,6 +193,9 @@ public class Range implements WritableCo
         trange.stopKeyInclusive, trange.infiniteStartKey, trange.infiniteStopKey);
   }
   
+  /**
+   * @return the first key in the range, null if the key is infinite
+   */
   public Key getStartKey() {
     if (infiniteStartKey) {
       return null;
@@ -201,7 +206,7 @@ public class Range implements WritableCo
   /**
    * 
    * @param key
-   * @return true of the given key is before the range, otherwise false
+   * @return true if the given key is before the range, otherwise false
    */
   public boolean beforeStartKey(Key key) {
     if (infiniteStartKey) {
@@ -216,7 +221,7 @@ public class Range implements WritableCo
   /**
    * @return the last key in the range, null if the end key is infinite
    */
-
+  
   public Key getEndKey() {
     if (infiniteStopKey) {
       return null;
@@ -228,7 +233,7 @@ public class Range implements WritableCo
    * @param key
    * @return true if the given key is after the range, otherwise false
    */
-
+  
   public boolean afterEndKey(Key key) {
     if (infiniteStopKey)
       return false;
@@ -258,6 +263,10 @@ public class Range implements WritableCo
     return compareTo(otherRange) == 0;
   }
   
+  /**
+   * Compares this range to another range. Compares in the order start key, inclusiveness of start key, end key, inclusiveness of end key. Infinite keys sort
+   * first, and non-infinite keys are compared with {@link Key#compareTo(Key)}. Inclusive sorts before non-inclusive.
+   */
   public int compareTo(Range o) {
     int comp;
     
@@ -323,7 +332,7 @@ public class Range implements WritableCo
    * @param ranges
    * @return list of merged ranges
    */
-
+  
   public static List<Range> mergeOverlapping(Collection<Range> ranges) {
     if (ranges.size() == 0)
       return Collections.emptyList();
@@ -395,7 +404,7 @@ public class Range implements WritableCo
    * @throws IllegalArgumentException
    *           if ranges does not overlap
    */
-
+  
   public Range clip(Range range) {
     return clip(range, false);
   }
@@ -409,7 +418,7 @@ public class Range implements WritableCo
    *          If the ranges do not overlap and true is passed, then null is returned otherwise an exception is thrown.
    * @return the intersection
    */
-
+  
   public Range clip(Range range, boolean returnNullIfDisjoint) {
     
     Key sk = range.getStartKey();
@@ -462,7 +471,7 @@ public class Range implements WritableCo
    * @throws IllegalArgumentException
    *           if min > max
    */
-
+  
   public Range bound(Column min, Column max) {
     
     if (min.compareTo(max) > 0) {

Modified: incubator/accumulo/trunk/examples/wikisearch/README
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/examples/wikisearch/README?rev=1304000&r1=1303999&r2=1304000&view=diff
==============================================================================
--- incubator/accumulo/trunk/examples/wikisearch/README (original)
+++ incubator/accumulo/trunk/examples/wikisearch/README Thu Mar 22 19:32:10 2012
@@ -66,11 +66,3 @@
 	There are two parameters to the REST service, query and auths. The query parameter is the same string that you would type
 	into the search box at ui.jsp, and the auths parameter is a comma-separated list of wikis that you want to search (i.e.
 	enwiki,frwiki,dewiki, etc. Or you can use all) 
-	
-	10. Optional. Add the following line to the $ACCUMULO_HOME/conf/log4j.properties file to turn off debug messages in the specialized 
-	iterators, which will dramatically increase performance:
-	
-	log4j.logger.org.apache.accumulo.examples.wikisearch.iterator=INFO,A1
-	
-	This needs to be propagated to all the tablet server nodes, and accumulo needs to be restarted.
-	

Modified: incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java?rev=1304000&r1=1303999&r2=1304000&view=diff
==============================================================================
--- incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java (original)
+++ incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java Thu Mar 22 19:32:10 2012
@@ -2300,8 +2300,30 @@ public class Tablet {
     return initiateMinorCompaction(flushId);
   }
   
+  boolean minorCompactNow() {
+    long flushId;
+    try {
+      flushId = getFlushID();
+    } catch (NoNodeException e) {
+      log.info("Asked to initiate MinC when there was no flush id " + getExtent() + " " + e.getMessage());
+      return false;
+    }
+    MinorCompactionTask mct = createMinorCompactionTask(flushId);
+    if (mct == null)
+      return false;
+    mct.run();
+    return true;
+  }
+
   boolean initiateMinorCompaction(long flushId) {
-    
+    MinorCompactionTask mct = createMinorCompactionTask(flushId);
+    if (mct == null)
+      return false;
+    tabletResources.executeMinorCompaction(mct);
+    return true;
+  }
+  
+  private MinorCompactionTask createMinorCompactionTask(long flushId) {
     MinorCompactionTask mct;
     long t1, t2;
     
@@ -2326,14 +2348,14 @@ public class Tablet {
             logMessage.append(" tabletMemory.getMemTable().getNumEntries() " + tabletMemory.getMemTable().getNumEntries());
           logMessage.append(" updatingFlushID " + updatingFlushID);
           
-          return false;
+          return null;
         }
         // We're still recovering log entries
         if (datafileManager == null) {
           logMessage = new StringBuilder();
           logMessage.append(extent.toString());
           logMessage.append(" datafileManager " + datafileManager);
-          return false;
+          return null;
         }
         
         mct = prepareForMinC(flushId);
@@ -2345,11 +2367,8 @@ public class Tablet {
         log.debug(logMessage);
     }
     
-    tabletResources.executeMinorCompaction(mct);
-    
     log.debug(String.format("MinC initiate lock %.2f secs", (t2 - t1) / 1000.0));
-    
-    return true;
+    return mct;
   }
   
   long getFlushID() throws NoNodeException {

Modified: incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java?rev=1304000&r1=1303999&r2=1304000&view=diff
==============================================================================
--- incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java (original)
+++ incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java Thu Mar 22 19:32:10 2012
@@ -2409,21 +2409,18 @@ public class TabletServer extends Abstra
           // this opens the tablet file and fills in the endKey in the
           // extent
           tablet = new Tablet(TabletServer.this, locationToOpen, extentToOpen, trm, tabletsKeyValues);
-          if (tablet.initiateMinorCompaction()) {
-            /*
-             * If a minor compaction starts after a tablet opens, this indicates a log recovery occurred. This recovered data must be minor compacted.
-             * 
-             * There are three reasons to wait for this minor compaction to finish before placing the tablet in online tablets.
-             * 
-             * 1) The log recovery code does not handle data written to the tablet on multiple tablet servers. 2) The log recovery code does not block if memory
-             * is full. Therefore recovering lots of tablets that use a lot of memory could run out of memory. 3) The minor compaction finish event did not make
-             * it to the logs (the file will be in !METADATA, preventing replay of compacted data)... but do not want a majc to wipe the file out from !METADATA
-             * and then have another process failure... this could cause duplicate data to replay
-             */
-
-            tablet.waitForMinC();
-          } else if (tablet.getNumEntriesInMemory() > 0) {
-            log.warn("Minor compaction after recovery fails for " + extentToOpen);
+          /*
+           * If a minor compaction starts after a tablet opens, this indicates a log recovery occurred. This recovered data must be minor compacted.
+           * 
+           * There are three reasons to wait for this minor compaction to finish before placing the tablet in online tablets.
+           * 
+           * 1) The log recovery code does not handle data written to the tablet on multiple tablet servers. 2) The log recovery code does not block if memory
+           * is full. Therefore recovering lots of tablets that use a lot of memory could run out of memory. 3) The minor compaction finish event did not make
+           * it to the logs (the file will be in !METADATA, preventing replay of compacted data)... but do not want a majc to wipe the file out from !METADATA
+           * and then have another process failure... this could cause duplicate data to replay
+           */
+          if (tablet.getNumEntriesInMemory() > 0 && !tablet.minorCompactNow()) {
+            throw new RuntimeException("Minor compaction after recovery fails for " + extentToOpen);
           }
           
           Assignment assignment = new Assignment(extentToOpen, getTabletSession());

Modified: incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/log/RemoteLogger.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/log/RemoteLogger.java?rev=1304000&r1=1303999&r2=1304000&view=diff
==============================================================================
--- incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/log/RemoteLogger.java (original)
+++ incubator/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/log/RemoteLogger.java Thu Mar 22 19:32:10 2012
@@ -39,7 +39,6 @@ import org.apache.accumulo.server.conf.S
 import org.apache.accumulo.server.security.SecurityConstants;
 import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
-import org.apache.thrift.TServiceClient;
 import org.apache.thrift.transport.TTransportException;
 
 /**
@@ -239,10 +238,12 @@ public class RemoteLogger {
     }
 
     try {
-      client.close(null, logFile.id);
+      if (client != null)
+        client.close(null, logFile.id);
     } finally {
-      TServiceClient c = (TServiceClient) client;
-      c.getInputProtocol().getTransport().close();
+      MutationLogger.Iface tmp = client;
+      client = null;
+      ThriftUtil.returnClient(tmp);
     }
   }