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 ch...@apache.org on 2017/07/19 06:15:44 UTC

svn commit: r1802358 - in /jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index: IndexCommand.java IndexHelper.java IndexerSupport.java OutOfBandIndexer.java

Author: chetanm
Date: Wed Jul 19 06:15:43 2017
New Revision: 1802358

URL: http://svn.apache.org/viewvc?rev=1802358&view=rev
Log:
OAK-6353 - Use Document order traversal for reindexing performed on DocumentNodeStore setups

Refactor the flow to move checkpoint and some common logic outside
of OutOfBandIndexer

Added:
    jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexerSupport.java   (with props)
Modified:
    jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexCommand.java
    jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexHelper.java
    jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/OutOfBandIndexer.java

Modified: jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexCommand.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexCommand.java?rev=1802358&r1=1802357&r2=1802358&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexCommand.java (original)
+++ jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexCommand.java Wed Jul 19 06:15:43 2017
@@ -38,7 +38,6 @@ import org.apache.jackrabbit.oak.run.cli
 import org.apache.jackrabbit.oak.run.cli.Options;
 import org.apache.jackrabbit.oak.run.commons.Command;
 import org.apache.jackrabbit.util.ISO8601;
-import org.jetbrains.annotations.NotNull;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -123,11 +122,10 @@ public class IndexCommand implements Com
         dumpIndexDefinitions(indexOpts, indexHelper);
         performConsistencyCheck(indexOpts, indexHelper);
         dumpIndexContents(indexOpts, indexHelper);
-        reindexIndex(indexOpts, indexHelper);
-        importIndex(indexOpts, indexHelper);
+        reindexOperation(indexOpts, indexHelper);
+        importIndexOperation(indexOpts, indexHelper);
     }
 
-    @NotNull
     private IndexHelper createIndexHelper(NodeStoreFixture fixture,
                                           IndexOptions indexOpts, Closer closer) throws IOException {
         IndexHelper indexHelper = new IndexHelper(fixture.getStore(), fixture.getBlobStore(), fixture.getWhiteboard(),
@@ -147,31 +145,43 @@ public class IndexCommand implements Com
         }
     }
 
-    private void reindexIndex(IndexOptions indexOpts, IndexHelper indexHelper) throws IOException, CommitFailedException {
+    private void reindexOperation(IndexOptions indexOpts, IndexHelper indexHelper) throws IOException, CommitFailedException {
         if (!indexOpts.isReindex()){
             return;
         }
 
         String checkpoint = indexOpts.getCheckpoint();
-        reindex(indexHelper, checkpoint);
+        File destDir = reindex(indexHelper, checkpoint);
+        log.info("To complete indexing import the created index files via IndexerMBean#importIndex operation with " +
+                "[{}] as input", getPath(destDir));
     }
 
-    private void reindex(IndexHelper indexHelper, String checkpoint) throws IOException, CommitFailedException {
-        checkNotNull(checkpoint, "Checkpoint value is required for reindexing done in read only mode");
-        try (OutOfBandIndexer indexer = new OutOfBandIndexer(indexHelper, checkpoint)) {
-            File destDir = indexer.reindex();
-            log.info("To complete indexing import the created index files via IndexerMBean#importIndex operation with " +
-                    "[{}] as input", getPath(destDir));
-        }
-    }
-
-    private void importIndex(IndexOptions indexOpts, IndexHelper indexHelper) throws IOException, CommitFailedException {
+    private void importIndexOperation(IndexOptions indexOpts, IndexHelper indexHelper) throws IOException, CommitFailedException {
         if (indexOpts.isImportIndex()) {
             File importDir = indexOpts.getIndexImportDir();
             importIndex(indexHelper, importDir);
         }
     }
 
+    private File reindex(IndexHelper indexHelper, String checkpoint) throws IOException, CommitFailedException {
+        checkNotNull(checkpoint, "Checkpoint value is required for reindexing done in read only mode");
+
+        Stopwatch w = Stopwatch.createStarted();
+        IndexerSupport indexerSupport = new IndexerSupport(indexHelper, checkpoint);
+        log.info("Proceeding to index {} upto checkpoint {} {}", indexHelper.getIndexPaths(), checkpoint,
+                indexerSupport.getCheckpointInfo());
+
+        try (OutOfBandIndexer indexer = new OutOfBandIndexer(indexHelper, indexerSupport)) {
+            indexer.reindex();
+        }
+
+        indexerSupport.writeMetaInfo(checkpoint);
+        File destDir = indexerSupport.copyIndexFilesToOutput();
+        log.info("Indexing completed for indexes {} in {} ({} ms) and index files are copied to {}",
+                indexHelper.getIndexPaths(), w, w.elapsed(TimeUnit.MILLISECONDS), IndexCommand.getPath(destDir));
+        return destDir;
+    }
+
     private void importIndex(IndexHelper indexHelper, File importDir) throws IOException, CommitFailedException {
         new IndexImporterSupport(indexHelper).importIndex(importDir);
     }

Modified: jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexHelper.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexHelper.java?rev=1802358&r1=1802357&r2=1802358&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexHelper.java (original)
+++ jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexHelper.java Wed Jul 19 06:15:43 2017
@@ -167,6 +167,11 @@ public class IndexHelper implements Clos
         getExtractedTextCache().setExtractedTextProvider(new DataStoreTextWriter(dir, true));
     }
 
+    @CheckForNull
+    public <T> T getService(@Nonnull Class<T> type) {
+        return WhiteboardUtils.getService(whiteboard, type);
+    }
+
     @Override
     public void close() throws IOException {
         closer.close();

Added: jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexerSupport.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexerSupport.java?rev=1802358&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexerSupport.java (added)
+++ jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexerSupport.java Wed Jul 19 06:15:43 2017
@@ -0,0 +1,91 @@
+/*
+ * 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.
+ */
+
+package org.apache.jackrabbit.oak.index;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.jackrabbit.oak.plugins.index.importer.IndexerInfo;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class IndexerSupport {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+    /**
+     * Directory name in output directory under which indexes are
+     * stored
+     */
+    public static final String LOCAL_INDEX_ROOT_DIR = "indexes";
+    /**
+     * Checkpoint value which indicate that head state needs to be used
+     * This would be mostly used for testing purpose
+     */
+    private static final String HEAD_AS_CHECKPOINT = "head";
+    private Map<String, String> checkpointInfo = Collections.emptyMap();
+    private final IndexHelper indexHelper;
+    private File localIndexDir;
+    private String checkpoint;
+
+    public IndexerSupport(IndexHelper indexHelper, String checkpoint) {
+        this.indexHelper = indexHelper;
+        this.checkpoint = checkpoint;
+    }
+
+    public File getLocalIndexDir() throws IOException {
+        if (localIndexDir == null) {
+            localIndexDir = new File(indexHelper.getWorkDir(), LOCAL_INDEX_ROOT_DIR);
+            FileUtils.forceMkdir(localIndexDir);
+        }
+        return localIndexDir;
+    }
+
+    public File copyIndexFilesToOutput() throws IOException {
+        File destDir = new File(indexHelper.getOutputDir(), getLocalIndexDir().getName());
+        FileUtils.moveDirectoryToDirectory(getLocalIndexDir(), indexHelper.getOutputDir(), true);
+        return destDir;
+    }
+
+    public void writeMetaInfo(String checkpoint) throws IOException {
+        new IndexerInfo(getLocalIndexDir(), checkpoint).save();
+    }
+
+    public NodeState retrieveNodeStateForCheckpoint() {
+        NodeState checkpointedState;
+        if (HEAD_AS_CHECKPOINT.equals(checkpoint)) {
+            checkpointedState = indexHelper.getNodeStore().getRoot();
+            log.warn("Using head state for indexing. Such an index cannot be imported back");
+        } else {
+            checkpointedState = indexHelper.getNodeStore().retrieve(checkpoint);
+            checkNotNull(checkpointedState, "Not able to retrieve revision referred via checkpoint [%s]", checkpoint);
+            checkpointInfo = indexHelper.getNodeStore().checkpointInfo(checkpoint);
+        }
+        return checkpointedState;
+    }
+
+    public Map<String, String> getCheckpointInfo() {
+        return checkpointInfo;
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/IndexerSupport.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/OutOfBandIndexer.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/OutOfBandIndexer.java?rev=1802358&r1=1802357&r2=1802358&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/OutOfBandIndexer.java (original)
+++ jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/OutOfBandIndexer.java Wed Jul 19 06:15:43 2017
@@ -22,19 +22,10 @@ package org.apache.jackrabbit.oak.index;
 import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
-import java.io.OutputStream;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 
 import com.codahale.metrics.MetricRegistry;
-import com.google.common.base.Stopwatch;
 import com.google.common.io.Closer;
-import org.apache.commons.io.FileUtils;
 import org.apache.jackrabbit.oak.api.CommitFailedException;
-import org.apache.jackrabbit.oak.api.PropertyState;
-import org.apache.jackrabbit.oak.api.Type;
 import org.apache.jackrabbit.oak.plugins.index.CompositeIndexEditorProvider;
 import org.apache.jackrabbit.oak.plugins.index.CorruptIndexHandler;
 import org.apache.jackrabbit.oak.plugins.index.IndexConstants;
@@ -43,14 +34,12 @@ import org.apache.jackrabbit.oak.plugins
 import org.apache.jackrabbit.oak.plugins.index.IndexUpdateCallback;
 import org.apache.jackrabbit.oak.plugins.index.NodeTraversalCallback;
 import org.apache.jackrabbit.oak.plugins.index.importer.AsyncLaneSwitcher;
-import org.apache.jackrabbit.oak.plugins.index.importer.IndexerInfo;
 import org.apache.jackrabbit.oak.plugins.index.lucene.directory.DirectoryFactory;
 import org.apache.jackrabbit.oak.plugins.index.lucene.directory.FSDirectoryFactory;
 import org.apache.jackrabbit.oak.plugins.index.progress.MetricRateEstimator;
 import org.apache.jackrabbit.oak.plugins.index.progress.NodeCounterMBeanEstimator;
 import org.apache.jackrabbit.oak.plugins.index.property.PropertyIndexEditorProvider;
 import org.apache.jackrabbit.oak.plugins.memory.MemoryNodeStore;
-import org.apache.jackrabbit.oak.plugins.memory.PropertyStates;
 import org.apache.jackrabbit.oak.plugins.metric.MetricStatisticsProvider;
 import org.apache.jackrabbit.oak.spi.commit.CommitInfo;
 import org.apache.jackrabbit.oak.spi.commit.EditorDiff;
@@ -65,7 +54,6 @@ import org.slf4j.LoggerFactory;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 import static java.util.Arrays.asList;
-import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.ASYNC_PROPERTY_NAME;
 
 public class OutOfBandIndexer implements Closeable, IndexUpdateCallback, NodeTraversalCallback {
     private final Logger log = LoggerFactory.getLogger(getClass());
@@ -74,15 +62,6 @@ public class OutOfBandIndexer implements
      */
     private static final String REINDEX_LANE = "offline-reindex-async";
     /**
-     * Property name where previous value of 'async' is stored
-     */
-    private static final String ASYNC_PREVIOUS = "async-previous";
-    /**
-     * Value stored in previous async property if the index is not async
-     * i.e. when a sync index is reindexed in out of band mode
-     */
-    private static final String ASYNC_PREVIOUS_NONE = "none";
-    /**
      * Directory name in output directory under which indexes are
      * stored
      */
@@ -96,46 +75,30 @@ public class OutOfBandIndexer implements
 
     private final Closer closer = Closer.create();
     private final IndexHelper indexHelper;
-    private final String checkpoint;
-    private Map<String, String> checkpointInfo = Collections.emptyMap();
     private NodeStore copyOnWriteStore;
-    private File localIndexDir;
+    private IndexerSupport indexerSupport;
 
     //TODO Support for providing custom index definition i.e. where definition is not
     //present in target repository
 
-    public OutOfBandIndexer(IndexHelper indexHelper, String checkpoint) {
+    public OutOfBandIndexer(IndexHelper indexHelper, IndexerSupport indexerSupport) {
         this.indexHelper = checkNotNull(indexHelper);
-        this.checkpoint = checkNotNull(checkpoint);
+        this.indexerSupport = checkNotNull(indexerSupport);
     }
 
-    public File reindex() throws CommitFailedException, IOException {
-        Stopwatch w = Stopwatch.createStarted();
-
-        NodeState checkpointedState = retrieveNodeStateForCheckpoint();
+    public void reindex() throws CommitFailedException, IOException {
+        NodeState checkpointedState = indexerSupport.retrieveNodeStateForCheckpoint();
 
         copyOnWriteStore = new MemoryNodeStore(checkpointedState);
         NodeState baseState = copyOnWriteStore.getRoot();
         //TODO Check for indexPaths being empty
 
-        log.info("Proceeding to index {} upto checkpoint {} {}", indexHelper.getIndexPaths(), checkpoint, checkpointInfo);
-
         switchIndexLanesAndReindexFlag();
         preformIndexUpdate(baseState);
-        writeMetaInfo();
-        File destDir = copyIndexFilesToOutput();
-
-        log.info("Indexing completed for indexes {} in {} ({} ms) and index files are copied to {}",
-                indexHelper.getIndexPaths(), w, w.elapsed(TimeUnit.MILLISECONDS), IndexCommand.getPath(destDir));
-        return destDir;
     }
 
     private File getLocalIndexDir() throws IOException {
-        if (localIndexDir == null) {
-            localIndexDir = new File(indexHelper.getWorkDir(), LOCAL_INDEX_ROOT_DIR);
-            FileUtils.forceMkdir(localIndexDir);
-        }
-        return localIndexDir;
+        return indexerSupport.getLocalIndexDir();
     }
 
     @Override
@@ -214,29 +177,6 @@ public class OutOfBandIndexer implements
         log.info("Switched the async lane for indexes at {} to {} and marked them for reindex", indexHelper.getIndexPaths(), REINDEX_LANE);
     }
 
-    private NodeState retrieveNodeStateForCheckpoint() {
-        NodeState checkpointedState;
-        if (HEAD_AS_CHECKPOINT.equals(checkpoint)) {
-            checkpointedState = indexHelper.getNodeStore().getRoot();
-            log.warn("Using head state for indexing. Such an index cannot be imported back");
-        } else {
-            checkpointedState = indexHelper.getNodeStore().retrieve(checkpoint);
-            checkNotNull(checkpointedState, "Not able to retrieve revision referred via checkpoint [%s]", checkpoint);
-            checkpointInfo = indexHelper.getNodeStore().checkpointInfo(checkpoint);
-        }
-        return checkpointedState;
-    }
-
-    private void writeMetaInfo() throws IOException {
-        new IndexerInfo(getLocalIndexDir(), checkpoint).save();
-    }
-
-    private File copyIndexFilesToOutput() throws IOException {
-        File destDir = new File(indexHelper.getOutputDir(), getLocalIndexDir().getName());
-        FileUtils.moveDirectoryToDirectory(getLocalIndexDir(), indexHelper.getOutputDir(), true);
-        return destDir;
-    }
-
     private void configureEstimators(IndexUpdate indexUpdate) {
         StatisticsProvider statsProvider = indexHelper.getStatisticsProvider();
         if (statsProvider instanceof MetricStatisticsProvider) {