You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2010/01/18 16:02:20 UTC

svn commit: r900417 - in /lucene/java/branches/flex_1458/src/java/org/apache/lucene: index/ index/codecs/intblock/ index/codecs/sep/ util/

Author: mikemccand
Date: Mon Jan 18 15:02:20 2010
New Revision: 900417

URL: http://svn.apache.org/viewvc?rev=900417&view=rev
Log:
flex: added an oal.util.IntsRef; added lucene.{experimental,internal} tags; cleaned up unused imports/attrs; etc.

Added:
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/IntsRef.java   (with props)
Modified:
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocsEnum.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/PositionsEnum.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/IntBlockCodec.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/SimpleIntBlockIndexInput.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/SimpleIntBlockIndexOutput.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntStreamFactory.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepCodec.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepDocsReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepDocsWriter.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepPositionsReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepPositionsWriter.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntFactory.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntIndexInput.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntIndexOutput.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BytesRef.java

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocsEnum.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocsEnum.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocsEnum.java Mon Jan 18 15:02:20 2010
@@ -44,7 +44,7 @@
   // nocommit -- state in API that doc/freq are undefined
   // (defined?) after this?
   // nocommit -- fix this API so that intblock codecs are
-  // able to return their own int arrays, to save a copy
+  // able to return their own int arrays, to save a copy...  IntsRef?
   /** Bulk read: returns number of docs read.
    * 
    *  <p>NOTE: the default impl simply delegates to {@link

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/PositionsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/PositionsEnum.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/PositionsEnum.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/PositionsEnum.java Mon Jan 18 15:02:20 2010
@@ -43,6 +43,7 @@
 
   public abstract int getPayloadLength();
 
+  // nocommit -- how to use BytesRef here?
   // nocommit -- improve this so that readers that do their
   // own buffering can save a copy
   public abstract byte[] getPayload(byte[] data, int offset) throws IOException;

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java Mon Jan 18 15:02:20 2010
@@ -25,6 +25,7 @@
 
 import org.apache.lucene.index.codecs.sep.IntIndexInput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.IntsRef;
 
 /** Abstract base class that reads fixed-size blocks of ints
  *  from an IndexInput.  While this is a simple approach, a
@@ -80,15 +81,14 @@
     private long lastBlockFP;
     private final BlockReader blockReader;
     private final int blockSize;
-
-    private final BulkReadResult result = new BulkReadResult();
+    private final IntsRef bulkResult = new IntsRef();
 
     public Reader(final IndexInput in, final int[] pending, final BlockReader blockReader)
     throws IOException {
       this.in = in;
       this.pending = pending;
       this.blockSize = pending.length;
-      result.buffer = pending;
+      bulkResult.ints = pending;
       this.blockReader = blockReader;
       upto = blockSize;
     }
@@ -125,22 +125,22 @@
     }
 
     @Override
-    public BulkReadResult read(final int[] buffer, final int count) throws IOException {
+    public IntsRef read(final int count) throws IOException {
       this.maybeSeek();
       if (upto == blockSize) {
         blockReader.readBlock();
         upto = 0;
       }
-      result.offset = upto;
+      bulkResult.offset = upto;
       if (upto + count < blockSize) {
-        result.len = count;
+        bulkResult.length = count;
         upto += count;
       } else {
-        result.len = blockSize - upto;
+        bulkResult.length = blockSize - upto;
         upto = blockSize;
       }
 
-      return result;
+      return bulkResult;
     }
 
     @Override
@@ -204,7 +204,6 @@
       final State iState = (State) state;
       this.fp = iState.fp;
       this.upto = iState.upto;
-
     }
   }
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/IntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/IntBlockCodec.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/IntBlockCodec.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/IntBlockCodec.java Mon Jan 18 15:02:20 2010
@@ -40,6 +40,9 @@
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 
+/**
+ * @lucene.experimental
+ */
 public class IntBlockCodec extends Codec {
 
   public IntBlockCodec() {
@@ -77,14 +80,6 @@
     }
   }
 
-  /*
-  final static String DOC_EXTENSION = "doc";
-  final static String SKIP_EXTENSION = "skp";
-  final static String FREQ_EXTENSION = "frq";
-  final static String PROX_EXTENSION = "prx";
-  final static String PAYLOAD_EXTENSION = "pyl";
-  */
-
   @Override
   public FieldsProducer fieldsProducer(Directory dir, FieldInfos fieldInfos, SegmentInfo si, int readBufferSize, int indexDivisor) throws IOException {
     StandardDocsProducer docsReader = new SepDocsReader(dir, si, readBufferSize, new SimpleIntBlockFactory(1024));

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/SimpleIntBlockIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/SimpleIntBlockIndexInput.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/SimpleIntBlockIndexInput.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/SimpleIntBlockIndexInput.java Mon Jan 18 15:02:20 2010
@@ -27,8 +27,12 @@
 
 import java.io.IOException;
 
-/** Don't use this class!!  It naively encodes ints one vInt
- *  at a time.  Use it only for testing.  */
+/**
+ * Don't use this class!!  It naively encodes ints one vInt
+ * at a time.  Use it only for testing.
+ *
+ * @lucene.experimental
+ */
 public class SimpleIntBlockIndexInput extends FixedIntBlockIndexInput {
 
   public SimpleIntBlockIndexInput(Directory dir, String fileName, int readBufferSize) throws IOException {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/SimpleIntBlockIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/SimpleIntBlockIndexOutput.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/SimpleIntBlockIndexOutput.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/intblock/SimpleIntBlockIndexOutput.java Mon Jan 18 15:02:20 2010
@@ -27,8 +27,12 @@
 
 import java.io.IOException;
 
-/** Don't use this class!!  It naively encodes ints one vInt
- *  at a time.  Use it only for testing.  */
+/**
+ * Don't use this class!!  It naively encodes ints one vInt
+ * at a time.  Use it only for testing.
+ *
+ * @lucene.experimental
+ */
 public class SimpleIntBlockIndexOutput extends FixedIntBlockIndexOutput {
 
   public final static String CODEC = "SIMPLE_INT_BLOCKS";

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java Mon Jan 18 15:02:20 2010
@@ -18,13 +18,17 @@
  */
 
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.IntsRef;
 
 import java.io.IOException;
+import java.io.Closeable;
 
 /** Defines basic API for writing ints to an IndexOutput.
  *  IntBlockCodec interacts with this API. @see
- *  IntBlockReader */
-public abstract class IntIndexInput {
+ *  IntBlockReader
+ *
+ * @lucene.experimental */
+public abstract class IntIndexInput implements Closeable {
 
   public abstract Reader reader() throws IOException;
 
@@ -53,20 +57,30 @@
     public abstract void setState(IndexState state);
   }
 
-  public static final class BulkReadResult {
-    public int[] buffer;
-    public int offset;
-    public int len;
-  };
-
   public abstract static class Reader {
 
     /** Reads next single int */
     public abstract int next() throws IOException;
 
     /** Reads next chunk of ints */
-    public abstract BulkReadResult read(int[] buffer, int count) throws IOException;
+    private IntsRef bulkResult;
+
+    /** Read up to count ints. */
+    public IntsRef read(int count) throws IOException {
+      if (bulkResult == null) {
+        bulkResult = new IntsRef();
+        bulkResult.ints = new int[count];
+      } else {
+        bulkResult.grow(count);
+      }
+      for(int i=0;i<count;i++) {
+        bulkResult.ints[i] = next();
+      }
+      bulkResult.length = count;
+      return bulkResult;
+    }
 
+    // nocommit
     public abstract String descFilePointer() throws IOException;
   }
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java Mon Jan 18 15:02:20 2010
@@ -23,13 +23,16 @@
 import org.apache.lucene.store.IndexOutput;
 
 import java.io.IOException;
+import java.io.Closeable;
 
 /** Defines basic API for writing ints to an IndexOutput.
  *  IntBlockCodec interacts with this API. @see
  *  IntBlockReader.
  *
- * <p>NOTE: block sizes could be variable */
-public abstract class IntIndexOutput {
+ * <p>NOTE: block sizes could be variable
+ *
+ * @lucene.experimental */
+public abstract class IntIndexOutput implements Closeable {
   /** Write an int to the primary file */
   public abstract void write(int v) throws IOException;
 
@@ -55,5 +58,6 @@
 
   public abstract void close() throws IOException;
 
+  // nocommit
   public abstract String descFilePointer() throws IOException;
 }
\ No newline at end of file

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntStreamFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntStreamFactory.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntStreamFactory.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/IntStreamFactory.java Mon Jan 18 15:02:20 2010
@@ -22,6 +22,7 @@
 
 import java.io.IOException;
 
+/** @lucene.experimental */
 public abstract class IntStreamFactory {
   public IntIndexInput openInput(Directory dir, String fileName) throws IOException {
     return openInput(dir, fileName, BufferedIndexInput.BUFFER_SIZE);

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepCodec.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepCodec.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepCodec.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepCodec.java Mon Jan 18 15:02:20 2010
@@ -37,6 +37,7 @@
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 
+/** @lucene.experimental */
 public class SepCodec extends Codec {
 
   public SepCodec() {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepDocsReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepDocsReader.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepDocsReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepDocsReader.java Mon Jan 18 15:02:20 2010
@@ -34,7 +34,10 @@
 import org.apache.lucene.util.Bits;
 
 /** Concrete class that reads the current doc/freq/skip
- *  postings format */
+ *  postings format.    
+ *
+ * @lucene.experimental
+ */
 
 // nocommit -- should we switch "hasProx" higher up?  and
 // create two separate docs readers, one that also reads

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepDocsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepDocsWriter.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepDocsWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepDocsWriter.java Mon Jan 18 15:02:20 2010
@@ -29,8 +29,9 @@
 import org.apache.lucene.store.IndexOutput;
 
 /** Writes frq to .frq, docs to .doc, pos to .pos, payloads
- *  to .pyl, skip data to .skp */
-
+ *  to .pyl, skip data to .skp
+ *
+ * @lucene.experimental */
 public final class SepDocsWriter extends StandardDocsConsumer {
   final static String CODEC = "SepDocFreqSkip";
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepPositionsReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepPositionsReader.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepPositionsReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepPositionsReader.java Mon Jan 18 15:02:20 2010
@@ -29,6 +29,7 @@
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 
+/** @lucene.experimental */
 public class SepPositionsReader extends StandardPositionsProducer {
   
   final IntIndexInput posIn;

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepPositionsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepPositionsWriter.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepPositionsWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepPositionsWriter.java Mon Jan 18 15:02:20 2010
@@ -26,6 +26,7 @@
 import org.apache.lucene.index.codecs.PositionsConsumer;
 import org.apache.lucene.index.codecs.Codec;
 
+/** @lucene.experimental */
 public final class SepPositionsWriter extends PositionsConsumer {
 
   final static String CODEC = "SepPositionsPayloads";

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java Mon Jan 18 15:02:20 2010
@@ -27,6 +27,8 @@
 /**
  * Implements the skip list reader for the default posting list format
  * that stores positions and payloads.
+ *
+ * @lucene.experimental
  */
 
 // TODO: rewrite this as recursive classes?

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java Mon Jan 18 15:02:20 2010
@@ -30,6 +30,8 @@
 /**
  * Implements the skip list writer for the default posting list format
  * that stores positions and payloads.
+ *
+ * @lucene.experimental
  */
 class SepSkipListWriter extends MultiLevelSkipListWriter {
   private int[] lastSkipDoc;

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntFactory.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntFactory.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntFactory.java Mon Jan 18 15:02:20 2010
@@ -20,6 +20,7 @@
 import org.apache.lucene.store.Directory;
 import java.io.IOException;
 
+/** @lucene.experimental */
 public class SingleIntFactory extends IntStreamFactory {
   @Override
   public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntIndexInput.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntIndexInput.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntIndexInput.java Mon Jan 18 15:02:20 2010
@@ -24,7 +24,12 @@
 import org.apache.lucene.store.IndexInput;
 
 /** Reads IndexInputs written with {@link
- * SingleIntIndexoutput} */
+ *  SingleIntIndexoutput}.  NOTE: this class is just for
+ *  demonstration puprposes (it is a very slow way to read a
+ *  block of ints).
+ *
+ * @lucene.experimental
+ */
 public class SingleIntIndexInput extends IntIndexInput {
   private final IndexInput in;
 
@@ -48,11 +53,8 @@
     // clone:
     private final IndexInput in;
 
-    private final BulkReadResult result = new BulkReadResult();
-
     public Reader(IndexInput in) {
       this.in = in;
-      result.offset = 0;
     }
 
     /** Reads next single int */
@@ -61,17 +63,6 @@
       return in.readVInt();
     }
 
-    /** Reads next chunk of ints */
-    @Override
-    public BulkReadResult read(int[] buffer, int count) throws IOException {
-      result.buffer = buffer;
-      for(int i=0;i<count;i++) {
-        buffer[i] = in.readVInt();
-      }
-      result.len = count;
-      return result;
-    }
-
     @Override
     public String descFilePointer() {
       return Long.toString(in.getFilePointer());

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntIndexOutput.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntIndexOutput.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/sep/SingleIntIndexOutput.java Mon Jan 18 15:02:20 2010
@@ -24,15 +24,15 @@
 import java.io.IOException;
 
 /** Writes ints directly to the file (not in blocks) as
- *  vInt */
-
+ *  vInt
+ * 
+ * @lucene.experimental
+*/
 public class SingleIntIndexOutput extends IntIndexOutput {
   private final IndexOutput out;
   final static String CODEC = "SINGLE_INTS";
   final static int VERSION_START = 0;
   final static int VERSION_CURRENT = VERSION_START;
-  private long markPosition;
-  private long lastSavePosition;
 
   public SingleIntIndexOutput(Directory dir, String fileName) throws IOException {
     out = dir.createOutput(fileName);

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BytesRef.java?rev=900417&r1=900416&r2=900417&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BytesRef.java Mon Jan 18 15:02:20 2010
@@ -20,7 +20,9 @@
 import java.io.UnsupportedEncodingException;
 
 /** Represents byte[], as a slice (offset + length) into an
- *  existing byte[]. */
+ *  existing byte[].
+ *
+ *  @lucene.internal */
 public final class BytesRef {
 
   public byte[] bytes;
@@ -66,11 +68,11 @@
 
   public boolean bytesEquals(BytesRef other) {
     if (length == other.length) {
-      int upto = offset;
       int otherUpto = other.offset;
       final byte[] otherBytes = other.bytes;
-      for(int i=0;i<length;i++) {
-        if (bytes[upto++] != otherBytes[otherUpto++]) {
+      final int end = offset + length;
+      for(int upto=offset;upto<end;upto++,otherUpto++) {
+        if (bytes[upto] != otherBytes[otherUpto]) {
           return false;
         }
       }
@@ -82,11 +84,7 @@
 
   @Override
   public Object clone() {
-    BytesRef other = new BytesRef();
-    other.bytes = new byte[length];
-    System.arraycopy(bytes, offset, other.bytes, 0, length);
-    other.length = length;
-    return other;
+    return new BytesRef(this);
   }
 
   public boolean startsWith(BytesRef other, int pos) {
@@ -110,25 +108,16 @@
   }
 
   public boolean endsWith(BytesRef other) {
-    return startsWith(other, length - other.length);   
+    return startsWith(other, length - other.length);
   }
   
   @Override
   public int hashCode() {
     final int prime = 31;
-    int result = 1;
-    result = prime * result + hash(bytes);
-    return result;
-  }
-  
-  private int hash(byte a[]) {
-    if (a == null) {
-      return 0;
-    }
-    int result = 1;
-    int upTo = offset;
-    for(int i = 0; i < length; i++) {
-      result = 31 * result + bytes[upTo++];
+    int result = 0;
+    final int end = offset + length;
+    for(int i=offset;i<end;i++) {
+      result = prime * result + bytes[i];
     }
     return result;
   }

Added: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/IntsRef.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/IntsRef.java?rev=900417&view=auto
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/IntsRef.java (added)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/IntsRef.java Mon Jan 18 15:02:20 2010
@@ -0,0 +1,96 @@
+package org.apache.lucene.util;
+
+/**
+ * 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.
+ */
+
+/** Represents int[], as a slice (offset + length) into an
+ *  existing int[].
+ *
+ *  @lucene.internal */
+public final class IntsRef {
+
+  public int[] ints;
+  public int offset;
+  public int length;
+
+  public IntsRef() {
+  }
+
+  public IntsRef(int[] ints, int offset, int length) {
+    this.ints = ints;
+    this.offset = offset;
+    this.length = length;
+  }
+
+  public IntsRef(IntsRef other) {
+    copy(other);
+  }
+
+  @Override
+  public Object clone() {
+    return new IntsRef(this);
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 0;
+    final int end = offset + length;
+    for(int i = offset; i < end; i++) {
+      result = prime * result + ints[i];
+    }
+    return result;
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    return this.intsEquals((IntsRef) other);
+  }
+
+  public boolean intsEquals(IntsRef other) {
+    if (length == other.length) {
+      int otherUpto = other.offset;
+      final int[] otherInts = other.ints;
+      final int end = offset + length;
+      for(int upto=offset;upto<end;upto++,otherUpto++) {
+        if (ints[upto] != otherInts[otherUpto]) {
+          return false;
+        }
+      }
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  public void copy(IntsRef other) {
+    if (ints == null) {
+      ints = new int[other.length];
+    } else {
+      ints = ArrayUtil.grow(ints, other.length);
+    }
+    System.arraycopy(other.ints, other.offset, ints, 0, other.length);
+    length = other.length;
+    offset = 0;
+  }
+
+  public void grow(int newLength) {
+    if (ints.length < newLength) {
+      ints = ArrayUtil.grow(ints, newLength);
+    }
+  }
+}

Propchange: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/IntsRef.java
------------------------------------------------------------------------------
    svn:eol-style = native