lucene-java-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From dor...@apache.org
Subject svn commit: r557678 - in /lucene/java/trunk: ./ src/java/org/apache/lucene/store/ src/test/org/apache/lucene/store/
Date Thu, 19 Jul 2007 16:27:49 GMT
Author: doronc
Date: Thu Jul 19 09:27:48 2007
New Revision: 557678

URL: http://svn.apache.org/viewvc?view=rev&rev=557678
Log:
LUCENE-957: RAMDirectory fixed for more than Integer.MAX_VALUE bytes.

Added:
    lucene/java/trunk/src/test/org/apache/lucene/store/TestHugeRamFile.java   (with props)
Modified:
    lucene/java/trunk/CHANGES.txt
    lucene/java/trunk/src/java/org/apache/lucene/store/RAMFile.java
    lucene/java/trunk/src/java/org/apache/lucene/store/RAMInputStream.java
    lucene/java/trunk/src/java/org/apache/lucene/store/RAMOutputStream.java
    lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMInputStream.java

Modified: lucene/java/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?view=diff&rev=557678&r1=557677&r2=557678
==============================================================================
--- lucene/java/trunk/CHANGES.txt (original)
+++ lucene/java/trunk/CHANGES.txt Thu Jul 19 09:27:48 2007
@@ -39,10 +39,13 @@
     IndexWriter's methods could cause deletes to be lost.
     (Steven Parkes via Mike McCandless)
       
- 4. LUCENE-962: Fixed case where an unhandled exception in
+ 5. LUCENE-962: Fixed case where an unhandled exception in
     IndexWriter.addDocument or IndexWriter.updateDocument could cause
     unreferenced files in the index to not be deleted
     (Steven Parkes via Mike McCandless)
+  
+ 6. LUCENE-957: RAMDirectory fixed to properly handle directories
+    larger than Integer.MAX_VALUE. (Doron Cohen)
       
 New features
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/RAMFile.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/RAMFile.java?view=diff&rev=557678&r1=557677&r2=557678
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/RAMFile.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/RAMFile.java Thu Jul 19 09:27:48 2007
@@ -60,7 +60,7 @@
 
   // Only one writing stream with no concurrent reading streams, so no file synchronization
required
   final byte[] addBuffer(int size) {
-    byte[] buffer = new byte[size];
+    byte[] buffer = newBuffer(size);
     if (directory!=null)
       synchronized (directory) {             // Ensure addition of buffer and adjustment
to directory size are atomic wrt directory
         buffers.add(buffer);
@@ -70,6 +70,16 @@
     else
       buffers.add(buffer);
     return buffer;
+  }
+
+  /**
+   * Expert: allocate a new buffer. 
+   * Subclasses can allocate differently. 
+   * @param size size of allocated buffer.
+   * @return allocated buffer.
+   */
+  byte[] newBuffer(int size) {
+    return new byte[size];
   }
 
   // Only valid if in a directory

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/RAMInputStream.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/RAMInputStream.java?view=diff&rev=557678&r1=557677&r2=557678
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/RAMInputStream.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/RAMInputStream.java Thu Jul 19 09:27:48
2007
@@ -38,9 +38,12 @@
   private long bufferStart;
   private int bufferLength;
 
-  public RAMInputStream(RAMFile f) {
+  RAMInputStream(RAMFile f) throws IOException {
     file = f;
     length = file.length;
+    if (length/BUFFER_SIZE >= Integer.MAX_VALUE) {
+      throw new IOException("Too large RAMFile! "+length); 
+    }
 
     // make sure that we switch to the
     // first needed buffer lazily
@@ -87,11 +90,9 @@
     } else {
       currentBuffer = (byte[]) file.buffers.get(currentBufferIndex);
       bufferPosition = 0;
-      bufferStart = BUFFER_SIZE * currentBufferIndex;
-      bufferLength = (int) (length - bufferStart);
-      if (bufferLength > BUFFER_SIZE) {
-        bufferLength = BUFFER_SIZE;
-      }
+      bufferStart = (long) BUFFER_SIZE * (long) currentBufferIndex;
+      long buflen = length - bufferStart;
+      bufferLength = buflen > BUFFER_SIZE ? BUFFER_SIZE : (int) buflen;
     }
   }
 
@@ -100,8 +101,7 @@
   }
 
   public void seek(long pos) throws IOException {
-    long bufferStart = currentBufferIndex * BUFFER_SIZE;
-    if (pos < bufferStart || pos >= bufferStart + BUFFER_SIZE) {
+    if (currentBuffer==null || pos < bufferStart || pos >= bufferStart + BUFFER_SIZE)
{
       currentBufferIndex = (int) (pos / BUFFER_SIZE);
       switchCurrentBuffer();
     }

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/RAMOutputStream.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/RAMOutputStream.java?view=diff&rev=557678&r1=557677&r2=557678
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/RAMOutputStream.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/RAMOutputStream.java Thu Jul 19 09:27:48
2007
@@ -130,7 +130,7 @@
       currentBuffer = (byte[]) file.buffers.get(currentBufferIndex);
     }
     bufferPosition = 0;
-    bufferStart = BUFFER_SIZE * currentBufferIndex;
+    bufferStart = (long) BUFFER_SIZE * (long) currentBufferIndex;
     bufferLength = currentBuffer.length;
   }
 

Modified: lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMInputStream.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMInputStream.java?view=diff&rev=557678&r1=557677&r2=557678
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMInputStream.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMInputStream.java Thu Jul 19
09:27:48 2007
@@ -1,5 +1,7 @@
 package org.apache.lucene.store;
 
+import java.io.IOException;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -27,8 +29,9 @@
   private String name;
   private boolean isClone;
 
-  /** Construct an empty output buffer. */
-  public MockRAMInputStream(MockRAMDirectory dir, String name, RAMFile f) {
+  /** Construct an empty output buffer. 
+   * @throws IOException */
+  public MockRAMInputStream(MockRAMDirectory dir, String name, RAMFile f) throws IOException
{
     super(f);
     this.name = name;
     this.dir = dir;

Added: lucene/java/trunk/src/test/org/apache/lucene/store/TestHugeRamFile.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/store/TestHugeRamFile.java?view=auto&rev=557678
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/store/TestHugeRamFile.java (added)
+++ lucene/java/trunk/src/test/org/apache/lucene/store/TestHugeRamFile.java Thu Jul 19 09:27:48
2007
@@ -0,0 +1,101 @@
+package org.apache.lucene.store;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.HashMap;
+
+import junit.framework.TestCase;
+
+/** Test huge RAMFile with more than Integer.MAX_VALUE bytes. */
+public class TestHugeRamFile extends TestCase {
+  
+  private static final long MAX_VALUE = (long) 2 * (long) Integer.MAX_VALUE;
+
+  /** Fake a huge ram file by using the same byte buffer for all 
+   * buffers under maxint. */
+  private static class DenseRAMFile extends RAMFile {
+    private long capacity = 0;
+    private HashMap singleBuffers = new HashMap();
+    byte[] newBuffer(int size) {
+      capacity += size;
+      if (capacity <= MAX_VALUE) {
+        // below maxint we reuse buffers
+        byte buf[] = (byte[]) singleBuffers.get(new Integer(size));
+        if (buf==null) {
+          buf = new byte[size]; 
+          //System.out.println("allocate: "+size);
+          singleBuffers.put(new Integer(size),buf);
+        }
+        return buf;
+      }
+      //System.out.println("allocate: "+size); System.out.flush();
+      return new byte[size];
+    }
+  }
+  
+  /** Test huge RAMFile with more than Integer.MAX_VALUE bytes. (LUCENE-957) */
+  public void testHugeFile() throws IOException {
+    DenseRAMFile f = new DenseRAMFile();
+    // output part
+    RAMOutputStream out = new RAMOutputStream(f);
+    byte b1[] = new byte[RAMOutputStream.BUFFER_SIZE];
+    byte b2[] = new byte[RAMOutputStream.BUFFER_SIZE / 3];
+    for (int i = 0; i < b1.length; i++) {
+      b1[i] = (byte) (i & 0x0007F);
+    }
+    for (int i = 0; i < b2.length; i++) {
+      b2[i] = (byte) (i & 0x0003F);
+    }
+    long n = 0;
+    assertEquals("output length must match",n,out.length());
+    while (n <= MAX_VALUE - b1.length) {
+      out.writeBytes(b1,0,b1.length);
+      out.flush();
+      n += b1.length;
+      assertEquals("output length must match",n,out.length());
+    }
+    //System.out.println("after writing b1's, length = "+out.length()+" (MAX_VALUE="+MAX_VALUE+")");
+    int m = b2.length;
+    long L = 12;
+    for (int j=0; j<L; j++) {
+      for (int i = 0; i < b2.length; i++) {
+        b2[i]++;
+      }
+      out.writeBytes(b2,0,m);
+      out.flush();
+      n += m;
+      assertEquals("output length must match",n,out.length());
+    }
+    out.close();
+    // input part
+    RAMInputStream in = new RAMInputStream(f);
+    assertEquals("input length must match",n,in.length());
+    //System.out.println("input length = "+in.length()+" % 1024 = "+in.length()%1024);
+    for (int j=0; j<L; j++) {
+      long loc = n - (L-j)*m; 
+      in.seek(loc/3);
+      in.seek(loc);
+      for (int i=0; i<m; i++) {
+        byte bt = in.readByte();
+        byte expected = (byte) (1 + j + (i & 0x0003F));
+        assertEquals("must read same value that was written! j="+j+" i="+i,expected,bt);
+      }
+    }
+  }
+}

Propchange: lucene/java/trunk/src/test/org/apache/lucene/store/TestHugeRamFile.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/java/trunk/src/test/org/apache/lucene/store/TestHugeRamFile.java
------------------------------------------------------------------------------
    svn:executable = *



Mime
View raw message