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/06/27 11:47:26 UTC

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

Author: chetanm
Date: Tue Jun 27 11:47:25 2017
New Revision: 1800052

URL: http://svn.apache.org/viewvc?rev=1800052&view=rev
Log:
OAK-6271 - Support for importing index files

Initial implementation

Added:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncIndexerLock.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncLaneSwitcher.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/ClusterNodeStoreLock.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporter.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporterProvider.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexerInfo.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/NodeStoreUtils.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/PropUtils.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncLaneSwitcherTest.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/ClusterNodeStoreLockTest.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporterTest.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexerInfoTest.java   (with props)
Modified:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/AsyncIndexUpdate.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=1800052&r1=1800051&r2=1800052&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 Tue Jun 27 11:47:25 2017
@@ -810,7 +810,7 @@ public class AsyncIndexUpdate implements
         indexUpdate.setNodeCountEstimator(estimator);
     }
 
-    static String leasify(String name) {
+    public static String leasify(String name) {
         return name + "-lease";
     }
 

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncIndexerLock.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncIndexerLock.java?rev=1800052&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncIndexerLock.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncIndexerLock.java Tue Jun 27 11:47:25 2017
@@ -0,0 +1,56 @@
+/*
+ * 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.plugins.index.importer;
+
+import org.apache.jackrabbit.oak.api.CommitFailedException;
+
+/**
+ * Lock used to prevent AsyncIndexUpdate from running when Index import
+ * is in progress
+ */
+interface AsyncIndexerLock<T extends AsyncIndexerLock.LockToken> {
+
+    AsyncIndexerLock NOOP_LOCK = new AsyncIndexerLock() {
+        private final LockToken noopToken = new LockToken() {};
+
+        @Override
+        public LockToken lock(String asyncIndexerLane) {
+            return noopToken;
+        }
+
+        @Override
+        public void unlock(LockToken token) {
+
+        }
+    };
+
+    /**
+     * Marker interface. The implementation can return any
+     * opaque implementation which would be handed back in unlock
+     * call
+     */
+    interface LockToken {
+
+    }
+
+    T lock(String asyncIndexerLane) throws CommitFailedException;
+
+    void unlock(T token) throws CommitFailedException;
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncIndexerLock.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncLaneSwitcher.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncLaneSwitcher.java?rev=1800052&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncLaneSwitcher.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncLaneSwitcher.java Tue Jun 27 11:47:25 2017
@@ -0,0 +1,105 @@
+/*
+ * 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.plugins.index.importer;
+
+import org.apache.jackrabbit.oak.api.PropertyState;
+import org.apache.jackrabbit.oak.api.Type;
+import org.apache.jackrabbit.oak.plugins.index.IndexConstants;
+import org.apache.jackrabbit.oak.plugins.memory.PropertyStates;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+
+import static com.google.common.base.Preconditions.checkState;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.ASYNC_PROPERTY_NAME;
+
+/**
+ * Coordinates the switching of indexing lane for indexes which are
+ * to be imported. Its support idempotent operation i.e. if an
+ * indexer is switched to temp lane then a repeat of same
+ * operation would be no op.
+ */
+public class AsyncLaneSwitcher {
+    /**
+     * Property name where previous value of 'async' is stored
+     */
+    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
+     */
+    static final String ASYNC_PREVIOUS_NONE = "none";
+
+    /**
+     * Index lane name which is used for indexing
+     */
+    private static final String TEMP_LANE_PREFIX = "temp-";
+
+    /**
+     * Make a copy of current async value and replace it with one required for offline reindexing
+     * The switch lane operation can be safely repeated and if the index
+     * lane is found to be switched already it would not be modified
+     */
+    public static void switchLane(NodeBuilder idxBuilder, String laneName) {
+        PropertyState currentAsyncState = idxBuilder.getProperty(ASYNC_PROPERTY_NAME);
+        PropertyState newAsyncState = PropertyStates.createProperty(ASYNC_PROPERTY_NAME, laneName, Type.STRING);
+
+        if (idxBuilder.hasProperty(ASYNC_PREVIOUS)){
+            //Lane already switched
+            return;
+        }
+
+        PropertyState previousAsyncState;
+        if (currentAsyncState == null) {
+            previousAsyncState = PropertyStates.createProperty(ASYNC_PREVIOUS, ASYNC_PREVIOUS_NONE);
+        } else {
+            //Ensure that previous state is copied with correct type
+            previousAsyncState = clone(ASYNC_PREVIOUS, currentAsyncState);
+        }
+
+        idxBuilder.setProperty(previousAsyncState);
+        idxBuilder.setProperty(newAsyncState);
+    }
+
+    public static String getTempLaneName(String laneName){
+        return TEMP_LANE_PREFIX + laneName;
+    }
+
+    public static void revertSwitch(NodeBuilder idxBuilder, String indexPath) {
+        PropertyState previousAsync = idxBuilder.getProperty(ASYNC_PREVIOUS);
+        checkState(previousAsync != null, "No previous async state property found for index [%s]", indexPath);
+
+        if (ASYNC_PREVIOUS_NONE.equals(previousAsync.getValue(Type.STRING))) {
+            idxBuilder.removeProperty(IndexConstants.ASYNC_PROPERTY_NAME);
+        } else {
+            idxBuilder.setProperty(clone(IndexConstants.ASYNC_PROPERTY_NAME, previousAsync));
+        }
+        idxBuilder.removeProperty(ASYNC_PREVIOUS);
+    }
+
+    private static PropertyState clone(String newName, PropertyState currentAsyncState) {
+        PropertyState clonedState;
+        if (currentAsyncState.isArray()) {
+            clonedState = PropertyStates.createProperty(newName, currentAsyncState.getValue(Type.STRINGS), Type.STRINGS);
+        } else {
+            clonedState = PropertyStates.createProperty(newName, currentAsyncState.getValue(Type.STRING), Type.STRING);
+        }
+        return clonedState;
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncLaneSwitcher.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/ClusterNodeStoreLock.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/ClusterNodeStoreLock.java?rev=1800052&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/ClusterNodeStoreLock.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/ClusterNodeStoreLock.java Tue Jun 27 11:47:25 2017
@@ -0,0 +1,98 @@
+/*
+ * 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.plugins.index.importer;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.jackrabbit.oak.api.CommitFailedException;
+import org.apache.jackrabbit.oak.plugins.index.AsyncIndexUpdate;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeStore;
+import org.apache.jackrabbit.oak.stats.Clock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Lock implementation for clustered scenario. The locking is done
+ * by setting the lease time for the lane to distant future which
+ * prevent AsyncIndexUpdate from  running.
+ */
+public class ClusterNodeStoreLock implements AsyncIndexerLock<ClusteredLockToken> {
+    /**
+     * Use a looong lease time to ensure that async indexer does not start
+     * in between the import process which can take some time
+     */
+    private static final long LOCK_TIMEOUT = TimeUnit.DAYS.toMillis(100);
+    private final Logger log = LoggerFactory.getLogger(getClass());
+    private final NodeStore nodeStore;
+    private final Clock clock;
+
+    public ClusterNodeStoreLock(NodeStore nodeStore){
+        this(nodeStore, Clock.SIMPLE);
+    }
+
+    public ClusterNodeStoreLock(NodeStore nodeStore, Clock clock) {
+        this.nodeStore = nodeStore;
+        this.clock = clock;
+    }
+
+    @Override
+    public ClusteredLockToken lock(String asyncIndexerLane) throws CommitFailedException {
+        NodeBuilder builder = nodeStore.getRoot().builder();
+        NodeBuilder async = builder.child(":async");
+
+        String leaseName = AsyncIndexUpdate.leasify(asyncIndexerLane);
+        long leaseEndTime = clock.getTime() + LOCK_TIMEOUT;
+
+        if (async.hasProperty(leaseName)){
+            log.info("AsyncIndexer found to be running currently. Lease update would cause its" +
+                    "commit to fail. Such a failure should be ignored");
+        }
+
+        //TODO Attempt few times if merge failure due to current running indexer cycle
+        async.setProperty(leaseName, leaseEndTime);
+        NodeStoreUtils.mergeWithConcurrentCheck(nodeStore, builder);
+
+        log.info("Acquired the lock for async indexer lane [{}]", asyncIndexerLane);
+
+        return new ClusteredLockToken(asyncIndexerLane, leaseEndTime);
+    }
+
+    @Override
+    public void unlock(ClusteredLockToken token) throws CommitFailedException {
+        String leaseName = AsyncIndexUpdate.leasify(token.laneName);
+
+        NodeBuilder builder = nodeStore.getRoot().builder();
+        NodeBuilder async = builder.child(":async");
+        async.removeProperty(leaseName);
+        NodeStoreUtils.mergeWithConcurrentCheck(nodeStore, builder);
+        log.info("Remove the lock for async indexer lane [{}]", token.laneName);
+    }
+}
+
+class ClusteredLockToken implements AsyncIndexerLock.LockToken {
+    final String laneName;
+    final long timeout;
+
+    ClusteredLockToken(String laneName, long timeout) {
+        this.laneName = laneName;
+        this.timeout = timeout;
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/ClusterNodeStoreLock.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporter.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporter.java?rev=1800052&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporter.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporter.java Tue Jun 27 11:47:25 2017
@@ -0,0 +1,244 @@
+/*
+ * 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.plugins.index.importer;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
+import org.apache.jackrabbit.oak.api.CommitFailedException;
+import org.apache.jackrabbit.oak.plugins.index.IndexConstants;
+import org.apache.jackrabbit.oak.plugins.index.IndexEditorProvider;
+import org.apache.jackrabbit.oak.plugins.index.IndexUpdate;
+import org.apache.jackrabbit.oak.plugins.index.IndexUpdateCallback;
+import org.apache.jackrabbit.oak.plugins.index.IndexUtils;
+import org.apache.jackrabbit.oak.plugins.index.importer.AsyncIndexerLock.LockToken;
+import org.apache.jackrabbit.oak.spi.commit.EditorDiff;
+import org.apache.jackrabbit.oak.spi.commit.VisibleEditor;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.apache.jackrabbit.oak.spi.state.NodeStateUtils;
+import org.apache.jackrabbit.oak.spi.state.NodeStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.jackrabbit.oak.plugins.index.importer.NodeStoreUtils.mergeWithConcurrentCheck;
+
+public class IndexImporter {
+    /**
+     * Symbolic name use to indicate sync indexes
+     */
+    static final String ASYNC_LANE_SYNC = "sync";
+
+    private final Logger log = LoggerFactory.getLogger(getClass());
+    private final NodeStore nodeStore;
+    private final File indexDir;
+    private final Map<String, IndexImporterProvider> importers = new HashMap<>();
+    private final IndexerInfo indexerInfo;
+    private final Map<String, File> indexes;
+    private final ListMultimap<String, IndexInfo> asyncLaneToIndexMapping;
+    private final NodeState indexedState;
+    private final IndexEditorProvider indexEditorProvider;
+    private final AsyncIndexerLock indexerLock;
+
+    public IndexImporter(NodeStore nodeStore, File indexDir, IndexEditorProvider indexEditorProvider,
+                         AsyncIndexerLock indexerLock) throws IOException {
+        this.nodeStore = nodeStore;
+        this.indexDir = indexDir;
+        this.indexEditorProvider = indexEditorProvider;
+        indexerInfo = IndexerInfo.fromDirectory(indexDir);
+        this.indexerLock = indexerLock;
+        indexes = indexerInfo.getIndexes();
+        asyncLaneToIndexMapping = mapIndexesToLanes(indexes);
+        indexedState = checkNotNull(nodeStore.retrieve(indexerInfo.checkpoint), "Cannot retrieve " +
+                "checkpointed state [%s]", indexerInfo.checkpoint);
+    }
+
+    public void importIndex() throws IOException, CommitFailedException {
+        log.info("Proceeding to import {} indexes from {}", indexes.keySet(), indexDir.getAbsolutePath());
+
+        //TODO Need to review it for idempotent design. A failure in any step should not
+        //leave setup in in consistent state and provide option for recovery
+
+        //Step 1 - Switch the index lanes so that async indexer does not touch them
+        //while we are importing the index data
+        switchLanes();
+        log.info("Done with switching of index lanes before import");
+
+        //Step 2 - Import the existing index data
+        importIndexData();
+        log.info("Done with importing of index data");
+
+        //Step 3 - Bring index upto date
+        bringIndexUpToDate();
+
+        //Step 4 - Release the checkpoint
+        releaseCheckpoint();
+    }
+
+    public void addImporterProvider(IndexImporterProvider importerProvider) {
+        importers.put(importerProvider.getType(), importerProvider);
+    }
+
+    void switchLanes() throws CommitFailedException {
+        NodeBuilder builder = nodeStore.getRoot().builder();
+        for (IndexInfo indexInfo : asyncLaneToIndexMapping.values()){
+            NodeBuilder idxBuilder = NodeStoreUtils.childBuilder(builder, indexInfo.indexPath);
+            AsyncLaneSwitcher.switchLane(idxBuilder, AsyncLaneSwitcher.getTempLaneName(indexInfo.asyncLaneName));
+        }
+        mergeWithConcurrentCheck(nodeStore, builder);
+    }
+
+    void importIndexData() throws CommitFailedException {
+        NodeState root = nodeStore.getRoot();
+        NodeBuilder builder = root.builder();
+        for (IndexInfo indexInfo : asyncLaneToIndexMapping.values()) {
+            log.info("Importing index data for {}", indexInfo.indexPath);
+            NodeBuilder idxBuilder = NodeStoreUtils.childBuilder(builder, indexInfo.indexPath);
+            //TODO Drop existing hidden folders
+            //Increment reindex count
+            getImporter(indexInfo.type).importIndex(root, idxBuilder, indexInfo.indexDir);
+        }
+        mergeWithConcurrentCheck(nodeStore, builder);
+    }
+
+    void bringIndexUpToDate() throws CommitFailedException {
+        for (String laneName : asyncLaneToIndexMapping.keySet()) {
+            if (ASYNC_LANE_SYNC.equals(laneName)){
+                continue; //TODO Handle sync indexes
+            }
+            bringAsyncIndexUpToDate(laneName, asyncLaneToIndexMapping.get(laneName));
+        }
+    }
+
+    private void bringAsyncIndexUpToDate(String laneName, List<IndexInfo> indexInfos) throws CommitFailedException {
+        LockToken lockToken = interruptCurrentIndexing(laneName);
+
+        String checkpoint = getAsync().getString(laneName);
+        checkNotNull(checkpoint, "No current checkpoint found for lane [%s]", laneName);
+
+        //TODO Support case where checkpoint got lost or complete reindexing is done
+
+        NodeState after = nodeStore.retrieve(checkpoint);
+        checkNotNull(after, "No state found for checkpoint [%s] for lane [%s]",checkpoint, laneName);
+        NodeState before = indexedState;
+
+        NodeBuilder builder = nodeStore.getRoot().builder();
+
+        IndexUpdate indexUpdate = new IndexUpdate(
+                indexEditorProvider,
+                AsyncLaneSwitcher.getTempLaneName(laneName),
+                nodeStore.getRoot(),
+                builder,
+                IndexUpdateCallback.NOOP
+        );
+
+        CommitFailedException exception =
+                EditorDiff.process(VisibleEditor.wrap(indexUpdate), before, after);
+
+        if (exception != null) {
+            throw exception;
+        }
+
+        revertLaneChange(builder, indexInfos);
+
+        mergeWithConcurrentCheck(nodeStore, builder);
+        resumeCurrentIndexing(lockToken);
+        log.info("Import done for indexes {}", indexInfos);
+    }
+
+    private void revertLaneChange(NodeBuilder builder, List<IndexInfo> indexInfos) {
+        for (IndexInfo info : indexInfos) {
+            NodeBuilder idxBuilder = NodeStoreUtils.childBuilder(builder, info.indexPath);
+            AsyncLaneSwitcher.revertSwitch(idxBuilder, info.indexPath);
+        }
+    }
+
+    private void resumeCurrentIndexing(LockToken lockToken) throws CommitFailedException {
+        indexerLock.unlock(lockToken);
+    }
+
+    private LockToken interruptCurrentIndexing(String laneName) throws CommitFailedException {
+        return indexerLock.lock(laneName);
+    }
+
+    private IndexImporterProvider getImporter(String type) {
+        IndexImporterProvider provider = importers.get(type);
+        return checkNotNull(provider, "No IndexImporterProvider found for type [%s]", type);
+    }
+
+    private ListMultimap<String, IndexInfo> mapIndexesToLanes(Map<String, File> indexes) {
+        NodeState rootState = nodeStore.getRoot();
+        ListMultimap<String, IndexInfo> map = ArrayListMultimap.create();
+        for (Map.Entry<String, File> e : indexes.entrySet()) {
+            String indexPath = e.getKey();
+
+            NodeState indexState = NodeStateUtils.getNode(rootState, indexPath);
+            checkArgument(indexState.exists(), "No index node found at path [%s]", indexPath);
+
+            //TODO Also check for previous lane in case of reattempt
+            String type = indexState.getString(IndexConstants.TYPE_PROPERTY_NAME);
+            checkNotNull(type, "No 'type' property found for index at path [%s]", indexPath);
+
+            String asyncName = IndexUtils.getAsyncLaneName(indexState, indexPath);
+            if (asyncName == null) {
+                asyncName = ASYNC_LANE_SYNC;
+            }
+
+            map.put(asyncName, new IndexInfo(indexPath, e.getValue(), asyncName, type));
+        }
+        return map;
+    }
+
+    private void releaseCheckpoint() {
+        nodeStore.release(indexerInfo.checkpoint);
+        log.info("Released the referred checkpoint [{}]", indexerInfo.checkpoint);
+    }
+
+    private NodeState getAsync() {
+        return nodeStore.getRoot().getChildNode(":async");
+    }
+
+    private static class IndexInfo {
+        final String indexPath;
+        final File indexDir;
+        final String asyncLaneName;
+        final String type;
+
+        private IndexInfo(String indexPath, File indexDir, String asyncLaneName, String type) {
+            this.indexPath = indexPath;
+            this.indexDir = indexDir;
+            this.asyncLaneName = asyncLaneName;
+            this.type = type;
+        }
+
+        @Override
+        public String toString() {
+            return indexPath;
+        }
+    }
+
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporter.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporterProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporterProvider.java?rev=1800052&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporterProvider.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporterProvider.java Tue Jun 27 11:47:25 2017
@@ -0,0 +1,43 @@
+/*
+ * 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.plugins.index.importer;
+
+import java.io.File;
+
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+
+/**
+ * This class is responsible for importing the index
+ * data from the file system into the repository.
+ */
+public interface IndexImporterProvider {
+
+    /**
+     * Import the index data from given directory into the
+     * NodeBuilder created for the index at given path
+     */
+    void importIndex(NodeState root, NodeBuilder defn, File indexDir);
+
+    /**
+     * Index type for this implementation
+     */
+    String getType();
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporterProvider.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexerInfo.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexerInfo.java?rev=1800052&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexerInfo.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexerInfo.java Tue Jun 27 11:47:25 2017
@@ -0,0 +1,104 @@
+/*
+ * 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.plugins.index.importer;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Properties;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.io.filefilter.DirectoryFileFilter;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Represents the index data created by oak-run tooling on the file system.
+ * It looks for 'indexer-info.properties' file in given directory to read the
+ * check point information.
+ *
+ * Then for each sub directory it looks for 'index-details.txt' file
+ * which contains index specific implementation details. It looks for
+ * property 'indexPath' which is used to associate the index data to
+ * index location in repository
+ */
+public class IndexerInfo {
+    /**
+     * File name stored in final index directory which contains meta
+     * information like checkpoint details. This can be used by
+     * importer while importing the indexes
+     */
+    public static final String INDEXER_META = "indexer-info.properties";
+
+    /**
+     * Name of meta file which stores the index related meta information
+     * in properties file format
+     */
+    public static final String INDEX_METADATA_FILE_NAME = "index-details.txt";
+
+    /**
+     * Property name in index-details.txt which refers to the
+     * index path in repository
+     */
+    public static final String PROP_INDEX_PATH = "indexPath";
+
+    public final String checkpoint;
+    private final File rootDir;
+
+    public IndexerInfo(File rootDir, String checkpoint) {
+        this.rootDir = rootDir;
+        this.checkpoint = checkNotNull(checkpoint);
+    }
+
+    public void save() throws IOException {
+        File infoFile = new File(rootDir, INDEXER_META);
+        Properties p = new Properties();
+        p.setProperty("checkpoint", checkpoint);
+        PropUtils.writeTo(p, infoFile, "Indexer info");
+    }
+
+    public Map<String, File> getIndexes() throws IOException {
+        ImmutableMap.Builder<String, File> indexes = ImmutableMap.builder();
+        for (File dir : rootDir.listFiles(((FileFilter) DirectoryFileFilter.DIRECTORY))) {
+            File metaFile = new File(dir, INDEX_METADATA_FILE_NAME);
+            if (metaFile.exists()) {
+                Properties p = PropUtils.loadFromFile(metaFile);
+                String indexPath = p.getProperty(PROP_INDEX_PATH);
+                if (indexPath != null) {
+                    indexes.put(indexPath, dir);
+                }
+            }
+        }
+        return indexes.build();
+    }
+
+    public static IndexerInfo fromDirectory(File rootDir) throws IOException {
+        File infoFile = new File(rootDir, INDEXER_META);
+        checkArgument(infoFile.exists(), "No [%s] file found in [%s]. Not a valid exported index " +
+                "directory", INDEXER_META, rootDir.getAbsolutePath());
+        Properties p = PropUtils.loadFromFile(infoFile);
+        return new IndexerInfo(
+                rootDir,
+                PropUtils.getProp(p, "checkpoint")
+        );
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexerInfo.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/NodeStoreUtils.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/NodeStoreUtils.java?rev=1800052&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/NodeStoreUtils.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/NodeStoreUtils.java Tue Jun 27 11:47:25 2017
@@ -0,0 +1,55 @@
+/*
+ * 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.plugins.index.importer;
+
+import java.util.Collections;
+
+import org.apache.jackrabbit.oak.api.CommitFailedException;
+import org.apache.jackrabbit.oak.commons.PathUtils;
+import org.apache.jackrabbit.oak.plugins.commit.AnnotatingConflictHandler;
+import org.apache.jackrabbit.oak.plugins.commit.ConflictHook;
+import org.apache.jackrabbit.oak.plugins.commit.ConflictValidatorProvider;
+import org.apache.jackrabbit.oak.spi.commit.CommitInfo;
+import org.apache.jackrabbit.oak.spi.commit.CompositeEditorProvider;
+import org.apache.jackrabbit.oak.spi.commit.CompositeHook;
+import org.apache.jackrabbit.oak.spi.commit.EditorHook;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeStore;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static java.util.Collections.singletonList;
+
+final class NodeStoreUtils {
+
+    static void mergeWithConcurrentCheck(NodeStore nodeStore, NodeBuilder builder) throws CommitFailedException {
+        CompositeHook hooks = new CompositeHook(
+                new ConflictHook(new AnnotatingConflictHandler()),
+                new EditorHook(CompositeEditorProvider.compose(singletonList(new ConflictValidatorProvider())))
+        );
+        nodeStore.merge(builder, hooks, CommitInfo.EMPTY);
+    }
+
+    static NodeBuilder childBuilder(NodeBuilder nb, String path) {
+        for (String name : PathUtils.elements(checkNotNull(path))) {
+            nb = nb.child(name);
+        }
+        return nb;
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/NodeStoreUtils.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/PropUtils.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/PropUtils.java?rev=1800052&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/PropUtils.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/PropUtils.java Tue Jun 27 11:47:25 2017
@@ -0,0 +1,53 @@
+/*
+ * 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.plugins.index.importer;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Properties;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+final class PropUtils {
+
+    static Properties loadFromFile(File file) throws IOException {
+        try (InputStream is = new BufferedInputStream(new FileInputStream(file))) {
+            Properties p = new Properties();
+            p.load(is);
+            return p;
+        }
+    }
+
+    static String getProp(Properties p, String key) {
+        return checkNotNull(p.getProperty(key), "No property named [%s] found", key);
+    }
+
+    static void writeTo(Properties p, File file, String comment) throws IOException {
+        try (OutputStream os = new BufferedOutputStream(new FileOutputStream(file))) {
+            p.store(os, comment);
+        }
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/importer/PropUtils.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncLaneSwitcherTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncLaneSwitcherTest.java?rev=1800052&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncLaneSwitcherTest.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncLaneSwitcherTest.java Tue Jun 27 11:47:25 2017
@@ -0,0 +1,104 @@
+/*
+ * 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.plugins.index.importer;
+
+import java.util.Arrays;
+
+import org.apache.jackrabbit.oak.api.PropertyState;
+import org.apache.jackrabbit.oak.api.Type;
+import org.apache.jackrabbit.oak.plugins.index.IndexConstants;
+import org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.junit.Test;
+
+import static java.util.Arrays.asList;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.ASYNC_PROPERTY_NAME;
+import static org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState.EMPTY_NODE;
+import static org.junit.Assert.*;
+
+public class AsyncLaneSwitcherTest {
+
+    private NodeBuilder builder = EMPTY_NODE.builder();
+
+    @Test
+    public void switchNone() throws Exception{
+        AsyncLaneSwitcher.switchLane(builder, "foo");
+
+        PropertyState previous = builder.getProperty(AsyncLaneSwitcher.ASYNC_PREVIOUS);
+        assertNotNull(previous);
+        assertEquals(AsyncLaneSwitcher.ASYNC_PREVIOUS_NONE, previous.getValue(Type.STRING));
+    }
+
+    @Test
+    public void switchSingleAsync() throws Exception{
+        builder.setProperty(ASYNC_PROPERTY_NAME, "async");
+
+        AsyncLaneSwitcher.switchLane(builder, "foo");
+
+        PropertyState previous = builder.getProperty(AsyncLaneSwitcher.ASYNC_PREVIOUS);
+        assertNotNull(previous);
+        assertEquals("async", previous.getValue(Type.STRING));
+    }
+
+    @Test
+    public void switchAsyncArray() throws Exception{
+        builder.setProperty(ASYNC_PROPERTY_NAME, asList("async", "nrt"), Type.STRINGS);
+
+        AsyncLaneSwitcher.switchLane(builder, "foo");
+
+        PropertyState previous = builder.getProperty(AsyncLaneSwitcher.ASYNC_PREVIOUS);
+        assertNotNull(previous);
+        assertEquals(asList("async", "nrt"), previous.getValue(Type.STRINGS));
+    }
+
+    @Test
+    public void multipleSwitch() throws Exception{
+        builder.setProperty(ASYNC_PROPERTY_NAME, "async");
+
+        AsyncLaneSwitcher.switchLane(builder, "foo");
+        AsyncLaneSwitcher.switchLane(builder, "foo");
+
+        PropertyState previous = builder.getProperty(AsyncLaneSwitcher.ASYNC_PREVIOUS);
+        assertNotNull(previous);
+        assertEquals("async", previous.getValue(Type.STRING));
+    }
+
+    @Test
+    public void revert() throws Exception{
+        builder.setProperty(ASYNC_PROPERTY_NAME, "async");
+
+        AsyncLaneSwitcher.switchLane(builder, "foo");
+        assertNotNull(builder.getProperty(AsyncLaneSwitcher.ASYNC_PREVIOUS));
+
+        AsyncLaneSwitcher.revertSwitch(builder, "/fooIndex");
+        assertNull(builder.getProperty(AsyncLaneSwitcher.ASYNC_PREVIOUS));
+        assertEquals("async", builder.getString(ASYNC_PROPERTY_NAME));
+    }
+
+    @Test
+    public void revert_Sync() throws Exception{
+        AsyncLaneSwitcher.switchLane(builder, "foo");
+
+        AsyncLaneSwitcher.revertSwitch(builder, "/fooIndex");
+        assertNull(builder.getProperty(AsyncLaneSwitcher.ASYNC_PREVIOUS));
+        assertNull(builder.getProperty(ASYNC_PROPERTY_NAME));
+    }
+
+}
\ No newline at end of file

Propchange: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/AsyncLaneSwitcherTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/ClusterNodeStoreLockTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/ClusterNodeStoreLockTest.java?rev=1800052&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/ClusterNodeStoreLockTest.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/ClusterNodeStoreLockTest.java Tue Jun 27 11:47:25 2017
@@ -0,0 +1,77 @@
+/*
+ * 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.plugins.index.importer;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.jackrabbit.oak.plugins.index.AsyncIndexUpdate;
+import org.apache.jackrabbit.oak.plugins.index.IndexEditorProvider;
+import org.apache.jackrabbit.oak.plugins.index.importer.AsyncIndexerLock.LockToken;
+import org.apache.jackrabbit.oak.plugins.index.property.PropertyIndexEditorProvider;
+import org.apache.jackrabbit.oak.plugins.memory.MemoryNodeStore;
+import org.apache.jackrabbit.oak.spi.commit.CommitInfo;
+import org.apache.jackrabbit.oak.spi.commit.EmptyHook;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.apache.jackrabbit.oak.spi.state.NodeStore;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.ASYNC_PROPERTY_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.IndexUtils.createIndexDefinition;
+import static org.junit.Assert.*;
+
+public class ClusterNodeStoreLockTest {
+    private NodeStore store = new MemoryNodeStore();
+    private IndexEditorProvider provider;
+    private String name = "async";
+
+    @Before
+    public void setup() throws Exception {
+        provider = new PropertyIndexEditorProvider();
+        NodeBuilder builder = store.getRoot().builder();
+        createIndexDefinition(builder.child(INDEX_DEFINITIONS_NAME),
+                "rootIndex", true, false, ImmutableSet.of("foo"), null)
+                .setProperty(ASYNC_PROPERTY_NAME, name);
+        builder.child("testRoot").setProperty("foo", "abc");
+        store.merge(builder, EmptyHook.INSTANCE, CommitInfo.EMPTY);
+    }
+
+    @Test
+    public void locking() throws Exception{
+        new AsyncIndexUpdate(name, store, provider).run();
+
+        assertFalse(getAsync().hasProperty(AsyncIndexUpdate.leasify(name)));
+
+        AsyncIndexerLock lock = new ClusterNodeStoreLock(store);
+        LockToken token = lock.lock("async");
+
+        assertTrue(getAsync().hasProperty(AsyncIndexUpdate.leasify(name)));
+
+        lock.unlock(token);
+        assertFalse(getAsync().hasProperty(AsyncIndexUpdate.leasify(name)));
+    }
+
+    private NodeState getAsync() {
+        return store.getRoot().getChildNode(":async");
+    }
+
+    //TODO Test for check if changing lease actually cause current running indexer to fail
+}
\ No newline at end of file

Propchange: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/ClusterNodeStoreLockTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporterTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporterTest.java?rev=1800052&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporterTest.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporterTest.java Tue Jun 27 11:47:25 2017
@@ -0,0 +1,241 @@
+/*
+ * 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.plugins.index.importer;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Properties;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import org.apache.jackrabbit.oak.commons.PathUtils;
+import org.apache.jackrabbit.oak.plugins.index.AsyncIndexUpdate;
+import org.apache.jackrabbit.oak.plugins.index.IndexConstants;
+import org.apache.jackrabbit.oak.plugins.index.IndexEditorProvider;
+import org.apache.jackrabbit.oak.plugins.index.property.PropertyIndexEditorProvider;
+import org.apache.jackrabbit.oak.plugins.index.property.PropertyIndexLookup;
+import org.apache.jackrabbit.oak.plugins.memory.MemoryNodeStore;
+import org.apache.jackrabbit.oak.plugins.memory.PropertyValues;
+import org.apache.jackrabbit.oak.query.NodeStateNodeTypeInfoProvider;
+import org.apache.jackrabbit.oak.query.ast.NodeTypeInfo;
+import org.apache.jackrabbit.oak.query.ast.NodeTypeInfoProvider;
+import org.apache.jackrabbit.oak.query.ast.SelectorImpl;
+import org.apache.jackrabbit.oak.query.index.FilterImpl;
+import org.apache.jackrabbit.oak.spi.commit.CommitInfo;
+import org.apache.jackrabbit.oak.spi.commit.EmptyHook;
+import org.apache.jackrabbit.oak.spi.query.Filter;
+import org.apache.jackrabbit.oak.spi.query.QueryEngineSettings;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.apache.jackrabbit.oak.spi.state.NodeStateUtils;
+import org.apache.jackrabbit.oak.spi.state.NodeStore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static com.google.common.collect.ImmutableSet.of;
+import static org.apache.jackrabbit.JcrConstants.NT_BASE;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.ASYNC_PROPERTY_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.IndexUtils.createIndexDefinition;
+import static org.apache.jackrabbit.oak.plugins.index.importer.AsyncIndexerLock.NOOP_LOCK;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+public class IndexImporterTest {
+
+    @Rule
+    public final TemporaryFolder temporaryFolder = new TemporaryFolder(new File("target"));
+
+    private NodeStore store = new MemoryNodeStore();
+    private IndexEditorProvider provider = new PropertyIndexEditorProvider();
+
+    @Test(expected = IllegalArgumentException.class)
+    public void importIndex_NoMeta() throws Exception{
+        IndexImporter importer = new IndexImporter(store, temporaryFolder.getRoot(), provider, NOOP_LOCK);
+    }
+
+    @Test(expected = NullPointerException.class)
+    public void lostCheckpoint() throws Exception{
+        IndexerInfo info = new IndexerInfo(temporaryFolder.getRoot(), "unknown-checkpoint");
+        info.save();
+        IndexImporter importer = new IndexImporter(store, temporaryFolder.getRoot(), provider, NOOP_LOCK);
+    }
+
+    @Test
+    public void switchLanes() throws Exception{
+        NodeBuilder builder = store.getRoot().builder();
+        builder.child("idx-a").setProperty("type", "property");
+
+        builder.child("idx-b").setProperty("type", "property");
+        builder.child("idx-b").setProperty("async", "async");
+
+        store.merge(builder, EmptyHook.INSTANCE, CommitInfo.EMPTY);
+
+        createIndexDirs("/idx-a", "/idx-b");
+
+        IndexImporter importer = new IndexImporter(store, temporaryFolder.getRoot(), provider, NOOP_LOCK);
+        importer.switchLanes();
+
+        NodeState idxa = NodeStateUtils.getNode(store.getRoot(), "/idx-a");
+        assertEquals(AsyncLaneSwitcher.getTempLaneName(IndexImporter.ASYNC_LANE_SYNC), idxa.getString(ASYNC_PROPERTY_NAME));
+
+        NodeState idxb = NodeStateUtils.getNode(store.getRoot(), "/idx-b");
+        assertEquals(AsyncLaneSwitcher.getTempLaneName("async"), idxb.getString(ASYNC_PROPERTY_NAME));
+    }
+
+    @Test(expected = NullPointerException.class)
+    public void importData_NoProvider() throws Exception{
+        NodeBuilder builder = store.getRoot().builder();
+        builder.child("idx-a").setProperty("type", "property");
+
+        store.merge(builder, EmptyHook.INSTANCE, CommitInfo.EMPTY);
+
+        createIndexDirs("/idx-a");
+
+        IndexImporter importer = new IndexImporter(store, temporaryFolder.getRoot(), provider, NOOP_LOCK);
+        importer.switchLanes();
+        importer.importIndexData();
+    }
+
+    @Test
+    public void importData_CallbackInvoked() throws Exception{
+        NodeBuilder builder = store.getRoot().builder();
+        builder.child("idx-a").setProperty("type", "property");
+        builder.child("idx-a").setProperty("foo", "bar");
+
+        store.merge(builder, EmptyHook.INSTANCE, CommitInfo.EMPTY);
+
+        createIndexDirs("/idx-a");
+
+        IndexImporter importer = new IndexImporter(store, temporaryFolder.getRoot(), provider, NOOP_LOCK);
+
+        IndexImporterProvider provider = new IndexImporterProvider() {
+            @Override
+            public void importIndex(NodeState root, NodeBuilder defn, File indexDir) {
+                assertEquals("bar", defn.getString("foo"));
+                assertEquals("idx-a", indexDir.getName());
+                defn.setProperty("imported", true);
+            }
+
+            @Override
+            public String getType() {
+                return "property";
+            }
+        };
+        importer.addImporterProvider(provider);
+        importer.switchLanes();
+        importer.importIndexData();
+
+        assertTrue(store.getRoot().getChildNode("idx-a").getBoolean("imported"));
+    }
+
+    @Test
+    public void importData_IncrementalUpdate() throws Exception{
+        NodeBuilder builder = store.getRoot().builder();
+        createIndexDefinition(builder.child(INDEX_DEFINITIONS_NAME),
+                "fooIndex", true, false, ImmutableSet.of("foo"), null)
+                .setProperty(ASYNC_PROPERTY_NAME, "async");
+        builder.child("a").setProperty("foo", "abc");
+        builder.child("b").setProperty("foo", "abc");
+        store.merge(builder, EmptyHook.INSTANCE, CommitInfo.EMPTY);
+
+        new AsyncIndexUpdate("async", store, provider).run();
+
+        String checkpoint = createIndexDirs("/oak:index/fooIndex");
+
+        builder = store.getRoot().builder();
+        builder.child("c").setProperty("foo", "abc");
+        builder.child("d").setProperty("foo", "abc");
+        store.merge(builder, EmptyHook.INSTANCE, CommitInfo.EMPTY);
+
+        new AsyncIndexUpdate("async", store, provider).run();
+
+        FilterImpl f = createFilter(store.getRoot(), NT_BASE);
+        PropertyIndexLookup lookup = new PropertyIndexLookup(store.getRoot());
+        assertEquals(of("a", "b", "c", "d"), find(lookup, "foo", "abc", f));
+
+        IndexImporterProvider importerProvider = new IndexImporterProvider() {
+            @Override
+            public void importIndex(NodeState root, NodeBuilder defn, File indexDir) {
+                assertEquals("fooIndex", indexDir.getName());
+                defn.getChildNode(IndexConstants.INDEX_CONTENT_NODE_NAME).remove();
+
+                NodeState cpState = store.retrieve(checkpoint);
+                NodeState indexData = NodeStateUtils.getNode(cpState, "/oak:index/fooIndex/:index");
+                defn.setChildNode(IndexConstants.INDEX_CONTENT_NODE_NAME, indexData);
+            }
+
+            @Override
+            public String getType() {
+                return "property";
+            }
+        };
+
+        builder = store.getRoot().builder();
+        builder.child("e").setProperty("foo", "abc");
+        store.merge(builder, EmptyHook.INSTANCE, CommitInfo.EMPTY);
+
+        IndexImporter importer = new IndexImporter(store, temporaryFolder.getRoot(), provider, NOOP_LOCK);
+        importer.addImporterProvider(importerProvider);
+        importer.addImporterProvider(importerProvider);
+
+        importer.importIndex();
+
+        lookup = new PropertyIndexLookup(store.getRoot());
+        //It would not pickup /e as thats not yet indexed as part of last checkpoint
+        assertEquals(of("a", "b", "c", "d"), find(lookup, "foo", "abc", f));
+        assertNull(store.retrieve(checkpoint));
+    }
+
+    private static FilterImpl createFilter(NodeState root, String nodeTypeName) {
+        NodeTypeInfoProvider nodeTypes = new NodeStateNodeTypeInfoProvider(root);
+        NodeTypeInfo type = nodeTypes.getNodeTypeInfo(nodeTypeName);
+        SelectorImpl selector = new SelectorImpl(type, nodeTypeName);
+        return new FilterImpl(selector, "SELECT * FROM [" + nodeTypeName + "]",
+                new QueryEngineSettings());
+    }
+
+    private static Set<String> find(PropertyIndexLookup lookup, String name,
+                                    String value, Filter filter) {
+        return Sets.newHashSet(lookup.query(filter, name, value == null ? null
+                : PropertyValues.newString(value)));
+    }
+
+    private String createIndexDirs(String... indexPaths) throws IOException {
+        String checkpoint = store.checkpoint(1000000);
+        IndexerInfo info = new IndexerInfo(temporaryFolder.getRoot(), checkpoint);
+        info.save();
+
+        for (String indexPath : indexPaths){
+            String dirName = PathUtils.getName(indexPath);
+            File indexDir = new File(temporaryFolder.getRoot(), dirName);
+            File indexMeta = new File(indexDir, IndexerInfo.INDEX_METADATA_FILE_NAME);
+            Properties p = new Properties();
+            p.setProperty(IndexerInfo.PROP_INDEX_PATH, indexPath);
+            indexDir.mkdir();
+            PropUtils.writeTo(p, indexMeta, "index info");
+        }
+
+        return checkpoint;
+    }
+}
\ No newline at end of file

Propchange: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexImporterTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexerInfoTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexerInfoTest.java?rev=1800052&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexerInfoTest.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexerInfoTest.java Tue Jun 27 11:47:25 2017
@@ -0,0 +1,67 @@
+/*
+ * 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.plugins.index.importer;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.junit.Assert.*;
+
+public class IndexerInfoTest {
+    @Rule
+    public final TemporaryFolder temporaryFolder = new TemporaryFolder(new File("target"));
+
+    @Test
+    public void basics() throws Exception{
+        IndexerInfo info = new IndexerInfo(temporaryFolder.getRoot(), "foo");
+        info.save();
+
+        IndexerInfo info2 = IndexerInfo.fromDirectory(temporaryFolder.getRoot());
+        assertEquals(info.checkpoint, info2.checkpoint);
+    }
+
+    @Test
+    public void indexDirs() throws Exception{
+        IndexerInfo info = new IndexerInfo(temporaryFolder.getRoot(), "foo");
+        info.save();
+
+        List<String> indexPaths = Arrays.asList("/foo", "/bar");
+        for (String indexPath : indexPaths){
+            File indexDir = new File(temporaryFolder.getRoot(), indexPath.substring(1));
+            File indexMeta = new File(indexDir, IndexerInfo.INDEX_METADATA_FILE_NAME);
+            Properties p = new Properties();
+            p.setProperty(IndexerInfo.PROP_INDEX_PATH, indexPath);
+            indexDir.mkdir();
+            PropUtils.writeTo(p, indexMeta, "index info");
+        }
+
+        Map<String, File> indexes = info.getIndexes();
+        assertThat(indexes.keySet(), containsInAnyOrder("/foo", "/bar"));
+    }
+
+}
\ No newline at end of file

Propchange: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/importer/IndexerInfoTest.java
------------------------------------------------------------------------------
    svn:eol-style = native