You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2011/08/31 13:22:06 UTC

svn commit: r1163576 - in /lucene/dev/trunk/lucene/src: java/org/apache/lucene/index/ java/org/apache/lucene/index/codecs/ java/org/apache/lucene/index/values/ java/org/apache/lucene/util/ test/org/apache/lucene/index/values/

Author: simonw
Date: Wed Aug 31 11:22:05 2011
New Revision: 1163576

URL: http://svn.apache.org/viewvc?rev=1163576&view=rev
Log:
LUCENE-3408: Remove unnecessary memory barriers in DWPT

Added:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Counter.java
Modified:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PerDocWriteState.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DocValuesConsumer.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Bytes.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Floats.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/IndexDocValuesArray.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Ints.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/IntsImpl.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Writer.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/values/TestDocValues.java

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Wed Aug 31 11:22:05 2011
@@ -23,7 +23,6 @@ import static org.apache.lucene.util.Byt
 import java.io.IOException;
 import java.io.PrintStream;
 import java.text.NumberFormat;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
@@ -32,6 +31,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.FlushInfo;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.BitVector;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.ByteBlockPool.Allocator;
 import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -155,7 +155,7 @@ public class DocumentsWriterPerThread {
   final Directory directory;
   final DocState docState;
   final DocConsumer consumer;
-  final AtomicLong bytesUsed;
+  final Counter bytesUsed;
   
   SegmentWriteState flushState;
   //Deletes for our still-in-RAM (to be flushed next) segment
@@ -184,7 +184,7 @@ public class DocumentsWriterPerThread {
     this.docState = new DocState(this);
     this.docState.similarityProvider = parent.indexWriter.getConfig()
         .getSimilarityProvider();
-    bytesUsed = new AtomicLong(0);
+    bytesUsed = Counter.newCounter();
     byteBlockAllocator = new DirectTrackingAllocator(bytesUsed);
     consumer = indexingChain.getChain(this);
     pendingDeletes = new BufferedDeletes();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PerDocWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PerDocWriteState.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PerDocWriteState.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PerDocWriteState.java Wed Aug 31 11:22:05 2011
@@ -16,11 +16,11 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 import java.io.PrintStream;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.codecs.PerDocConsumer;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Counter;
 
 /**
  * Encapsulates all necessary state to initiate a {@link PerDocConsumer} and
@@ -33,13 +33,13 @@ public class PerDocWriteState {
   public final Directory directory;
   public final String segmentName;
   public final FieldInfos fieldInfos;
-  public final AtomicLong bytesUsed;
+  public final Counter bytesUsed;
   public final SegmentCodecs segmentCodecs;
   public final int codecId;
   public final IOContext context;
 
   PerDocWriteState(PrintStream infoStream, Directory directory,
-      String segmentName, FieldInfos fieldInfos, AtomicLong bytesUsed,
+      String segmentName, FieldInfos fieldInfos, Counter bytesUsed,
       int codecId, IOContext context) {
     this.infoStream = infoStream;
     this.directory = directory;
@@ -58,7 +58,7 @@ public class PerDocWriteState {
     segmentName = state.segmentName;
     fieldInfos = state.fieldInfos;
     codecId = state.codecId;
-    bytesUsed = new AtomicLong(0);
+    bytesUsed = Counter.newCounter();
     context = state.context;
   }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java Wed Aug 31 11:22:05 2011
@@ -19,12 +19,12 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Comparator;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.BytesRefHash.BytesStartArray;
 import org.apache.lucene.util.BytesRefHash.MaxBytesLengthExceededException;
 
@@ -54,7 +54,7 @@ final class TermsHashPerField extends In
   final BytesRefHash bytesHash;
 
   ParallelPostingsArray postingsArray;
-  private final AtomicLong bytesUsed;
+  private final Counter bytesUsed;
 
   public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHash termsHash, final TermsHash nextTermsHash, final FieldInfo fieldInfo) {
     intPool = termsHash.intPool;
@@ -63,7 +63,7 @@ final class TermsHashPerField extends In
     docState = termsHash.docState;
     this.termsHash = termsHash;
     bytesUsed = termsHash.trackAllocations ? termsHash.docWriter.bytesUsed
-        : new AtomicLong();
+        : Counter.newCounter();
     fieldState = docInverterPerField.fieldState;
     this.consumer = termsHash.consumer.addField(this, fieldInfo);
     PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
@@ -283,10 +283,10 @@ final class TermsHashPerField extends In
   private static final class PostingsBytesStartArray extends BytesStartArray {
 
     private final TermsHashPerField perField;
-    private final AtomicLong bytesUsed;
+    private final Counter bytesUsed;
 
     private PostingsBytesStartArray(
-        TermsHashPerField perField, AtomicLong bytesUsed) {
+        TermsHashPerField perField, Counter bytesUsed) {
       this.perField = perField;
       this.bytesUsed = bytesUsed;
     }
@@ -320,7 +320,7 @@ final class TermsHashPerField extends In
     }
 
     @Override
-    public AtomicLong bytesUsed() {
+    public Counter bytesUsed() {
       return bytesUsed;
     }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java Wed Aug 31 11:22:05 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index.codecs;
 import java.io.IOException;
 import java.util.Comparator;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
@@ -32,6 +31,7 @@ import org.apache.lucene.store.CompoundF
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
 
 /**
  * 
@@ -41,7 +41,7 @@ public class DefaultDocValuesConsumer ex
   private final String segmentName;
   private final int codecId;
   private final Directory directory;
-  private final AtomicLong bytesUsed;
+  private final Counter bytesUsed;
   private final Comparator<BytesRef> comparator;
   private boolean useCompoundFile;
   private final IOContext context;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DocValuesConsumer.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DocValuesConsumer.java Wed Aug 31 11:22:05 2011
@@ -18,13 +18,13 @@ package org.apache.lucene.index.codecs;
  */
 import java.io.IOException;
 import java.util.Collection;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.values.IndexDocValues;
 import org.apache.lucene.index.values.PerDocFieldValues;
 import org.apache.lucene.index.values.Writer;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.Counter;
 
 /**
  * Abstract API that consumes {@link PerDocFieldValues}.
@@ -37,7 +37,7 @@ import org.apache.lucene.util.Bits;
  */
 public abstract class DocValuesConsumer {
 
-  protected final AtomicLong bytesUsed;
+  protected final Counter bytesUsed;
 
   /**
    * Creates a new {@link DocValuesConsumer}.
@@ -47,8 +47,8 @@ public abstract class DocValuesConsumer 
    *          internally allocated memory. All tracked bytes must be released
    *          once {@link #finish(int)} has been called.
    */
-  protected DocValuesConsumer(AtomicLong bytesUsed) {
-    this.bytesUsed = bytesUsed == null ? new AtomicLong(0) : bytesUsed;
+  protected DocValuesConsumer(Counter bytesUsed) {
+    this.bytesUsed = bytesUsed == null ? Counter.newCounter() : bytesUsed;
   }
 
   /**

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Bytes.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Bytes.java Wed Aug 31 11:22:05 2011
@@ -34,6 +34,7 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.PagedBytes;
 
@@ -107,7 +108,7 @@ public final class Bytes {
    *           if the files for the writer can not be created.
    */
   public static Writer getWriter(Directory dir, String id, Mode mode,
-      Comparator<BytesRef> comp, boolean fixedSize, AtomicLong bytesUsed, IOContext context)
+      Comparator<BytesRef> comp, boolean fixedSize, Counter bytesUsed, IOContext context)
       throws IOException {
     // TODO -- i shouldn't have to specify fixed? can
     // track itself & do the write thing at write time?
@@ -349,7 +350,7 @@ public final class Bytes {
     private final IOContext context;
 
     protected BytesWriterBase(Directory dir, String id, String codecName,
-        int version, AtomicLong bytesUsed, IOContext context) throws IOException {
+        int version, Counter bytesUsed, IOContext context) throws IOException {
       super(bytesUsed);
       this.id = id;
       this.dir = dir;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java Wed Aug 31 11:22:05 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index.values;
  */
 
 import java.io.IOException;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.values.Bytes.BytesBaseSource;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
@@ -32,6 +31,7 @@ import org.apache.lucene.util.AttributeS
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -55,14 +55,14 @@ class FixedDerefBytesImpl {
     private int size = -1;
     private int[] docToID;
     private final BytesRefHash hash;
-    public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
+    public Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
         throws IOException {
       this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
           bytesUsed, context);
     }
 
     public Writer(Directory dir, String id, Allocator allocator,
-        AtomicLong bytesUsed, IOContext context) throws IOException {
+        Counter bytesUsed, IOContext context) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
       hash = new BytesRefHash(new ByteBlockPool(allocator),
           BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java Wed Aug 31 11:22:05 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.index.values;
 
 import java.io.IOException;
 import java.util.Comparator;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.values.Bytes.BytesBaseSortedSource;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
@@ -35,6 +34,7 @@ import org.apache.lucene.util.ByteBlockP
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -62,13 +62,13 @@ class FixedSortedBytesImpl {
     private final BytesRefHash hash;
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
-        AtomicLong bytesUsed, IOContext context) throws IOException {
+        Counter bytesUsed, IOContext context) throws IOException {
       this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
           bytesUsed, context);
     }
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
-        Allocator allocator, AtomicLong bytesUsed, IOContext context) throws IOException {
+        Allocator allocator, Counter bytesUsed, IOContext context) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
       ByteBlockPool pool = new ByteBlockPool(allocator);
       hash = new BytesRefHash(pool, BytesRefHash.DEFAULT_CAPACITY,

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java Wed Aug 31 11:22:05 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index.values;
 import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
 
 import java.io.IOException;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.values.Bytes.BytesBaseSource;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
@@ -33,6 +32,7 @@ import org.apache.lucene.util.AttributeS
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.PagedBytes;
 
@@ -56,7 +56,7 @@ class FixedStraightBytesImpl {
     private final int byteBlockSize;
     private IndexOutput datOut;
 
-    public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context) throws IOException {
+    public Writer(Directory dir, String id, Counter bytesUsed, IOContext context) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
       pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
       byteBlockSize = BYTE_BLOCK_SIZE;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Floats.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Floats.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Floats.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Floats.java Wed Aug 31 11:22:05 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index.values;
  */
 import java.io.IOException;
 import java.util.Collection;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.values.IndexDocValues.Source;
@@ -29,6 +28,7 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.FloatsRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -50,7 +50,7 @@ public class Floats {
   private static final byte[] DEFAULTS = new byte[] {0,0,0,0,0,0,0,0};
   
   public static Writer getWriter(Directory dir, String id, int precisionBytes,
-      AtomicLong bytesUsed, IOContext context) throws IOException {
+      Counter bytesUsed, IOContext context) throws IOException {
     if (precisionBytes != 4 && precisionBytes != 8) {
       throw new IllegalArgumentException("precisionBytes must be 4 or 8; got "
           + precisionBytes);
@@ -77,7 +77,7 @@ public class Floats {
     private final IOContext context; 
 
     protected FloatsWriter(Directory dir, String id, int precision,
-        AtomicLong bytesUsed, IOContext context) throws IOException {
+        Counter bytesUsed, IOContext context) throws IOException {
       super(bytesUsed);
       this.id = id;
       this.precision = (byte) precision;
@@ -161,7 +161,7 @@ public class Floats {
   // Writes 4 bytes (float) per value
   static final class Float4Writer extends FloatsWriter {
     private int[] values;
-    protected Float4Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
+    protected Float4Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
         throws IOException {
       super(dir, id, 4, bytesUsed, context);
       values = new int[1];
@@ -226,7 +226,7 @@ public class Floats {
   // Writes 8 bytes (double) per value
   static final class Float8Writer extends FloatsWriter {
     private long[] values;
-    protected Float8Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
+    protected Float8Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
         throws IOException {
       super(dir, id, 8, bytesUsed, context);
       values = new long[1];

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/IndexDocValuesArray.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/IndexDocValuesArray.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/IndexDocValuesArray.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/IndexDocValuesArray.java Wed Aug 31 11:22:05 2011
@@ -1,7 +1,6 @@
 package org.apache.lucene.index.values;
 
 import java.io.IOException;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.values.IndexDocValues.Source;
 import org.apache.lucene.index.values.IndexDocValues.SourceEnum;
@@ -9,6 +8,7 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.LongsRef;
 import org.apache.lucene.util.RamUsageEstimator;
 
@@ -34,13 +34,13 @@ import org.apache.lucene.util.RamUsageEs
  */
 abstract class IndexDocValuesArray extends Source {
 
-  private final AtomicLong bytesUsed;
+  private final Counter bytesUsed;
   private final int bytesPerValue;
   private int size = 0;
   private final ValueType type;
   protected int maxDocID = -1;
 
-  IndexDocValuesArray(AtomicLong bytesUsed, int bytesPerValue, ValueType type) {
+  IndexDocValuesArray(Counter bytesUsed, int bytesPerValue, ValueType type) {
     this.bytesUsed = bytesUsed;
     this.bytesPerValue = bytesPerValue;
     this.type = type;
@@ -113,13 +113,13 @@ abstract class IndexDocValuesArray exten
   final static class ByteValues extends IndexDocValuesArray {
     private byte[] values;
 
-    ByteValues(AtomicLong bytesUsed) {
+    ByteValues(Counter bytesUsed) {
       super(bytesUsed, 1, ValueType.FIXED_INTS_8);
       values = new byte[0];
     }
 
     ByteValues(IndexInput input, int numDocs) throws IOException {
-      super(new AtomicLong(), 1, ValueType.FIXED_INTS_8);
+      super(Counter.newCounter(), 1, ValueType.FIXED_INTS_8);
       values = new byte[numDocs];
       adjustSize(numDocs);
       input.readBytes(values, 0, values.length, false);
@@ -190,14 +190,14 @@ abstract class IndexDocValuesArray exten
   final static class ShortValues extends IndexDocValuesArray {
     private short[] values;
 
-    ShortValues(AtomicLong bytesUsed) {
+    ShortValues(Counter bytesUsed) {
       super(bytesUsed, RamUsageEstimator.NUM_BYTES_SHORT,
           ValueType.FIXED_INTS_16);
       values = new short[0];
     }
 
     ShortValues(IndexInput input, int numDocs) throws IOException {
-      super(new AtomicLong(), RamUsageEstimator.NUM_BYTES_SHORT,
+      super(Counter.newCounter(), RamUsageEstimator.NUM_BYTES_SHORT,
           ValueType.FIXED_INTS_16);
       values = new short[numDocs];
       adjustSize(numDocs);
@@ -274,13 +274,13 @@ abstract class IndexDocValuesArray exten
   final static class IntValues extends IndexDocValuesArray {
     private int[] values;
 
-    IntValues(AtomicLong bytesUsed) {
+    IntValues(Counter bytesUsed) {
       super(bytesUsed, RamUsageEstimator.NUM_BYTES_INT, ValueType.FIXED_INTS_32);
       values = new int[0];
     }
 
     IntValues(IndexInput input, int numDocs) throws IOException {
-      super(new AtomicLong(), RamUsageEstimator.NUM_BYTES_INT,
+      super(Counter.newCounter(), RamUsageEstimator.NUM_BYTES_INT,
           ValueType.FIXED_INTS_32);
       values = new int[numDocs];
       adjustSize(numDocs);
@@ -356,14 +356,14 @@ abstract class IndexDocValuesArray exten
   final static class LongValues extends IndexDocValuesArray {
     private long[] values;
 
-    LongValues(AtomicLong bytesUsed) {
+    LongValues(Counter bytesUsed) {
       super(bytesUsed, RamUsageEstimator.NUM_BYTES_LONG,
           ValueType.FIXED_INTS_64);
       values = new long[0];
     }
 
     LongValues(IndexInput input, int numDocs) throws IOException {
-      super(new AtomicLong(), RamUsageEstimator.NUM_BYTES_LONG,
+      super(Counter.newCounter(), RamUsageEstimator.NUM_BYTES_LONG,
           ValueType.FIXED_INTS_64);
       values = new long[numDocs];
       adjustSize(numDocs);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Ints.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Ints.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Ints.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Ints.java Wed Aug 31 11:22:05 2011
@@ -18,12 +18,12 @@ package org.apache.lucene.index.values;
  */
 
 import java.io.IOException;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.values.IntsImpl.IntsReader;
 import org.apache.lucene.index.values.IntsImpl.IntsWriter;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Counter;
 
 /**
  * @lucene.experimental
@@ -35,7 +35,7 @@ public class Ints {
   }
 
   public static Writer getWriter(Directory dir, String id,
-      AtomicLong bytesUsed, ValueType type, IOContext context) throws IOException {
+      Counter bytesUsed, ValueType type, IOContext context) throws IOException {
     return new IntsWriter(dir, id, bytesUsed, type, context);
   }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/IntsImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/IntsImpl.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/IntsImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/IntsImpl.java Wed Aug 31 11:22:05 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index.values;
  */
 import java.io.IOException;
 import java.util.Collection;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.values.IndexDocValues.Source;
@@ -33,6 +32,7 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LongsRef;
 import org.apache.lucene.util.packed.PackedInts;
@@ -70,7 +70,7 @@ class IntsImpl {
     private final IOContext context;
     
 
-    protected IntsWriter(Directory dir, String id, AtomicLong bytesUsed,
+    protected IntsWriter(Directory dir, String id, Counter bytesUsed,
         ValueType valueType, IOContext context) throws IOException {
       super(bytesUsed);
       this.context = context;
@@ -367,13 +367,13 @@ class IntsImpl {
   private static ValuesEnum directEnum(byte ord, AttributeSource attrSource, IndexInput input, int numDocs) throws IOException {
     switch (ord) {
     case FIXED_16:
-      return new ShortValues((AtomicLong)null).getDirectEnum(attrSource, input, numDocs);
+      return new ShortValues((Counter)null).getDirectEnum(attrSource, input, numDocs);
     case FIXED_32:
-      return new IntValues((AtomicLong)null).getDirectEnum(attrSource, input, numDocs);
+      return new IntValues((Counter)null).getDirectEnum(attrSource, input, numDocs);
     case FIXED_64:
-      return new LongValues((AtomicLong)null).getDirectEnum(attrSource, input, numDocs);
+      return new LongValues((Counter)null).getDirectEnum(attrSource, input, numDocs);
     case FIXED_8:
-      return new ByteValues((AtomicLong)null).getDirectEnum(attrSource, input, numDocs);
+      return new ByteValues((Counter)null).getDirectEnum(attrSource, input, numDocs);
     case PACKED:
       return new PackedIntsEnumImpl(attrSource, input);
     default:

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java Wed Aug 31 11:22:05 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index.values;
  */
 
 import java.io.IOException;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.values.Bytes.BytesBaseSource;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
@@ -35,6 +34,7 @@ import org.apache.lucene.util.ByteBlockP
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -60,12 +60,12 @@ class VarDerefBytesImpl {
       TrackingDirectBytesStartArray {
     int[] address;
 
-    AddressByteStartArray(int size, AtomicLong bytesUsed) {
+    AddressByteStartArray(int size, Counter bytesUsed) {
       super(size, bytesUsed);
     }
 
     @Override
-    public AtomicLong bytesUsed() {
+    public Counter bytesUsed() {
       return bytesUsed;
     }
 
@@ -118,14 +118,14 @@ class VarDerefBytesImpl {
         bytesUsed);
     private final BytesRefHash hash;
 
-    public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
+    public Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
         throws IOException {
       this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
           bytesUsed, context);
     }
 
     public Writer(Directory dir, String id, Allocator allocator,
-        AtomicLong bytesUsed, IOContext context) throws IOException {
+        Counter bytesUsed, IOContext context) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
       hash = new BytesRefHash(new ByteBlockPool(allocator), 16, array);
       docToAddress = new int[1];

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java Wed Aug 31 11:22:05 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index.values;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Comparator;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.values.Bytes.BytesBaseSortedSource;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
@@ -34,6 +33,7 @@ import org.apache.lucene.util.AttributeS
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -62,13 +62,13 @@ class VarSortedBytesImpl {
     private final BytesRefHash hash; 
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
-        AtomicLong bytesUsed, IOContext context) throws IOException {
+        Counter bytesUsed, IOContext context) throws IOException {
       this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
           bytesUsed, context);
     }
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
-        Allocator allocator, AtomicLong bytesUsed, IOContext context) throws IOException {
+        Allocator allocator, Counter bytesUsed, IOContext context) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
       this.hash = new BytesRefHash(new ByteBlockPool(allocator),
           BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java Wed Aug 31 11:22:05 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index.values;
  */
 
 import java.io.IOException;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.values.Bytes.BytesBaseSource;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
@@ -31,6 +30,7 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -57,7 +57,7 @@ class VarStraightBytesImpl {
     private final ByteBlockPool pool;
     private IndexOutput datOut;
     private boolean merge = false;
-    public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
+    public Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
         throws IOException {
       super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
       pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Writer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Writer.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Writer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Writer.java Wed Aug 31 11:22:05 2011
@@ -18,13 +18,13 @@ package org.apache.lucene.index.values;
  */
 import java.io.IOException;
 import java.util.Comparator;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.codecs.DocValuesConsumer;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
 
 /**
  * Abstract API for per-document stored primitive values of type <tt>byte[]</tt>
@@ -48,7 +48,7 @@ public abstract class Writer extends Doc
    *          internally allocated memory. All tracked bytes must be released
    *          once {@link #finish(int)} has been called.
    */
-  protected Writer(AtomicLong bytesUsed) {
+  protected Writer(Counter bytesUsed) {
     super(bytesUsed);
   }
 
@@ -193,7 +193,7 @@ public abstract class Writer extends Doc
    * @throws IOException
    */
   public static Writer create(ValueType type, String id, Directory directory,
-      Comparator<BytesRef> comp, AtomicLong bytesUsed, IOContext context) throws IOException {
+      Comparator<BytesRef> comp, Counter bytesUsed, IOContext context) throws IOException {
     if (comp == null) {
       comp = BytesRef.getUTF8SortedAsUnicodeComparator();
     }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java Wed Aug 31 11:22:05 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.util;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.store.DataOutput;
 
@@ -85,13 +84,13 @@ public final class ByteBlockPool {
   }
   
   public static class DirectTrackingAllocator extends Allocator {
-    private final AtomicLong bytesUsed;
+    private final Counter bytesUsed;
     
-    public DirectTrackingAllocator(AtomicLong bytesUsed) {
+    public DirectTrackingAllocator(Counter bytesUsed) {
       this(BYTE_BLOCK_SIZE, bytesUsed);
     }
 
-    public DirectTrackingAllocator(int blockSize, AtomicLong bytesUsed) {
+    public DirectTrackingAllocator(int blockSize, Counter bytesUsed) {
       super(blockSize);
       this.bytesUsed = bytesUsed;
     }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java Wed Aug 31 11:22:05 2011
@@ -60,7 +60,7 @@ public final class BytesRefHash {
   private int lastCount = -1;
   private int[] ords;
   private final BytesStartArray bytesStartArray;
-  private AtomicLong bytesUsed;
+  private Counter bytesUsed;
 
   /**
    * Creates a new {@link BytesRefHash} with a {@link ByteBlockPool} using a
@@ -90,7 +90,7 @@ public final class BytesRefHash {
     Arrays.fill(ords, -1);
     this.bytesStartArray = bytesStartArray;
     bytesStart = bytesStartArray.init();
-    bytesUsed = bytesStartArray.bytesUsed() == null? new AtomicLong(0) : bytesStartArray.bytesUsed();;
+    bytesUsed = bytesStartArray.bytesUsed() == null? Counter.newCounter() : bytesStartArray.bytesUsed();
     bytesUsed.addAndGet(hashSize * RamUsageEstimator.NUM_BYTES_INT);
   }
 
@@ -523,25 +523,25 @@ public final class BytesRefHash {
     public abstract int[] clear();
 
     /**
-     * A {@link AtomicLong} reference holding the number of bytes used by this
+     * A {@link Counter} reference holding the number of bytes used by this
      * {@link BytesStartArray}. The {@link BytesRefHash} uses this reference to
      * track it memory usage
      * 
      * @return a {@link AtomicLong} reference holding the number of bytes used
      *         by this {@link BytesStartArray}.
      */
-    public abstract AtomicLong bytesUsed();
+    public abstract Counter bytesUsed();
   }
   
   /**
-   * A direct {@link BytesStartArray} that tracks all memory allocation using an {@link AtomicLong} instance.
+   * A direct {@link BytesStartArray} that tracks all memory allocation using an {@link Counter} instance.
    */
   public static class TrackingDirectBytesStartArray extends BytesStartArray {
     protected final int initSize;
     private int[] bytesStart;
-    protected final AtomicLong bytesUsed;
+    protected final Counter bytesUsed;
     
-    public TrackingDirectBytesStartArray(int initSize, AtomicLong bytesUsed) {
+    public TrackingDirectBytesStartArray(int initSize, Counter bytesUsed) {
       this.initSize = initSize;
       this.bytesUsed = bytesUsed;
     }
@@ -572,7 +572,7 @@ public final class BytesRefHash {
     }
 
     @Override
-    public AtomicLong bytesUsed() {
+    public Counter bytesUsed() {
       return bytesUsed;
     }
   }
@@ -580,10 +580,10 @@ public final class BytesRefHash {
   public static class DirectBytesStartArray extends BytesStartArray {
     protected final int initSize;
     private int[] bytesStart;
-    private final AtomicLong bytesUsed;
+    private final Counter bytesUsed;
     
     public DirectBytesStartArray(int initSize) {
-      this.bytesUsed = new AtomicLong(0);
+      this.bytesUsed = Counter.newCounter();
       this.initSize = initSize;
     }
 
@@ -606,7 +606,7 @@ public final class BytesRefHash {
     }
 
     @Override
-    public AtomicLong bytesUsed() {
+    public Counter bytesUsed() {
       return bytesUsed;
     }
   }

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Counter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Counter.java?rev=1163576&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Counter.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/Counter.java Wed Aug 31 11:22:05 2011
@@ -0,0 +1,93 @@
+package org.apache.lucene.util;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * 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.
+ */
+
+/**
+ * Simple counter class
+ * 
+ * @lucene.internal
+ * @lucene.experimental
+ */
+public abstract class Counter {
+
+  /**
+   * Adds the given delta to the counters current value
+   * 
+   * @param delta
+   *          the delta to add
+   * @return the counters updated value
+   */
+  public abstract long addAndGet(long delta);
+
+  /**
+   * Returns the counters current value
+   * 
+   * @return the counters current value
+   */
+  public abstract long get();
+
+  /**
+   * Returns a new counter. The returned counter is not thread-safe.
+   */
+  public static Counter newCounter() {
+    return newCounter(false);
+  }
+
+  /**
+   * Returns a new counter.
+   * 
+   * @param threadSafe
+   *          <code>true</code> if the returned counter can be used by multiple
+   *          threads concurrently.
+   * @return a new counter.
+   */
+  public static Counter newCounter(boolean threadSafe) {
+    return threadSafe ? new AtomicCounter() : new SerialCounter();
+  }
+
+  private final static class SerialCounter extends Counter {
+    private long count = 0;
+
+    @Override
+    public long addAndGet(long delta) {
+      return count += delta;
+    }
+
+    @Override
+    public long get() {
+      return count;
+    };
+  }
+
+  private final static class AtomicCounter extends Counter {
+    private AtomicLong count;
+
+    @Override
+    public long addAndGet(long delta) {
+      return count.addAndGet(delta);
+    }
+
+    @Override
+    public long get() {
+      return count.get();
+    }
+
+  }
+}

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java Wed Aug 31 11:22:05 2011
@@ -22,7 +22,7 @@ import org.apache.lucene.util.ByteBlockP
  */
 
 /**
- * A {@link ByteBlockPool.Allocator} implementation that recycles unused byte
+ * A threadsafe {@link ByteBlockPool.Allocator} implementation that recycles unused byte
  * blocks in a buffer and reuses them in subsequent calls to
  * {@link #getByteBlock()}.
  * 
@@ -122,7 +122,7 @@ public final class RecyclingByteBlockAll
   /**
    * @return the number of bytes currently allocated by this {@link Allocator}
    */
-  public long bytesUsed() {
+  public synchronized long bytesUsed() {
     return bytesUsed.get();
   }
 

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/values/TestDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/values/TestDocValues.java?rev=1163576&r1=1163575&r2=1163576&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/values/TestDocValues.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/values/TestDocValues.java Wed Aug 31 11:22:05 2011
@@ -19,12 +19,12 @@ package org.apache.lucene.index.values;
 
 import java.io.IOException;
 import java.util.Comparator;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.values.IndexDocValues.SortedSource;
 import org.apache.lucene.index.values.IndexDocValues.Source;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.FloatsRef;
 import org.apache.lucene.util.LongsRef;
 import org.apache.lucene.util.LuceneTestCase;
@@ -60,7 +60,7 @@ public class TestDocValues extends Lucen
         .getUTF8SortedAsUnicodeComparator() : null;
 
     Directory dir = newDirectory();
-    final AtomicLong trackBytes = new AtomicLong(0);
+    final Counter trackBytes = Counter.newCounter();
     Writer w = Bytes.getWriter(dir, "test", mode, comp, fixedSize, trackBytes, newIOContext(random));
     int maxDoc = 220;
     final String[] values = new String[maxDoc];
@@ -182,7 +182,7 @@ public class TestDocValues extends Lucen
         ValueType.VAR_INTS, };
     for (int i = 0; i < minMax.length; i++) {
       Directory dir = newDirectory();
-      final AtomicLong trackBytes = new AtomicLong(0);
+      final Counter trackBytes = Counter.newCounter();
       Writer w = Ints.getWriter(dir, "test", trackBytes, ValueType.VAR_INTS, newIOContext(random));
       w.add(0, minMax[i][0]);
       w.add(1, minMax[i][1]);
@@ -223,7 +223,7 @@ public class TestDocValues extends Lucen
   public void testGetInt8Array() throws IOException {
     byte[] sourceArray = new byte[] {1,2,3};
     Directory dir = newDirectory();
-    final AtomicLong trackBytes = new AtomicLong(0);
+    final Counter trackBytes = Counter.newCounter();
     Writer w = Ints.getWriter(dir, "test", trackBytes, ValueType.FIXED_INTS_8, newIOContext(random));
     for (int i = 0; i < sourceArray.length; i++) {
       w.add(i, (long) sourceArray[i]);
@@ -244,7 +244,7 @@ public class TestDocValues extends Lucen
   public void testGetInt16Array() throws IOException {
     short[] sourceArray = new short[] {1,2,3};
     Directory dir = newDirectory();
-    final AtomicLong trackBytes = new AtomicLong(0);
+    final Counter trackBytes = Counter.newCounter();
     Writer w = Ints.getWriter(dir, "test", trackBytes, ValueType.FIXED_INTS_16, newIOContext(random));
     for (int i = 0; i < sourceArray.length; i++) {
       w.add(i, (long) sourceArray[i]);
@@ -265,7 +265,7 @@ public class TestDocValues extends Lucen
   public void testGetInt64Array() throws IOException {
     long[] sourceArray = new long[] {1,2,3};
     Directory dir = newDirectory();
-    final AtomicLong trackBytes = new AtomicLong(0);
+    final Counter trackBytes = Counter.newCounter();
     Writer w = Ints.getWriter(dir, "test", trackBytes, ValueType.FIXED_INTS_64, newIOContext(random));
     for (int i = 0; i < sourceArray.length; i++) {
       w.add(i, sourceArray[i]);
@@ -286,7 +286,7 @@ public class TestDocValues extends Lucen
   public void testGetInt32Array() throws IOException {
     int[] sourceArray = new int[] {1,2,3};
     Directory dir = newDirectory();
-    final AtomicLong trackBytes = new AtomicLong(0);
+    final Counter trackBytes = Counter.newCounter();
     Writer w = Ints.getWriter(dir, "test", trackBytes, ValueType.FIXED_INTS_32, newIOContext(random));
     for (int i = 0; i < sourceArray.length; i++) {
       w.add(i, (long) sourceArray[i]);
@@ -307,7 +307,7 @@ public class TestDocValues extends Lucen
   public void testGetFloat32Array() throws IOException {
     float[] sourceArray = new float[] {1,2,3};
     Directory dir = newDirectory();
-    final AtomicLong trackBytes = new AtomicLong(0);
+    final Counter trackBytes = Counter.newCounter();
     Writer w = Floats.getWriter(dir, "test", 4, trackBytes, newIOContext(random));
     for (int i = 0; i < sourceArray.length; i++) {
       w.add(i, sourceArray[i]);
@@ -328,7 +328,7 @@ public class TestDocValues extends Lucen
   public void testGetFloat64Array() throws IOException {
     double[] sourceArray = new double[] {1,2,3};
     Directory dir = newDirectory();
-    final AtomicLong trackBytes = new AtomicLong(0);
+    final Counter trackBytes = Counter.newCounter();
     Writer w = Floats.getWriter(dir, "test", 8, trackBytes, newIOContext(random));
     for (int i = 0; i < sourceArray.length; i++) {
       w.add(i, sourceArray[i]);
@@ -352,7 +352,7 @@ public class TestDocValues extends Lucen
     final long[] values = new long[NUM_VALUES];
     for (int rx = 1; rx < maxBit; rx++, maxV *= 2) {
       Directory dir = newDirectory();
-      final AtomicLong trackBytes = new AtomicLong(0);
+      final Counter trackBytes = Counter.newCounter();
       Writer w = Ints.getWriter(dir, "test", trackBytes, type, newIOContext(random));
       for (int i = 0; i < NUM_VALUES; i++) {
         final long v = random.nextLong() % (1 + maxV);
@@ -415,7 +415,7 @@ public class TestDocValues extends Lucen
 
   private void runTestFloats(int precision, double delta) throws IOException {
     Directory dir = newDirectory();
-    final AtomicLong trackBytes = new AtomicLong(0);
+    final Counter trackBytes = Counter.newCounter();
     Writer w = Floats.getWriter(dir, "test", precision, trackBytes, newIOContext(random));
     final int NUM_VALUES = 777 + random.nextInt(777);;
     final double[] values = new double[NUM_VALUES];