You are viewing a plain text version of this content. The canonical link for it is here.
Posted to oak-commits@jackrabbit.apache.org by th...@apache.org on 2015/09/24 15:36:21 UTC

svn commit: r1705054 - in /jackrabbit/oak/trunk/oak-lucene/src: main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ test/java/org/apache/jackrabbit/oak/plugins/index/lucene/

Author: thomasm
Date: Thu Sep 24 13:36:20 2015
New Revision: 1705054

URL: http://svn.apache.org/viewvc?rev=1705054&view=rev
Log:
OAK-2808 Active deletion of 'deleted' Lucene index files from DataStore without relying on full scale Blob GC (WIP)

Modified:
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinition.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexNode.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorContext.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectory.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/package-info.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneBlobCacheTest.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectoryTest.java

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinition.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinition.java?rev=1705054&r1=1705053&r2=1705054&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinition.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinition.java Thu Sep 24 13:36:20 2015
@@ -79,6 +79,7 @@ import static org.apache.jackrabbit.oak.
 import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.DECLARING_NODE_TYPES;
 import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.ENTRY_COUNT_PROPERTY_NAME;
 import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.REINDEX_COUNT;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.ACTIVE_DELETE;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.BLOB_SIZE;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.COMPAT_MODE;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.EVALUATE_PATH_RESTRICTION;
@@ -109,6 +110,12 @@ class IndexDefinition implements Aggrega
     private static final Logger log = LoggerFactory.getLogger(IndexDefinition.class);
 
     /**
+     * Default number of seconds after which to delete actively. Default is -1, meaning disabled.
+     * The plan is to use 3600 (1 hour) in the future.
+     */
+    static final int DEFAULT_ACTIVE_DELETE = -1; // 60 * 60;
+
+    /**
      * Blob size to use by default. To avoid issues in OAK-2105 the size should not
      * be power of 2.
      */
@@ -151,6 +158,8 @@ class IndexDefinition implements Aggrega
 
     private final String funcName;
 
+    private final int activeDelete;
+    
     private final int blobSize;
 
     private final Codec codec;
@@ -227,6 +236,7 @@ class IndexDefinition implements Aggrega
         this.definition = defn;
         this.indexName = determineIndexName(defn, indexPath);
         this.blobSize = getOptionalValue(defn, BLOB_SIZE, DEFAULT_BLOB_SIZE);
+        this.activeDelete = getOptionalValue(defn, ACTIVE_DELETE, DEFAULT_ACTIVE_DELETE);
         this.testMode = getOptionalValue(defn, LuceneIndexConstants.TEST_MODE, false);
 
         this.aggregates = collectAggregates(defn);
@@ -1395,4 +1405,8 @@ class IndexDefinition implements Aggrega
         return 0;
     }
 
+    public boolean getActiveDeleteEnabled() {
+        return activeDelete >= 0;
+    }
+
 }

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexNode.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexNode.java?rev=1705054&r1=1705053&r2=1705054&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexNode.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexNode.java Thu Sep 24 13:36:20 2015
@@ -46,7 +46,7 @@ class IndexNode {
         IndexDefinition definition = new IndexDefinition(root, defnNodeState, indexPath);
         NodeState data = defnNodeState.getChildNode(INDEX_DATA_CHILD_NAME);
         if (data.exists()) {
-            directory = new OakDirectory(new ReadOnlyBuilder(data), definition, true);
+            directory = new OakDirectory(new ReadOnlyBuilder(defnNodeState), definition, true);
             if (cloner != null){
                 directory = cloner.wrapForRead(indexPath, definition, directory);
             }

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java?rev=1705054&r1=1705053&r2=1705054&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java Thu Sep 24 13:36:20 2015
@@ -26,6 +26,8 @@ public interface LuceneIndexConstants {
 
     String INDEX_DATA_CHILD_NAME = ":data";
 
+    String TRASH_CHILD_NAME = ":trash";
+
     Version VERSION = Version.LUCENE_47;
 
     Analyzer ANALYZER = new OakAnalyzer(VERSION);
@@ -81,6 +83,11 @@ public interface LuceneIndexConstants {
     String ORDERED_PROP_NAMES = "orderedProps";
 
     /**
+     * Actively the data store files after this many hours.
+     */
+    String ACTIVE_DELETE = "activeDelete";
+
+    /**
      * Size in bytes used for splitting the index files when storing them in NodeStore
      */
     String BLOB_SIZE = "blobSize";

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorContext.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorContext.java?rev=1705054&r1=1705053&r2=1705054&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorContext.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorContext.java Thu Sep 24 13:36:20 2015
@@ -17,7 +17,6 @@
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
 import static org.apache.jackrabbit.oak.commons.IOUtils.humanReadableByteCount;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.INDEX_DATA_CHILD_NAME;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.PERSISTENCE_PATH;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.VERSION;
 import static org.apache.lucene.store.NoLockFactory.getNoLockFactory;
@@ -96,7 +95,7 @@ public class LuceneIndexEditorContext {
             throws IOException {
         String path = definition.getString(PERSISTENCE_PATH);
         if (path == null) {
-            return new OakDirectory(definition.child(INDEX_DATA_CHILD_NAME), indexDefinition, false);
+            return new OakDirectory(definition, indexDefinition, false);
         } else {
             // try {
             File file = new File(path);

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectory.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectory.java?rev=1705054&r1=1705053&r2=1705054&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectory.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectory.java Thu Sep 24 13:36:20 2015
@@ -20,6 +20,9 @@ import java.io.ByteArrayInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.SequenceInputStream;
+import java.security.SecureRandom;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
@@ -32,6 +35,7 @@ import com.google.common.primitives.Ints
 import org.apache.jackrabbit.oak.api.Blob;
 import org.apache.jackrabbit.oak.api.PropertyState;
 import org.apache.jackrabbit.oak.api.Type;
+import org.apache.jackrabbit.oak.commons.StringUtils;
 import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
 import org.apache.jackrabbit.oak.util.PerfLogger;
 import org.apache.lucene.store.AlreadyClosedException;
@@ -55,6 +59,7 @@ import static org.apache.jackrabbit.JcrC
 import static org.apache.jackrabbit.JcrConstants.JCR_LASTMODIFIED;
 import static org.apache.jackrabbit.oak.api.Type.BINARIES;
 import static org.apache.jackrabbit.oak.api.Type.STRINGS;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.INDEX_DATA_CHILD_NAME;
 import static org.apache.jackrabbit.oak.plugins.memory.PropertyStates.createProperty;
 
 /**
@@ -65,18 +70,27 @@ class OakDirectory extends Directory {
     static final PerfLogger PERF_LOGGER = new PerfLogger(LoggerFactory.getLogger(OakDirectory.class.getName() + ".perf"));
     static final String PROP_DIR_LISTING = "dirListing";
     static final String PROP_BLOB_SIZE = "blobSize";
+    static final String PROP_UNIQUE_KEY = "uniqueKey";
+    static final int UNIQUE_KEY_SIZE = 16;
+    
+    private final static SecureRandom secureRandom = new SecureRandom();
+    
+    protected final NodeBuilder builder;
     protected final NodeBuilder directoryBuilder;
     private final IndexDefinition definition;
     private LockFactory lockFactory;
     private final boolean readOnly;
     private final Set<String> fileNames = Sets.newConcurrentHashSet();
+    private final boolean activeDeleteEnabled;
 
-    public OakDirectory(NodeBuilder directoryBuilder, IndexDefinition definition, boolean readOnly) {
+    public OakDirectory(NodeBuilder builder, IndexDefinition definition, boolean readOnly) {
         this.lockFactory = NoLockFactory.getNoLockFactory();
-        this.directoryBuilder = directoryBuilder;
+        this.builder = builder;
+        this.directoryBuilder = builder.child(INDEX_DATA_CHILD_NAME);
         this.definition = definition;
         this.readOnly = readOnly;
         this.fileNames.addAll(getListing());
+        this.activeDeleteEnabled = definition.getActiveDeleteEnabled();
     }
 
     @Override
@@ -93,7 +107,29 @@ class OakDirectory extends Directory {
     public void deleteFile(String name) throws IOException {
         checkArgument(!readOnly, "Read only directory");
         fileNames.remove(name);
-        directoryBuilder.getChildNode(name).remove();
+        NodeBuilder f = directoryBuilder.getChildNode(name);
+        if (activeDeleteEnabled) {
+            PropertyState property = f.getProperty(JCR_DATA);
+            ArrayList<Blob> data;
+            if (property != null && property.getType() == BINARIES) {
+                data = newArrayList(property.getValue(BINARIES));
+            } else {
+                data = newArrayList();
+            }
+            NodeBuilder trash = builder.child(LuceneIndexConstants.TRASH_CHILD_NAME);
+            long index;
+            if (!trash.hasProperty("index")) {
+                index = 1;
+            } else {    
+                index = trash.getProperty("index").getValue(Type.LONG) + 1;                
+            }
+            trash.setProperty("index", index);
+            NodeBuilder trashEntry = trash.child("run_" + index);
+            trashEntry.setProperty("time", System.currentTimeMillis());
+            trashEntry.setProperty("name", name);
+            trashEntry.setProperty(JCR_DATA, data, BINARIES);
+        }
+        f.remove();
     }
 
     @Override
@@ -114,6 +150,10 @@ class OakDirectory extends Directory {
         NodeBuilder file;
         if (!directoryBuilder.hasChildNode(name)) {
             file = directoryBuilder.child(name);
+            byte[] uniqueKey = new byte[UNIQUE_KEY_SIZE];
+            secureRandom.nextBytes(uniqueKey);
+            String key = StringUtils.convertBytesToHex(uniqueKey);
+            file.setProperty(PROP_UNIQUE_KEY, key);
             file.setProperty(PROP_BLOB_SIZE, definition.getBlobSize());
         } else {
             file = directoryBuilder.child(name);
@@ -248,6 +288,11 @@ class OakDirectory extends Directory {
          * The data of the currently loaded blob.
          */
         private byte[] blob;
+        
+        /**
+         * The unique key that is used to make the content unique (to allow removing binaries from the blob store without risking to remove binaries that are still needed).
+         */
+        private final byte[] uniqueKey;
 
         /**
          * Whether the currently loaded blob was modified since the blob was
@@ -259,6 +304,7 @@ class OakDirectory extends Directory {
             this.name = name;
             this.file = file;
             this.blobSize = determineBlobSize(file);
+            this.uniqueKey = readUniqueKey(file);
             this.blob = new byte[blobSize];
 
             PropertyState property = file.getProperty(JCR_DATA);
@@ -272,6 +318,9 @@ class OakDirectory extends Directory {
             if (!data.isEmpty()) {
                 Blob last = data.get(data.size() - 1);
                 this.length -= blobSize - last.length();
+                if (uniqueKey != null) {
+                    this.length -= uniqueKey.length;
+                }
             }
         }
 
@@ -279,6 +328,7 @@ class OakDirectory extends Directory {
             this.name = that.name;
             this.file = that.file;
             this.blobSize = that.blobSize;
+            this.uniqueKey = that.uniqueKey;
             this.blob = new byte[blobSize];
 
             this.position = that.position;
@@ -307,7 +357,12 @@ class OakDirectory extends Directory {
         private void flushBlob() throws IOException {
             if (blobModified) {
                 int n = (int) Math.min(blobSize, length - index * blobSize);
-                Blob b = file.createBlob(new ByteArrayInputStream(blob, 0, n));
+                InputStream in = new ByteArrayInputStream(blob, 0, n);
+                if (uniqueKey != null) {
+                    in = new SequenceInputStream(in, 
+                            new ByteArrayInputStream(uniqueKey));
+                }
+                Blob b = file.createBlob(in);
                 if (index < data.size()) {
                     data.set(index, b);
                 } else {
@@ -391,6 +446,14 @@ class OakDirectory extends Directory {
             return DEFAULT_BLOB_SIZE;
         }
 
+        private static byte[] readUniqueKey(NodeBuilder file) {
+            if (file.hasProperty(PROP_UNIQUE_KEY)) {
+                String key = file.getString(PROP_UNIQUE_KEY);
+                return StringUtils.convertHexToBytes(key);
+            }
+            return null;
+        }
+
         public void flush() throws IOException {
             flushBlob();
             if (dataModified) {

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/package-info.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/package-info.java?rev=1705054&r1=1705053&r2=1705054&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/package-info.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/package-info.java Thu Sep 24 13:36:20 2015
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-@Version("2.4.0")
+@Version("2.5.0")
 @Export(optional = "provide:=true")
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 

Modified: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneBlobCacheTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneBlobCacheTest.java?rev=1705054&r1=1705053&r2=1705054&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneBlobCacheTest.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneBlobCacheTest.java Thu Sep 24 13:36:20 2015
@@ -110,7 +110,7 @@ public class LuceneBlobCacheTest {
     }
 
     private Directory createDir(NodeBuilder builder, boolean readOnly){
-        return new OakDirectory(builder.child(INDEX_DATA_CHILD_NAME),
+        return new OakDirectory(builder,
                 new IndexDefinition(root, builder.getNodeState()), readOnly);
     }
 

Modified: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectoryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectoryTest.java?rev=1705054&r1=1705053&r2=1705054&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectoryTest.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectoryTest.java Thu Sep 24 13:36:20 2015
@@ -147,7 +147,7 @@ public class OakDirectoryTest {
         assertEquals(blobSize, testNode.getProperty(PROP_BLOB_SIZE).getValue(Type.LONG).longValue());
 
         List<Blob> blobs = newArrayList(testNode.getProperty(JCR_DATA).getValue(BINARIES));
-        assertEquals(blobSize, blobs.get(0).length());
+        assertEquals(blobSize + OakDirectory.UNIQUE_KEY_SIZE, blobs.get(0).length());
 
         return data;
     }
@@ -162,7 +162,7 @@ public class OakDirectoryTest {
     }
 
     private Directory createDir(NodeBuilder builder, boolean readOnly){
-        return new OakDirectory(builder.child(INDEX_DATA_CHILD_NAME),
+        return new OakDirectory(builder,
                 new IndexDefinition(root, builder.getNodeState()), readOnly);
     }