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 2009/03/20 18:13:43 UTC

svn commit: r756635 - in /lucene/java/trunk: ./ src/java/org/apache/lucene/document/ src/java/org/apache/lucene/index/ src/test/org/apache/lucene/document/

Author: mikemccand
Date: Fri Mar 20 17:13:42 2009
New Revision: 756635

URL: http://svn.apache.org/viewvc?rev=756635&view=rev
Log:
LUCENE-652: add org.apache.lucene.document.CompressionTools; deprecate Field.Store.COMPRESS

Added:
    lucene/java/trunk/src/java/org/apache/lucene/document/CompressionTools.java   (with props)
Modified:
    lucene/java/trunk/CHANGES.txt
    lucene/java/trunk/src/java/org/apache/lucene/document/Field.java
    lucene/java/trunk/src/java/org/apache/lucene/index/FieldsReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/FieldsWriter.java
    lucene/java/trunk/src/test/org/apache/lucene/document/TestBinaryDocument.java

Modified: lucene/java/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?rev=756635&r1=756634&r2=756635&view=diff
==============================================================================
--- lucene/java/trunk/CHANGES.txt (original)
+++ lucene/java/trunk/CHANGES.txt Fri Mar 20 17:13:42 2009
@@ -59,6 +59,10 @@
 9. LUCENE-1186: Add Analyzer.close() to free internal ThreadLocal
    resources.  (Christian Kohlschütter via Mike McCandless)
 
+10. LUCENE-652: Added org.apache.lucene.document.CompressionTools, to
+    enable compressing & decompressing binary content, external to
+    Lucene's indexing.  Deprecated Field.Store.COMPRESS.
+
 Bug fixes
 
 1. LUCENE-1415: MultiPhraseQuery has incorrect hashCode() and equals()

Added: lucene/java/trunk/src/java/org/apache/lucene/document/CompressionTools.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/document/CompressionTools.java?rev=756635&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/document/CompressionTools.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/document/CompressionTools.java Fri Mar 20 17:13:42 2009
@@ -0,0 +1,110 @@
+package org.apache.lucene.document;
+
+/**
+ * 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.util.zip.Deflater;
+import java.util.zip.Inflater;
+import java.util.zip.DataFormatException;
+import java.io.ByteArrayOutputStream;
+
+/** Simple utility class providing static methods to
+ *  compress and decompress binary data for stored fields.
+ *  This class uses java.util.zip.Deflater and Inflater
+ *  classes to compress and decompress.
+ *
+ *  To compress a String:
+ *  <pre>
+ *    String string = ...
+ *    byte[] bytes = compress(string.getBytes("UTF-8");
+ *  </pre>
+ *  and  to decompress:
+ *  <pre>
+ *    new String(decompress(bytes), "UTF-8");
+ *  </pre>
+ */
+
+public class CompressionTools {
+
+  // Export only static methods
+  private CompressionTools() {}
+
+  /** Compresses the specified byte range using the
+   *  specified compressionLevel (constants are defined in
+   *  java.util.zip.Deflater). */
+  public static byte[] compress(byte[] value, int offset, int length, int compressionLevel) {
+
+    /* Create an expandable byte array to hold the compressed data.
+     * You cannot use an array that's the same size as the orginal because
+     * there is no guarantee that the compressed data will be smaller than
+     * the uncompressed data. */
+    ByteArrayOutputStream bos = new ByteArrayOutputStream(length);
+
+    Deflater compressor = new Deflater();
+
+    try {
+      compressor.setLevel(compressionLevel);
+      compressor.setInput(value, offset, length);
+      compressor.finish();
+
+      // Compress the data
+      final byte[] buf = new byte[1024];
+      while (!compressor.finished()) {
+        int count = compressor.deflate(buf);
+        bos.write(buf, 0, count);
+      }
+    } finally {
+      compressor.end();
+    }
+
+    return bos.toByteArray();
+  }
+
+  /** Compresses the specified byte range, with default BEST_COMPRESSION level */
+  public static byte[] compress(byte[] value, int offset, int length) {
+    return compress(value, offset, length, Deflater.BEST_COMPRESSION);
+  }
+  
+  /** Compresses all bytes in the array, with default BEST_COMPRESSION level */
+  public static byte[] compress(byte[] value) {
+    return compress(value, 0, value.length, Deflater.BEST_COMPRESSION);
+  }
+
+  /** Decompress the byte array previously returned by
+   *  compress */
+  public static byte[] decompress(byte[] value) throws DataFormatException {
+    // Create an expandable byte array to hold the decompressed data
+    ByteArrayOutputStream bos = new ByteArrayOutputStream(value.length);
+
+    Inflater decompressor = new Inflater();
+
+    try {
+      decompressor.setInput(value);
+
+      // Decompress the data
+      final byte[] buf = new byte[1024];
+      while (!decompressor.finished()) {
+        int count = decompressor.inflate(buf);
+        bos.write(buf, 0, count);
+      }
+    } finally {  
+      decompressor.end();
+    }
+    
+    return bos.toByteArray();
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/document/CompressionTools.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/document/Field.java?rev=756635&r1=756634&r2=756635&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/document/Field.java Fri Mar 20 17:13:42 2009
@@ -43,6 +43,7 @@
 
     /** Store the original field value in the index in a compressed form. This is
      * useful for long documents and for binary valued fields.
+     * @deprecated Please use {@link CompressionTools} instead
      */
     public static final Store COMPRESS = new Store("COMPRESS");
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/FieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FieldsReader.java?rev=756635&r1=756634&r2=756635&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FieldsReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FieldsReader.java Fri Mar 20 17:13:42 2009
@@ -25,11 +25,9 @@
 import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.util.CloseableThreadLocal;
 
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.Reader;
 import java.util.zip.DataFormatException;
-import java.util.zip.Inflater;
 
 /**
  * Class responsible for access to stored document fields.
@@ -596,40 +594,19 @@
         return null;     
     }
   }
-  
-  private final byte[] uncompress(final byte[] input)
-          throws CorruptIndexException, IOException {
-
-    // Create an expandable byte array to hold the decompressed data
-    ByteArrayOutputStream bos = new ByteArrayOutputStream(input.length);
-
-    Inflater decompressor = new Inflater();
 
+  private byte[] uncompress(byte[] b)
+          throws CorruptIndexException {
     try {
-      decompressor.setInput(input);
-
-      // Decompress the data
-      byte[] buf = new byte[1024];
-      while (!decompressor.finished()) {
-        try {
-          int count = decompressor.inflate(buf);
-          bos.write(buf, 0, count);
-        }
-        catch (DataFormatException e) {
-          // this will happen if the field is not compressed
-          CorruptIndexException newException = new CorruptIndexException("field data are in wrong format: " + e.toString());
-          newException.initCause(e);
-          throw newException;
-        }
-      }
-    } finally {  
-      decompressor.end();
+      return CompressionTools.decompress(b);
+    } catch (DataFormatException e) {
+      // this will happen if the field is not compressed
+      CorruptIndexException newException = new CorruptIndexException("field data are in wrong format: " + e.toString());
+      newException.initCause(e);
+      throw newException;
     }
-    
-    // Get the decompressed data
-    return bos.toByteArray();
   }
-  
+
   // Instances of this class hold field properties and data
   // for merge
   final static class FieldForMerge extends AbstractField {

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/FieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FieldsWriter.java?rev=756635&r1=756634&r2=756635&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FieldsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FieldsWriter.java Fri Mar 20 17:13:42 2009
@@ -16,13 +16,12 @@
  * the License.
  */
 
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.zip.Deflater;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.document.CompressionTools;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.store.IndexOutput;
@@ -203,10 +202,10 @@
         } else {
           // check if it is a binary field
           if (field.isBinary()) {
-            data = compress(field.getBinaryValue(), field.getBinaryOffset(), field.getBinaryLength());
+            data = CompressionTools.compress(field.getBinaryValue(), field.getBinaryOffset(), field.getBinaryLength());
           } else {
             byte x[] = field.stringValue().getBytes("UTF-8");
-            data = compress(x, 0, x.length);
+            data = CompressionTools.compress(x, 0, x.length);
           }
           len = data.length;
           offset = 0;
@@ -269,43 +268,4 @@
               writeField(fieldInfos.fieldInfo(field.name()), field);
         }
     }
-
-    private final byte[] compress (byte[] input, int offset, int length) {
-      // Create the compressor with highest level of compression
-      Deflater compressor = new Deflater();
-      compressor.setLevel(Deflater.BEST_COMPRESSION);
-
-      // Give the compressor the data to compress
-      compressor.setInput(input, offset, length);
-      compressor.finish();
-
-      /*
-       * Create an expandable byte array to hold the compressed data.
-       * You cannot use an array that's the same size as the orginal because
-       * there is no guarantee that the compressed data will be smaller than
-       * the uncompressed data.
-       */
-      ByteArrayOutputStream bos = new ByteArrayOutputStream(length);
-
-      try {
-        compressor.setLevel(Deflater.BEST_COMPRESSION);
-
-        // Give the compressor the data to compress
-        compressor.setInput(input);
-        compressor.finish();
-
-        // Compress the data
-        byte[] buf = new byte[1024];
-        while (!compressor.finished()) {
-          int count = compressor.deflate(buf);
-          bos.write(buf, 0, count);
-        }
-
-      } finally {      
-        compressor.end();
-      }
-
-      // Get the compressed data
-      return bos.toByteArray();
-    }
 }

Modified: lucene/java/trunk/src/test/org/apache/lucene/document/TestBinaryDocument.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/document/TestBinaryDocument.java?rev=756635&r1=756634&r2=756635&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/document/TestBinaryDocument.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/document/TestBinaryDocument.java Fri Mar 20 17:13:42 2009
@@ -5,7 +5,7 @@
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.store.MockRAMDirectory;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -43,7 +43,7 @@
     Fieldable binaryFldCompressed = new Field("binaryCompressed", binaryValCompressed.getBytes(), Field.Store.COMPRESS);
     Fieldable stringFldStored = new Field("stringStored", binaryValStored, Field.Store.YES, Field.Index.NO, Field.TermVector.NO);
     Fieldable stringFldCompressed = new Field("stringCompressed", binaryValCompressed, Field.Store.COMPRESS, Field.Index.NO, Field.TermVector.NO);
-    
+
     try {
       // binary fields with store off are not allowed
       new Field("fail", binaryValCompressed.getBytes(), Field.Store.NO);
@@ -60,12 +60,12 @@
     
     doc.add(stringFldStored);
     doc.add(stringFldCompressed);
-    
+
     /** test for field count */
     assertEquals(4, doc.fields.size());
     
     /** add the doc to a ram index */
-    RAMDirectory dir = new RAMDirectory();
+    MockRAMDirectory dir = new MockRAMDirectory();
     IndexWriter writer = new IndexWriter(dir, new StandardAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
     writer.addDocument(doc);
     writer.close();
@@ -90,13 +90,40 @@
     /** fetch the compressed string field and compare it's content with the original one */
     String stringFldCompressedTest = docFromReader.get("stringCompressed");
     assertTrue(stringFldCompressedTest.equals(binaryValCompressed));
-    
+
     /** delete the document from index */
     reader.deleteDocument(0);
     assertEquals(0, reader.numDocs());
     
     reader.close();
-    
+    dir.close();
   }
   
+  public void testCompressionTools()
+    throws Exception
+  {
+    Fieldable binaryFldCompressed = new Field("binaryCompressed", CompressionTools.compress(binaryValCompressed.getBytes()), Field.Store.YES);
+    
+    Document doc = new Document();
+    
+    doc.add(binaryFldCompressed);
+    
+    /** add the doc to a ram index */
+    MockRAMDirectory dir = new MockRAMDirectory();
+    IndexWriter writer = new IndexWriter(dir, new StandardAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
+    writer.addDocument(doc);
+    writer.close();
+    
+    /** open a reader and fetch the document */ 
+    IndexReader reader = IndexReader.open(dir);
+    Document docFromReader = reader.document(0);
+    assertTrue(docFromReader != null);
+    
+    /** fetch the binary compressed field and compare it's content with the original one */
+    String binaryFldCompressedTest = new String(CompressionTools.decompress(docFromReader.getBinaryValue("binaryCompressed")));
+    assertTrue(binaryFldCompressedTest.equals(binaryValCompressed));
+    
+    reader.close();
+    dir.close();
+  }
 }