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 ca...@apache.org on 2017/05/28 02:08:45 UTC

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

Author: catholicon
Date: Sun May 28 02:08:45 2017
New Revision: 1796469

URL: http://svn.apache.org/viewvc?rev=1796469&view=rev
Log:
OAK-6270: There should be a way for editors to be notified by AsyncIndexUpdate about success/failure of indexing commit

Added:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexCommitCallback.java
      - copied, changed from r1796468, jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexingContext.java
Modified:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/AsyncIndexUpdate.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexUpdate.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexingContext.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/IndexUpdateTest.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorProviderTest.java

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/AsyncIndexUpdate.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/AsyncIndexUpdate.java?rev=1796469&r1=1796468&r2=1796469&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/AsyncIndexUpdate.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/AsyncIndexUpdate.java Sun May 28 02:08:45 2017
@@ -712,7 +712,7 @@ public class AsyncIndexUpdate implements
         // sure to not delete the reference checkpoint, as the other index
         // task will take care of it
         taskSplitter.maybeSplit(beforeCheckpoint, callback.lease);
-        IndexUpdate indexUpdate;
+        IndexUpdate indexUpdate = null;
         try {
             NodeBuilder builder = store.getRoot().builder();
 
@@ -777,6 +777,13 @@ public class AsyncIndexUpdate implements
 
             corruptIndexHandler.markWorkingIndexes(indexUpdate.getUpdatedIndexPaths());
         } finally {
+            if (indexUpdate != null) {
+                if (updatePostRunStatus) {
+                    indexUpdate.commitProgress(IndexCommitCallback.IndexProgress.COMMIT_SUCCEDED);
+                } else {
+                    indexUpdate.commitProgress(IndexCommitCallback.IndexProgress.COMMIT_FAILED);
+                }
+            }
             callback.close();
         }
 

Copied: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexCommitCallback.java (from r1796468, jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexingContext.java)
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexCommitCallback.java?p2=jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexCommitCallback.java&p1=jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexingContext.java&r1=1796468&r2=1796469&rev=1796469&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexingContext.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexCommitCallback.java Sun May 28 02:08:45 2017
@@ -19,36 +19,19 @@
 
 package org.apache.jackrabbit.oak.plugins.index;
 
-import org.apache.jackrabbit.oak.spi.commit.CommitInfo;
-
-public interface IndexingContext {
-
-    /**
-     * Path of the index definition in the repository
-     * @return index path in the repository
-     */
-    String getIndexPath();
-
-    /**
-     * Commit info associated with commit as part of which
-     * IndexEditor is being invoked
-     */
-    CommitInfo getCommitInfo();
-
-    /**
-     * Flag indicating that index is being reindex
-     */
-    boolean isReindexing();
-
-    /**
-     * Flag indicating that indexed is being done
-     * asynchronously
-     */
-    boolean isAsync();
-
-    /**
-     * Invoked by IndexEditor to indicate that update of index has failed
-     * @param e exception stack for failed updated
-     */
-    void indexUpdateFailed(Exception e);
+/**
+ * Implementations of this interface can be notified of progress of
+ * commit that would update the index. e.g. {@link AsyncIndexUpdate}
+ * notifies {@link IndexUpdate} about how commit progresses, which,
+ * in turn notifies registered callbacks (via
+ * {@link IndexingContext#registerIndexCommitCallback}).
+ */
+public interface IndexCommitCallback {
+    void commitProgress(IndexProgress indexProgress);
+
+    enum IndexProgress {
+        COMMIT_SUCCEDED,
+        COMMIT_FAILED,
+        ABORT_REQUESTED
+    }
 }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexUpdate.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexUpdate.java?rev=1796469&r1=1796468&r2=1796469&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexUpdate.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexUpdate.java Sun May 28 02:08:45 2017
@@ -20,6 +20,7 @@ import static com.google.common.base.Pre
 import static com.google.common.collect.Lists.newArrayList;
 import static com.google.common.collect.Lists.newArrayListWithCapacity;
 import static com.google.common.collect.Sets.newHashSet;
+import static com.google.common.collect.Sets.newIdentityHashSet;
 import static org.apache.jackrabbit.oak.api.Type.BOOLEAN;
 import static org.apache.jackrabbit.oak.commons.PathUtils.concat;
 import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.ASYNC_PROPERTY_NAME;
@@ -50,6 +51,7 @@ import org.apache.jackrabbit.oak.api.Com
 import org.apache.jackrabbit.oak.api.PropertyState;
 import org.apache.jackrabbit.oak.api.Type;
 import org.apache.jackrabbit.oak.commons.PathUtils;
+import org.apache.jackrabbit.oak.plugins.index.IndexCommitCallback.IndexProgress;
 import org.apache.jackrabbit.oak.plugins.index.NodeTraversalCallback.PathSource;
 import org.apache.jackrabbit.oak.plugins.index.progress.IndexingProgressReporter;
 import org.apache.jackrabbit.oak.plugins.index.progress.NodeCountEstimator;
@@ -417,6 +419,10 @@ public class IndexUpdate implements Edit
         return compose(children);
     }
 
+    public void commitProgress(IndexProgress indexProgress) {
+        rootState.commitProgress(indexProgress);
+    }
+
     protected Set<String> getReindexedDefinitions() {
         return rootState.progressReporter.getReindexedIndexPaths();
     }
@@ -501,6 +507,7 @@ public class IndexUpdate implements Edit
         final NodeState root;
         final CommitInfo commitInfo;
         private boolean ignoreReindexFlags = IGNORE_REINDEX_FLAGS;
+        final Set<IndexCommitCallback> indexCommitCallbacks = newIdentityHashSet();
         final CorruptIndexHandler corruptIndexHandler;
         final IndexingProgressReporter progressReporter;
         private int changedNodeCount;
@@ -549,6 +556,20 @@ public class IndexUpdate implements Edit
             this.ignoreReindexFlags = ignoreReindexFlags;
         }
 
+        void registerIndexCommitCallbackInternal(IndexCommitCallback callback) {
+            indexCommitCallbacks.add(callback);
+        }
+
+        public void commitProgress(IndexProgress indexProgress) {
+            for (IndexCommitCallback icc : indexCommitCallbacks) {
+                try {
+                    icc.commitProgress(indexProgress);
+                } catch (Exception e) {
+                    log.warn("Commit progress callback threw an exception. Saving ourselves.", e);
+                }
+            }
+        }
+
         private class ReportingCallback implements ContextAwareCallback, IndexingContext {
             final String indexPath;
             final boolean reindex;
@@ -596,6 +617,11 @@ public class IndexUpdate implements Edit
             public void indexUpdateFailed(Exception e) {
                 corruptIndexHandler.indexUpdateFailed(async, indexPath, e);
             }
+
+            @Override
+            public void registerIndexCommitCallback(IndexCommitCallback callback) {
+                registerIndexCommitCallbackInternal(callback);
+            }
         }
     }
 

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexingContext.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexingContext.java?rev=1796469&r1=1796468&r2=1796469&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexingContext.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexingContext.java Sun May 28 02:08:45 2017
@@ -51,4 +51,11 @@ public interface IndexingContext {
      * @param e exception stack for failed updated
      */
     void indexUpdateFailed(Exception e);
+
+    /**
+     * registers {@code IndexCommitCallback} instance which can then be
+     * notified of how indexing commit progresses.
+     * @param callback
+     */
+    void registerIndexCommitCallback(IndexCommitCallback callback);
 }

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/IndexUpdateTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/IndexUpdateTest.java?rev=1796469&r1=1796468&r2=1796469&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/IndexUpdateTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/IndexUpdateTest.java Sun May 28 02:08:45 2017
@@ -41,6 +41,7 @@ import static org.junit.Assert.fail;
 import java.util.Calendar;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.annotation.Nonnull;
 
@@ -722,6 +723,84 @@ public class IndexUpdateTest {
         assertFalse(provider.getContext(indexPath).isReindexing());
     }
 
+    @Test
+    public void indexUpdateToleratesMalignCommitProgressCallback() throws Exception {
+        final IndexUpdateCallback noop = new IndexUpdateCallback() {
+            @Override
+            public void indexUpdate() {
+            }
+        };
+
+        NodeState before = builder.getNodeState();
+        createIndexDefinition(builder.child(INDEX_DEFINITIONS_NAME),
+                "rootIndex", true, false, ImmutableSet.of("foo"), null);
+        NodeState after = builder.getNodeState();
+
+        CallbackCapturingProvider provider = new CallbackCapturingProvider();
+        IndexUpdate indexUpdate = new IndexUpdate(provider, null, after, builder,
+                noop);
+        indexUpdate.enter(before, after);
+
+        ContextAwareCallback contextualCallback = (ContextAwareCallback) provider.callback;
+        IndexingContext context = contextualCallback.getIndexingContext();
+
+        context.registerIndexCommitCallback(new IndexCommitCallback() {
+            @Override
+            public void commitProgress(IndexProgress indexProgress) {
+                throw new NullPointerException("Malign callback");
+            }
+        });
+
+        indexUpdate.commitProgress(IndexCommitCallback.IndexProgress.COMMIT_SUCCEDED);
+    }
+
+    @Test
+    public void commitProgressCallback() throws Exception {
+        final IndexUpdateCallback noop = new IndexUpdateCallback() {
+            @Override
+            public void indexUpdate() {
+            }
+        };
+
+        NodeState before = builder.getNodeState();
+        createIndexDefinition(builder.child(INDEX_DEFINITIONS_NAME),
+                "rootIndex", true, false, ImmutableSet.of("foo"), null);
+        NodeState after = builder.getNodeState();
+
+        CallbackCapturingProvider provider = new CallbackCapturingProvider();
+        IndexUpdate indexUpdate = new IndexUpdate(provider, null, after, builder,
+                noop);
+        indexUpdate.enter(before, after);
+
+        ContextAwareCallback contextualCallback = (ContextAwareCallback) provider.callback;
+        IndexingContext context = contextualCallback.getIndexingContext();
+
+        final AtomicInteger numCallbacks = new AtomicInteger();
+        IndexCommitCallback callback1 = new IndexCommitCallback() {
+            @Override
+            public void commitProgress(IndexProgress indexProgress) {
+                numCallbacks.incrementAndGet();
+            }
+        };
+        IndexCommitCallback callback2 = new IndexCommitCallback() {
+            @Override
+            public void commitProgress(IndexProgress indexProgress) {
+                numCallbacks.incrementAndGet();
+            }
+        };
+
+        context.registerIndexCommitCallback(callback1);
+        context.registerIndexCommitCallback(callback2);
+        context.registerIndexCommitCallback(callback1);//intentionally adding same one twice
+
+        for (IndexCommitCallback.IndexProgress progress : IndexCommitCallback.IndexProgress.values()) {
+            numCallbacks.set(0);
+            indexUpdate.commitProgress(IndexCommitCallback.IndexProgress.COMMIT_SUCCEDED);
+            assertEquals("Either not all callbacks are called OR same callback got called twice for " + progress,
+                    2, numCallbacks.get());
+        }
+    }
+
     private static void markCorrupt(NodeBuilder builder, String indexName) {
         builder.getChildNode(INDEX_DEFINITIONS_NAME).getChildNode(indexName)
                 .setProperty(IndexConstants.CORRUPT_PROPERTY_NAME, ISO8601.format(Calendar.getInstance()));

Modified: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorProviderTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorProviderTest.java?rev=1796469&r1=1796468&r2=1796469&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorProviderTest.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorProviderTest.java Sun May 28 02:08:45 2017
@@ -25,6 +25,7 @@ import org.apache.commons.lang3.reflect.
 import org.apache.jackrabbit.oak.api.CommitFailedException;
 import org.apache.jackrabbit.oak.core.SimpleCommitContext;
 import org.apache.jackrabbit.oak.plugins.index.ContextAwareCallback;
+import org.apache.jackrabbit.oak.plugins.index.IndexCommitCallback;
 import org.apache.jackrabbit.oak.plugins.index.IndexUpdateCallback;
 import org.apache.jackrabbit.oak.plugins.index.IndexingContext;
 import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.IndexingMode;
@@ -177,6 +178,11 @@ public class LuceneIndexEditorProviderTe
         public IndexingContext getIndexingContext() {
             return this;
         }
+
+        @Override
+        public void registerIndexCommitCallback(IndexCommitCallback callback) {
+
+        }
     }
 
 }
\ No newline at end of file