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 al...@apache.org on 2016/10/17 14:20:12 UTC

svn commit: r1765293 [1/2] - in /jackrabbit/oak/trunk/oak-segment-tar/src: main/java/org/apache/jackrabbit/oak/backup/impl/ main/java/org/apache/jackrabbit/oak/segment/ main/java/org/apache/jackrabbit/oak/segment/file/ main/java/org/apache/jackrabbit/o...

Author: alexparvulescu
Date: Mon Oct 17 14:20:11 2016
New Revision: 1765293

URL: http://svn.apache.org/viewvc?rev=1765293&view=rev
Log:
OAK-4450 Properly split the FileStore into read-only and r/w variants


Added:
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/AbstractFileStore.java   (with props)
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/ReadOnlyFileStore.java   (with props)
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/ReadOnlyRevisions.java   (with props)
Modified:
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/backup/impl/FileStoreRestoreImpl.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentGraph.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentNodeStoreBuilders.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStore.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStoreBuilder.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/TarRevisions.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/tooling/ConsistencyChecker.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/tooling/RevisionHistory.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/Backup.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugSegments.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugStore.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugTars.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/Diff.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/GenerationGraph.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/SegmentGraph.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/Utils.java
    jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/SegmentBufferWriterTest.java
    jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/SegmentGraphTest.java
    jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/file/FileStoreIT.java
    jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/file/TarRevisionsTest.java

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/backup/impl/FileStoreRestoreImpl.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/backup/impl/FileStoreRestoreImpl.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/backup/impl/FileStoreRestoreImpl.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/backup/impl/FileStoreRestoreImpl.java Mon Oct 17 14:20:11 2016
@@ -36,6 +36,7 @@ import org.apache.jackrabbit.oak.segment
 import org.apache.jackrabbit.oak.segment.compaction.SegmentGCOptions;
 import org.apache.jackrabbit.oak.segment.file.FileStore;
 import org.apache.jackrabbit.oak.segment.file.InvalidFileStoreVersionException;
+import org.apache.jackrabbit.oak.segment.file.ReadOnlyFileStore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,7 +52,7 @@ public class FileStoreRestoreImpl implem
             throw new IOException("Folder " + source + " is not a valid FileStore directory");
         }
 
-        FileStore restore = fileStoreBuilder(source).buildReadOnly();
+        ReadOnlyFileStore restore = fileStoreBuilder(source).buildReadOnly();
         Stopwatch watch = Stopwatch.createStarted();
 
         FileStore store = fileStoreBuilder(destination).build();

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentGraph.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentGraph.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentGraph.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentGraph.java Mon Oct 17 14:20:11 2016
@@ -56,7 +56,7 @@ import com.google.common.collect.Multise
 import org.apache.jackrabbit.oak.api.Type;
 import org.apache.jackrabbit.oak.commons.json.JsonObject;
 import org.apache.jackrabbit.oak.commons.json.JsopTokenizer;
-import org.apache.jackrabbit.oak.segment.file.FileStore.ReadOnlyStore;
+import org.apache.jackrabbit.oak.segment.file.ReadOnlyFileStore;
 
 /**
  * Utility graph for parsing a segment graph.
@@ -153,7 +153,7 @@ public final class SegmentGraph {
      * @throws Exception
      */
     public static void writeSegmentGraph(
-            @Nonnull ReadOnlyStore fileStore,
+            @Nonnull ReadOnlyFileStore fileStore,
             @Nonnull OutputStream out,
             @Nonnull Date epoch,
             @CheckForNull String pattern) throws Exception {
@@ -230,7 +230,7 @@ public final class SegmentGraph {
      */
     @Nonnull
     public static Graph<UUID> parseSegmentGraph(
-            @Nonnull ReadOnlyStore fileStore,
+            @Nonnull ReadOnlyFileStore fileStore,
             @Nonnull Predicate<UUID> filter) throws IOException {
         SegmentNodeState root = checkNotNull(fileStore).getHead();
         HashSet<UUID> roots = newHashSet(root.getRecordId().asUUID());
@@ -248,7 +248,7 @@ public final class SegmentGraph {
      * @param out           stream to write the graph to
      * @throws Exception
      */
-    public static void writeGCGraph(@Nonnull ReadOnlyStore fileStore, @Nonnull OutputStream out)
+    public static void writeGCGraph(@Nonnull ReadOnlyFileStore fileStore, @Nonnull OutputStream out)
             throws Exception {
         PrintWriter writer = new PrintWriter(checkNotNull(out));
         try {
@@ -283,7 +283,7 @@ public final class SegmentGraph {
      * @throws IOException
      */
     @Nonnull
-    public static Graph<String> parseGCGraph(@Nonnull final ReadOnlyStore fileStore)
+    public static Graph<String> parseGCGraph(@Nonnull final ReadOnlyFileStore fileStore)
             throws IOException {
         SegmentNodeState root = checkNotNull(fileStore).getHead();
         HashSet<UUID> roots = newHashSet(root.getRecordId().asUUID());
@@ -317,7 +317,7 @@ public final class SegmentGraph {
      */
     @Nonnull
     public static <T> Graph<T> parseSegmentGraph(
-            @Nonnull final ReadOnlyStore fileStore,
+            @Nonnull final ReadOnlyFileStore fileStore,
             @Nonnull Set<UUID> roots,
             @Nonnull final Predicate<UUID> filter,
             @Nonnull final Function<UUID, T> map) throws IOException {

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentNodeStoreBuilders.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentNodeStoreBuilders.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentNodeStoreBuilders.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentNodeStoreBuilders.java Mon Oct 17 14:20:11 2016
@@ -23,6 +23,7 @@ import javax.annotation.Nonnull;
 
 import org.apache.jackrabbit.oak.segment.SegmentNodeStore.SegmentNodeStoreBuilder;
 import org.apache.jackrabbit.oak.segment.file.FileStore;
+import org.apache.jackrabbit.oak.segment.file.ReadOnlyFileStore;
 import org.apache.jackrabbit.oak.segment.http.HttpStore;
 import org.apache.jackrabbit.oak.segment.memory.MemoryStore;
 
@@ -59,4 +60,13 @@ public final class SegmentNodeStoreBuild
         return SegmentNodeStore.builder(store.getRevisions(),
                 store.getReader(), store.getWriter(), store.getBlobStore());
     }
+
+    /**
+     * Create a {@code SegmentNodeStoreBuilder} based on a {@code ReadOnlyFileStore@}.
+     */
+    @Nonnull
+    public static SegmentNodeStoreBuilder builder(@Nonnull ReadOnlyFileStore store) {
+        return SegmentNodeStore.builder(store.getRevisions(),
+                store.getReader(), store.getWriter(), store.getBlobStore());
+    }
 }

Added: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/AbstractFileStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/AbstractFileStore.java?rev=1765293&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/AbstractFileStore.java (added)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/AbstractFileStore.java Mon Oct 17 14:20:11 2016
@@ -0,0 +1,314 @@
+/*
+ * 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.segment.file;
+
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.collect.Maps.newHashMap;
+import static com.google.common.collect.Sets.newHashSet;
+import static java.lang.String.format;
+import static java.util.Collections.singletonMap;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import javax.annotation.CheckForNull;
+import javax.annotation.Nonnull;
+
+import org.apache.jackrabbit.oak.api.jmx.CacheStatsMBean;
+import org.apache.jackrabbit.oak.segment.CachingSegmentReader;
+import org.apache.jackrabbit.oak.segment.Revisions;
+import org.apache.jackrabbit.oak.segment.SegmentCache;
+import org.apache.jackrabbit.oak.segment.SegmentId;
+import org.apache.jackrabbit.oak.segment.SegmentIdFactory;
+import org.apache.jackrabbit.oak.segment.SegmentNodeState;
+import org.apache.jackrabbit.oak.segment.SegmentReader;
+import org.apache.jackrabbit.oak.segment.SegmentStore;
+import org.apache.jackrabbit.oak.segment.SegmentTracker;
+import org.apache.jackrabbit.oak.segment.SegmentWriter;
+import org.apache.jackrabbit.oak.spi.blob.BlobStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Supplier;
+
+/**
+ * The storage implementation for tar files.
+ */
+public abstract class AbstractFileStore implements SegmentStore, Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(AbstractFileStore.class);
+
+    private static final String MANIFEST_FILE_NAME = "manifest";
+
+    /**
+     * This value can be used as an invalid store version, since the store
+     * version is defined to be strictly greater than zero.
+     */
+    private static final int INVALID_STORE_VERSION = 0;
+
+    /**
+     * The store version is an always incrementing number, strictly greater than
+     * zero, that is changed every time there is a backwards incompatible
+     * modification to the format of the segment store.
+     */
+    static final int CURRENT_STORE_VERSION = 1;
+
+    private static final Pattern FILE_NAME_PATTERN =
+            Pattern.compile("(data|bulk)((0|[1-9][0-9]*)[0-9]{4})([a-z])?.tar");
+
+    static final String FILE_NAME_FORMAT = "data%05d%s.tar";
+
+    @Nonnull
+    final SegmentTracker tracker;
+
+    @Nonnull
+    final CachingSegmentReader segmentReader;
+
+    final File directory;
+
+    private final BlobStore blobStore;
+
+    final boolean memoryMapping;
+
+    @Nonnull
+    final SegmentCache segmentCache;
+
+    @Nonnull
+    private final SegmentIdFactory segmentIdFactory = new SegmentIdFactory() {
+
+        @Override
+        @Nonnull
+        public SegmentId newSegmentId(long msb, long lsb) {
+            return new SegmentId(AbstractFileStore.this, msb, lsb);
+        }
+
+    };
+
+    AbstractFileStore(final FileStoreBuilder builder) throws InvalidFileStoreVersionException, IOException {
+        this.directory = builder.getDirectory();
+        this.tracker = new SegmentTracker();
+        this.blobStore = builder.getBlobStore();
+        this.segmentCache = new SegmentCache(builder.getSegmentCacheSize());
+        this.segmentReader = new CachingSegmentReader(new Supplier<SegmentWriter>() {
+            @Override
+            public SegmentWriter get() {
+                return getWriter();
+            }
+        }, blobStore, builder.getStringCacheSize(), builder.getTemplateCacheSize());
+        this.memoryMapping = builder.getMemoryMapping();
+    }
+
+     File getManifestFile() {
+        return new File(directory, MANIFEST_FILE_NAME);
+    }
+
+     Manifest openManifest() throws IOException {
+        File file = getManifestFile();
+
+        if (file.exists()) {
+            return Manifest.load(file);
+        }
+
+        return null;
+    }
+
+     Manifest checkManifest(Manifest manifest) throws InvalidFileStoreVersionException {
+        if (manifest == null) {
+            throw new InvalidFileStoreVersionException("Using oak-segment-tar, but oak-segment should be used");
+        }
+
+        int storeVersion = manifest.getStoreVersion(INVALID_STORE_VERSION);
+
+        // A store version less than or equal to the highest invalid value means
+        // that something or someone is messing up with the manifest. This error
+        // is not recoverable and is thus represented as an ISE.
+
+        if (storeVersion <= INVALID_STORE_VERSION) {
+            throw new IllegalStateException("Invalid store version");
+        }
+
+        if (storeVersion < CURRENT_STORE_VERSION) {
+            throw new InvalidFileStoreVersionException("Using a too recent version of oak-segment-tar");
+        }
+
+        if (storeVersion > CURRENT_STORE_VERSION) {
+            throw new InvalidFileStoreVersionException("Using a too old version of oak-segment tar");
+        }
+
+        return manifest;
+    }
+
+    @Nonnull
+    public CacheStatsMBean getSegmentCacheStats() {
+        return segmentCache.getCacheStats();
+    }
+
+    @Nonnull
+    public CacheStatsMBean getStringCacheStats() {
+        return segmentReader.getStringCacheStats();
+    }
+
+    @Nonnull
+    public CacheStatsMBean getTemplateCacheStats() {
+        return segmentReader.getTemplateCacheStats();
+    }
+
+    static Map<Integer, Map<Character, File>> collectFiles(File directory) {
+        Map<Integer, Map<Character, File>> dataFiles = newHashMap();
+        Map<Integer, File> bulkFiles = newHashMap();
+
+        for (File file : directory.listFiles()) {
+            Matcher matcher = FILE_NAME_PATTERN.matcher(file.getName());
+            if (matcher.matches()) {
+                Integer index = Integer.parseInt(matcher.group(2));
+                if ("data".equals(matcher.group(1))) {
+                    Map<Character, File> files = dataFiles.get(index);
+                    if (files == null) {
+                        files = newHashMap();
+                        dataFiles.put(index, files);
+                    }
+                    Character generation = 'a';
+                    if (matcher.group(4) != null) {
+                        generation = matcher.group(4).charAt(0);
+                    }
+                    checkState(files.put(generation, file) == null);
+                } else {
+                    checkState(bulkFiles.put(index, file) == null);
+                }
+            }
+        }
+
+        if (!bulkFiles.isEmpty()) {
+            log.info("Upgrading TarMK file names in {}", directory);
+
+            if (!dataFiles.isEmpty()) {
+                // first put all the data segments at the end of the list
+                Integer[] indices =
+                        dataFiles.keySet().toArray(new Integer[dataFiles.size()]);
+                Arrays.sort(indices);
+                int position = Math.max(
+                        indices[indices.length - 1] + 1,
+                        bulkFiles.size());
+                for (Integer index : indices) {
+                    Map<Character, File> files = dataFiles.remove(index);
+                    Integer newIndex = position++;
+                    for (Character generation : newHashSet(files.keySet())) {
+                        File file = files.get(generation);
+                        File newFile = new File(
+                                directory,
+                                format(FILE_NAME_FORMAT, newIndex, generation));
+                        log.info("Renaming {} to {}", file, newFile);
+                        file.renameTo(newFile);
+                        files.put(generation, newFile);
+                    }
+                    dataFiles.put(newIndex, files);
+                }
+            }
+
+            // then add all the bulk segments at the beginning of the list
+            Integer[] indices =
+                    bulkFiles.keySet().toArray(new Integer[bulkFiles.size()]);
+            Arrays.sort(indices);
+            int position = 0;
+            for (Integer index : indices) {
+                File file = bulkFiles.remove(index);
+                Integer newIndex = position++;
+                File newFile = new File(
+                        directory, format(FILE_NAME_FORMAT, newIndex, "a"));
+                log.info("Renaming {} to {}", file, newFile);
+                file.renameTo(newFile);
+                dataFiles.put(newIndex, singletonMap('a', newFile));
+            }
+        }
+
+        return dataFiles;
+    }
+
+    @Nonnull
+    public SegmentTracker getTracker() {
+        return tracker;
+    }
+
+    public abstract SegmentWriter getWriter();
+
+    @Nonnull
+    public SegmentReader getReader() {
+        return segmentReader;
+    }
+
+    /**
+     * @return the {@link Revisions} object bound to the current store.
+     */
+    public abstract Revisions getRevisions();
+
+    /**
+     * Convenience method for accessing the root node for the current head.
+     * This is equivalent to
+     * <pre>
+     * fileStore.getReader().readHeadState(fileStore.getRevisions())
+     * </pre>
+     * @return the current head node state
+     */
+    @Nonnull
+    public SegmentNodeState getHead() {
+        return segmentReader.readHeadState(getRevisions());
+    }
+
+    @Override
+    @Nonnull
+    public SegmentId newSegmentId(long msb, long lsb) {
+        return tracker.newSegmentId(msb, lsb, segmentIdFactory);
+    }
+
+    @Override
+    @Nonnull
+    public SegmentId newBulkSegmentId() {
+        return tracker.newBulkSegmentId(segmentIdFactory);
+    }
+
+    @Override
+    @Nonnull
+    public SegmentId newDataSegmentId() {
+        return tracker.newDataSegmentId(segmentIdFactory);
+    }
+
+    /**
+     * @return  the external BlobStore (if configured) with this store, {@code null} otherwise.
+     */
+    @CheckForNull
+    public BlobStore getBlobStore() {
+        return blobStore;
+    }
+
+    static void closeAndLogOnFail(Closeable closeable) {
+        if (closeable != null) {
+            try {
+                closeable.close();
+            } catch (IOException ioe) {
+                // ignore and log
+                log.error(ioe.getMessage(), ioe);
+            }
+        }
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/AbstractFileStore.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStore.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStore.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStore.java Mon Oct 17 14:20:11 2016
@@ -18,20 +18,15 @@
  */
 package org.apache.jackrabbit.oak.segment.file;
 
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
 import static com.google.common.collect.Lists.newArrayList;
 import static com.google.common.collect.Lists.newArrayListWithCapacity;
 import static com.google.common.collect.Lists.newLinkedList;
-import static com.google.common.collect.Maps.newHashMap;
 import static com.google.common.collect.Maps.newLinkedHashMap;
 import static com.google.common.collect.Sets.newHashSet;
 import static java.lang.String.format;
 import static java.lang.System.currentTimeMillis;
 import static java.lang.Thread.currentThread;
 import static java.nio.ByteBuffer.wrap;
-import static java.util.Collections.emptyMap;
-import static java.util.Collections.singletonMap;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.SECONDS;
@@ -45,7 +40,6 @@ import static org.apache.jackrabbit.oak.
 import static org.apache.jackrabbit.oak.segment.file.TarRevisions.EXPEDITE_OPTION;
 import static org.apache.jackrabbit.oak.segment.file.TarRevisions.timeout;
 
-import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
 import java.io.RandomAccessFile;
@@ -55,7 +49,6 @@ import java.nio.channels.OverlappingFile
 import java.text.DecimalFormat;
 import java.text.DecimalFormatSymbols;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -68,8 +61,6 @@ import java.util.concurrent.atomic.Atomi
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import javax.annotation.CheckForNull;
 import javax.annotation.Nonnull;
@@ -81,25 +72,19 @@ import com.google.common.base.Stopwatch;
 import com.google.common.base.Supplier;
 import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableList;
+
 import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
 import org.apache.jackrabbit.oak.api.jmx.CacheStatsMBean;
 import org.apache.jackrabbit.oak.plugins.blob.ReferenceCollector;
 import org.apache.jackrabbit.oak.segment.BinaryReferenceConsumer;
-import org.apache.jackrabbit.oak.segment.CachingSegmentReader;
 import org.apache.jackrabbit.oak.segment.Compactor;
 import org.apache.jackrabbit.oak.segment.RecordId;
 import org.apache.jackrabbit.oak.segment.Segment;
 import org.apache.jackrabbit.oak.segment.SegmentBufferWriter;
-import org.apache.jackrabbit.oak.segment.SegmentCache;
-import org.apache.jackrabbit.oak.segment.SegmentGraph.SegmentGraphVisitor;
 import org.apache.jackrabbit.oak.segment.SegmentId;
-import org.apache.jackrabbit.oak.segment.SegmentIdFactory;
 import org.apache.jackrabbit.oak.segment.SegmentIdTable;
 import org.apache.jackrabbit.oak.segment.SegmentNodeState;
 import org.apache.jackrabbit.oak.segment.SegmentNotFoundException;
-import org.apache.jackrabbit.oak.segment.SegmentReader;
-import org.apache.jackrabbit.oak.segment.SegmentStore;
-import org.apache.jackrabbit.oak.segment.SegmentTracker;
 import org.apache.jackrabbit.oak.segment.SegmentWriter;
 import org.apache.jackrabbit.oak.segment.WriterCacheManager.Default;
 import org.apache.jackrabbit.oak.segment.compaction.SegmentGCOptions;
@@ -112,62 +97,27 @@ import org.slf4j.LoggerFactory;
 /**
  * The storage implementation for tar files.
  */
-public class FileStore implements SegmentStore, Closeable {
+public class FileStore extends AbstractFileStore {
 
     private static final Logger log = LoggerFactory.getLogger(FileStore.class);
 
     private static final int MB = 1024 * 1024;
 
-    /**
-     * This value can be used as an invalid store version, since the store
-     * version is defined to be strictly greater than zero.
-     */
-    private static final int INVALID_STORE_VERSION = 0;
-
-    /**
-     * The store version is an always incrementing number, strictly greater than
-     * zero, that is changed every time there is a backwards incompatible
-     * modification to the format of the segment store.
-     */
-    private static final int CURRENT_STORE_VERSION = 1;
-
-    private static final Pattern FILE_NAME_PATTERN =
-            Pattern.compile("(data|bulk)((0|[1-9][0-9]*)[0-9]{4})([a-z])?.tar");
-
-    static final String FILE_NAME_FORMAT = "data%05d%s.tar";
-
-    private static final String LOCK_FILE_NAME = "repo.lock";
-
-    private static final String MANIFEST_FILE_NAME = "manifest";
+    static final String LOCK_FILE_NAME = "repo.lock";
 
     /**
      * GC counter for logging purposes
      */
     private static final AtomicLong GC_COUNT = new AtomicLong(0);
 
-    static final boolean MEMORY_MAPPING_DEFAULT =
-            "64".equals(System.getProperty("sun.arch.data.model", "32"));
-
-    @Nonnull
-    private final SegmentTracker tracker;
-
     @Nonnull
     private final SegmentWriter segmentWriter;
 
     @Nonnull
-    private final CachingSegmentReader segmentReader;
-
-    @Nonnull
     private final BinaryReferenceConsumer binaryReferenceConsumer;
 
-    private final File directory;
-
-    private final BlobStore blobStore;
-
     private final int maxFileSize;
 
-    private final boolean memoryMapping;
-
     @Nonnull
     private final GarbageCollector garbageCollector;
 
@@ -179,8 +129,7 @@ public class FileStore implements Segmen
 
     private final FileLock lock;
 
-    @Nonnull
-    private final TarRevisions revisions;
+    private TarRevisions revisions;
 
     /**
      * Scheduler for running <em>short</em> background operations
@@ -198,7 +147,7 @@ public class FileStore implements Segmen
      * This flag is periodically updated by calling the {@code SegmentGCOptions}
      * at regular intervals.
      */
-    private final AtomicBoolean sufficientDiskSpace;
+    private final AtomicBoolean sufficientDiskSpace = new AtomicBoolean(true);
 
     /**
      * Flag signalling shutdown of the file store
@@ -209,45 +158,17 @@ public class FileStore implements Segmen
 
     private final FileStoreStats stats;
 
-    @Nonnull
-    private final SegmentCache segmentCache;
+    FileStore(final FileStoreBuilder builder) throws InvalidFileStoreVersionException, IOException {
+        super(builder);
 
-    private final SegmentIdFactory segmentIdFactory = new SegmentIdFactory() {
-
-        @Override
-        @Nonnull
-        public SegmentId newSegmentId(long msb, long lsb) {
-            return new SegmentId(FileStore.this, msb, lsb);
+        lockFile = new RandomAccessFile(new File(directory, LOCK_FILE_NAME), "rw");
+        try {
+            lock = lockFile.getChannel().lock();
+        } catch (OverlappingFileLockException ex) {
+            throw new IllegalStateException(directory.getAbsolutePath()
+                    + " is in use by another store.", ex);
         }
 
-    };
-
-    // FIXME OAK-4450: Properly split the FileStore into read-only and r/w variants
-    FileStore(final FileStoreBuilder builder, boolean readOnly) throws InvalidFileStoreVersionException, IOException {
-        this.directory = builder.getDirectory();
-        if (!readOnly) {
-            lockFile = new RandomAccessFile(new File(directory, LOCK_FILE_NAME), "rw");
-            try {
-                lock = lockFile.getChannel().lock();
-            } catch (OverlappingFileLockException ex) {
-                throw new IllegalStateException(directory.getAbsolutePath()
-                        + " is in use by another store.", ex);
-            }
-        } else {
-            lockFile = null;
-            lock = null;
-        }
-        this.tracker = new SegmentTracker();
-        this.revisions = builder.getRevisions();
-        this.blobStore = builder.getBlobStore();
-        this.segmentCache = new SegmentCache(builder.getSegmentCacheSize());
-        this.segmentReader = new CachingSegmentReader(new Supplier<SegmentWriter>() {
-            @Override
-            public SegmentWriter get() {
-                return segmentWriter;
-            }
-        }, blobStore, builder.getStringCacheSize(), builder.getTemplateCacheSize());
-
         this.binaryReferenceConsumer = new BinaryReferenceConsumer() {
             @Override
             public void consume(int generation, UUID segmentId, String binaryReference) {
@@ -271,7 +192,6 @@ public class FileStore implements Segmen
                 .with(builder.getCacheManager())
                 .build(this);
         this.maxFileSize = builder.getMaxFileSize() * MB;
-        this.memoryMapping = builder.getMemoryMapping();
         this.garbageCollector = new GarbageCollector(builder.getGcOptions(), builder.getGcListener(), new GCJournal(directory));
 
         Map<Integer, Map<Character, File>> map = collectFiles(directory);
@@ -288,114 +208,57 @@ public class FileStore implements Segmen
         Integer[] indices = map.keySet().toArray(new Integer[map.size()]);
         Arrays.sort(indices);
         for (int i = indices.length - 1; i >= 0; i--) {
-            if (!readOnly) {
-                readers.add(TarReader.open(map.get(indices[i]), memoryMapping));
-            } else {
-                // only try to read-only recover the latest file as that might
-                // be the *only* one still being accessed by a writer
-                boolean recover = i == indices.length - 1;
-                readers.add(TarReader.openRO(map.get(indices[i]),
-                        memoryMapping, recover));
-            }
+            readers.add(TarReader.open(map.get(indices[i]), memoryMapping));
         }
         this.stats = new FileStoreStats(builder.getStatsProvider(), this, size());
 
-        if (!readOnly) {
-            int writeNumber = 0;
-            if (indices.length > 0) {
-                writeNumber = indices[indices.length - 1] + 1;
-            }
-            this.tarWriter = new TarWriter(directory, stats, writeNumber);
-        } else {
-            this.tarWriter = null;
-        }
-
-        sufficientDiskSpace = new AtomicBoolean(true);
-
-        if (!readOnly) {
-            fileStoreScheduler.scheduleAtFixedRate(
-                    format("TarMK flush [%s]", directory), 5, SECONDS, new Runnable() {;
-                @Override
-                public void run() {
-                    try {
-                        flush();
-                    } catch (IOException e) {
-                        log.warn("Failed to flush the TarMK at {}", directory, e);
-                    }
-                }
-
-            });
-            fileStoreScheduler.scheduleAtFixedRate(
-                    format("TarMK filer reaper [%s]", directory), 5, SECONDS, new Runnable() {
-                        @Override
-                        public void run() {
-                            fileReaper.reap();
+        int writeNumber = 0;
+        if (indices.length > 0) {
+            writeNumber = indices[indices.length - 1] + 1;
+        }
+        this.tarWriter = new TarWriter(directory, stats, writeNumber);
+
+        fileStoreScheduler.scheduleAtFixedRate(
+                format("TarMK flush [%s]", directory), 5, SECONDS,
+                new Runnable() {
+                    @Override
+                    public void run() {
+                        try {
+                            flush();
+                        } catch (IOException e) {
+                            log.warn("Failed to flush the TarMK at {}",
+                                    directory, e);
                         }
-                    });
-            fileStoreScheduler.scheduleAtFixedRate(
-                    format("TarMK disk space check [%s]", directory), 1, MINUTES, new Runnable() {
-                        SegmentGCOptions gcOptions = builder.getGcOptions();
-                @Override
-                public void run() {
-                    checkDiskSpace(gcOptions);
-                }
-            });
-        }
+                    }
+                });
+        fileStoreScheduler.scheduleAtFixedRate(
+                format("TarMK filer reaper [%s]", directory), 5, SECONDS,
+                new Runnable() {
+                    @Override
+                    public void run() {
+                        fileReaper.reap();
+                    }
+                });
+        fileStoreScheduler.scheduleAtFixedRate(
+                format("TarMK disk space check [%s]", directory), 1, MINUTES,
+                new Runnable() {
+                    SegmentGCOptions gcOptions = builder.getGcOptions();
 
-        if (readOnly) {
-            log.info("TarMK ReadOnly opened: {} (mmap={})", directory,
-                    memoryMapping);
-        } else {
-            log.info("TarMK opened: {} (mmap={})", directory, memoryMapping);
-        }
+                    @Override
+                    public void run() {
+                        checkDiskSpace(gcOptions);
+                    }
+                });
+        log.info("TarMK opened: {} (mmap={})", directory, memoryMapping);
         log.debug("TarMK readers {}", this.readers);
     }
 
     FileStore bind(TarRevisions revisions) throws IOException {
-        revisions.bind(this, initialNode());
+        this.revisions = revisions;
+        this.revisions.bind(this, initialNode());
         return this;
     }
 
-    private File getManifestFile() {
-        return new File(directory, MANIFEST_FILE_NAME);
-    }
-
-    private Manifest openManifest() throws IOException {
-        File file = getManifestFile();
-
-        if (file.exists()) {
-            return Manifest.load(file);
-        }
-
-        return null;
-    }
-
-    private Manifest checkManifest(Manifest manifest) throws InvalidFileStoreVersionException {
-        if (manifest == null) {
-            throw new InvalidFileStoreVersionException("Using oak-segment-tar, but oak-segment should be used");
-        }
-
-        int storeVersion = manifest.getStoreVersion(INVALID_STORE_VERSION);
-
-        // A store version less than or equal to the highest invalid value means
-        // that something or someone is messing up with the manifest. This error
-        // is not recoverable and is thus represented as an ISE.
-
-        if (storeVersion <= INVALID_STORE_VERSION) {
-            throw new IllegalStateException("Invalid store version");
-        }
-
-        if (storeVersion < CURRENT_STORE_VERSION) {
-            throw new InvalidFileStoreVersionException("Using a too recent version of oak-segment-tar");
-        }
-
-        if (storeVersion > CURRENT_STORE_VERSION) {
-            throw new InvalidFileStoreVersionException("Using a too old version of oak-segment tar");
-        }
-
-        return manifest;
-    }
-
     private void saveManifest(Manifest manifest) throws IOException {
         manifest.setStoreVersion(CURRENT_STORE_VERSION);
         manifest.save(getManifestFile());
@@ -426,21 +289,6 @@ public class FileStore implements Segmen
         return revisions.getHead().getSegmentId().getGcGeneration();
     }
 
-    @Nonnull
-    public CacheStatsMBean getSegmentCacheStats() {
-        return segmentCache.getCacheStats();
-    }
-
-    @Nonnull
-    public CacheStatsMBean getStringCacheStats() {
-        return segmentReader.getStringCacheStats();
-    }
-
-    @Nonnull
-    public CacheStatsMBean getTemplateCacheStats() {
-        return segmentReader.getTemplateCacheStats();
-    }
-
     @CheckForNull
     public CacheStatsMBean getStringDeduplicationCacheStats() {
         return segmentWriter.getStringCacheStats();
@@ -472,77 +320,6 @@ public class FileStore implements Segmen
         });
     }
 
-    static Map<Integer, Map<Character, File>> collectFiles(File directory) {
-        Map<Integer, Map<Character, File>> dataFiles = newHashMap();
-        Map<Integer, File> bulkFiles = newHashMap();
-
-        for (File file : directory.listFiles()) {
-            Matcher matcher = FILE_NAME_PATTERN.matcher(file.getName());
-            if (matcher.matches()) {
-                Integer index = Integer.parseInt(matcher.group(2));
-                if ("data".equals(matcher.group(1))) {
-                    Map<Character, File> files = dataFiles.get(index);
-                    if (files == null) {
-                        files = newHashMap();
-                        dataFiles.put(index, files);
-                    }
-                    Character generation = 'a';
-                    if (matcher.group(4) != null) {
-                        generation = matcher.group(4).charAt(0);
-                    }
-                    checkState(files.put(generation, file) == null);
-                } else {
-                    checkState(bulkFiles.put(index, file) == null);
-                }
-            }
-        }
-
-        if (!bulkFiles.isEmpty()) {
-            log.info("Upgrading TarMK file names in {}", directory);
-
-            if (!dataFiles.isEmpty()) {
-                // first put all the data segments at the end of the list
-                Integer[] indices =
-                        dataFiles.keySet().toArray(new Integer[dataFiles.size()]);
-                Arrays.sort(indices);
-                int position = Math.max(
-                        indices[indices.length - 1] + 1,
-                        bulkFiles.size());
-                for (Integer index : indices) {
-                    Map<Character, File> files = dataFiles.remove(index);
-                    Integer newIndex = position++;
-                    for (Character generation : newHashSet(files.keySet())) {
-                        File file = files.get(generation);
-                        File newFile = new File(
-                                directory,
-                                format(FILE_NAME_FORMAT, newIndex, generation));
-                        log.info("Renaming {} to {}", file, newFile);
-                        file.renameTo(newFile);
-                        files.put(generation, newFile);
-                    }
-                    dataFiles.put(newIndex, files);
-                }
-            }
-
-            // then add all the bulk segments at the beginning of the list
-            Integer[] indices =
-                    bulkFiles.keySet().toArray(new Integer[bulkFiles.size()]);
-            Arrays.sort(indices);
-            int position = 0;
-            for (Integer index : indices) {
-                File file = bulkFiles.remove(index);
-                Integer newIndex = position++;
-                File newFile = new File(
-                        directory, format(FILE_NAME_FORMAT, newIndex, "a"));
-                log.info("Renaming {} to {}", file, newFile);
-                file.renameTo(newFile);
-                dataFiles.put(newIndex, singletonMap('a', newFile));
-            }
-        }
-
-        return dataFiles;
-    }
-
     /**
      * @return the size of this store. This method shouldn't be called from
      * a very tight loop as it contents with the {@link #fileStoreLock}.
@@ -675,68 +452,23 @@ public class FileStore implements Segmen
         garbageCollector.cancel();
     }
 
-    public Iterable<SegmentId> getSegmentIds() {
-        fileStoreLock.readLock().lock();
-        try {
-            List<SegmentId> ids = newArrayList();
-            if (tarWriter != null) {
-                for (UUID uuid : tarWriter.getUUIDs()) {
-                    long msb = uuid.getMostSignificantBits();
-                    long lsb = uuid.getLeastSignificantBits();
-                    ids.add(newSegmentId(msb, lsb));
-                }
-            }
-            for (TarReader reader : readers) {
-                for (UUID uuid : reader.getUUIDs()) {
-                    long msb = uuid.getMostSignificantBits();
-                    long lsb = uuid.getLeastSignificantBits();
-                    ids.add(newSegmentId(msb, lsb));
-                }
-            }
-            return ids;
-        } finally {
-            fileStoreLock.readLock().unlock();
-        }
-    }
-
-    @Nonnull
-    public SegmentTracker getTracker() {
-        return tracker;
-    }
-
+    @Override
     @Nonnull
     public SegmentWriter getWriter() {
         return segmentWriter;
     }
 
     @Nonnull
-    public SegmentReader getReader() {
-        return segmentReader;
-    }
-
-    @Nonnull
     public BinaryReferenceConsumer getBinaryReferenceConsumer() {
         return binaryReferenceConsumer;
     }
 
+    @Override
     @Nonnull
     public TarRevisions getRevisions() {
         return revisions;
     }
 
-    /**
-     * Convenience method for accessing the root node for the current head.
-     * This is equivalent to
-     * <pre>
-     * fileStore.getReader().readHeadState(fileStore.getRevisions())
-     * </pre>
-     * @return the current head node state
-     */
-    @Nonnull
-    public SegmentNodeState getHead() {
-        return segmentReader.readHeadState(revisions);
-    }
-
     @Override
     public void close() {
         // Flag the store as shutting / shut down
@@ -952,24 +684,6 @@ public class FileStore implements Segmen
         }
     }
 
-    @Override
-    @Nonnull
-    public SegmentId newSegmentId(long msb, long lsb) {
-        return tracker.newSegmentId(msb, lsb, segmentIdFactory);
-    }
-
-    @Override
-    @Nonnull
-    public SegmentId newBulkSegmentId() {
-        return tracker.newBulkSegmentId(segmentIdFactory);
-    }
-
-    @Override
-    @Nonnull
-    public SegmentId newDataSegmentId() {
-        return tracker.newDataSegmentId(segmentIdFactory);
-    }
-
     /**
      * Switch to a new tar writer.
      * This method may only be called when holding the write lock of {@link #fileStoreLock}
@@ -988,39 +702,6 @@ public class FileStore implements Segmen
         }
     }
 
-    /**
-     * @return  the external BlobStore (if configured) with this store, {@code null} otherwise.
-     */
-    @CheckForNull
-    public BlobStore getBlobStore() {
-        return blobStore;
-    }
-
-    public Map<String, Set<UUID>> getTarReaderIndex() {
-        Map<String, Set<UUID>> index = new HashMap<String, Set<UUID>>();
-        for (TarReader reader : readers) {
-            index.put(reader.getFile().getAbsolutePath(), reader.getUUIDs());
-        }
-        return index;
-    }
-
-    public Map<UUID, List<UUID>> getTarGraph(String fileName) throws IOException {
-        for (TarReader reader : readers) {
-            if (fileName.equals(reader.getFile().getName())) {
-                Map<UUID, List<UUID>> graph = newHashMap();
-                for (UUID uuid : reader.getUUIDs()) {
-                    graph.put(uuid, null);
-                }
-                Map<UUID, List<UUID>> g = reader.getGraph(false);
-                if (g != null) {
-                    graph.putAll(g);
-                }
-                return graph;
-            }
-        }
-        return emptyMap();
-    }
-
     private void checkDiskSpace(SegmentGCOptions gcOptions) {
         long repositoryDiskSpace = size();
         long availableDiskSpace = directory.getFreeSpace();
@@ -1040,123 +721,6 @@ public class FileStore implements Segmen
         }
     }
 
-    /**
-     * A read only {@link FileStore} implementation that supports
-     * going back to old revisions.
-     * <p>
-     * All write methods are no-ops.
-     */
-    public static class ReadOnlyStore extends FileStore {
-        private RecordId currentHead;
-
-        ReadOnlyStore(FileStoreBuilder builder) throws InvalidFileStoreVersionException, IOException {
-            super(builder, true);
-        }
-
-        @Override
-        ReadOnlyStore bind(@Nonnull TarRevisions revisions) throws IOException {
-            revisions.bind(this, new Supplier<RecordId>() {
-                @Override
-                public RecordId get() {
-                    throw new IllegalStateException("Cannot start readonly store from empty journal");
-                }
-            });
-            currentHead = revisions.getHead();
-            return this;
-        }
-
-        /**
-         * Go to the specified {@code revision}
-         *
-         * @param revision
-         */
-        public void setRevision(String revision) {
-            RecordId newHead = RecordId.fromString(this, revision);
-            if (super.revisions.setHead(currentHead, newHead)) {
-                currentHead = newHead;
-            }
-        }
-
-        /**
-         * Include the ids of all segments transitively reachable through forward references from
-         * {@code referencedIds}. See OAK-3864.
-         */
-        private static void includeForwardReferences(Iterable<TarReader> readers, Set<UUID> referencedIds)
-            throws IOException {
-            Set<UUID> fRefs = newHashSet(referencedIds);
-            do {
-                // Add direct forward references
-                for (TarReader reader : readers) {
-                    reader.calculateForwardReferences(fRefs);
-                    if (fRefs.isEmpty()) {
-                        break;  // Optimisation: bail out if no references left
-                    }
-                }
-                // ... as long as new forward references are found.
-            } while (referencedIds.addAll(fRefs));
-        }
-
-        /**
-         * Build the graph of segments reachable from an initial set of segments
-         * @param roots     the initial set of segments
-         * @param visitor   visitor receiving call back while following the segment graph
-         * @throws IOException
-         */
-        public void traverseSegmentGraph(
-            @Nonnull Set<UUID> roots,
-            @Nonnull SegmentGraphVisitor visitor) throws IOException {
-
-            List<TarReader> readers = super.readers;
-            includeForwardReferences(readers, roots);
-            for (TarReader reader : readers) {
-                reader.traverseSegmentGraph(checkNotNull(roots), checkNotNull(visitor));
-            }
-        }
-
-        @Override
-        public void writeSegment(SegmentId id, byte[] data,
-                int offset, int length) {
-            throw new UnsupportedOperationException("Read Only Store");
-        }
-
-        /**
-         * no-op
-         */
-        @Override
-        public void flush() { /* nop */ }
-
-        @Override
-        public void cleanup() {
-            throw new UnsupportedOperationException("Read Only Store");
-        }
-
-        @Override
-        public SafeRunnable getGCRunner() {
-            throw new UnsupportedOperationException("Read Only Store");
-        }
-
-        @Override
-        public boolean compact() {
-            throw new UnsupportedOperationException("Read Only Store");
-        }
-
-        @Override
-        public void gc() {
-            throw new UnsupportedOperationException("Read Only Store");
-        }
-    }
-
-    private static void closeAndLogOnFail(Closeable closeable) {
-        if (closeable != null) {
-            try {
-                closeable.close();
-            } catch (IOException ioe) {
-                // ignore and log
-                log.error(ioe.getMessage(), ioe);
-            }
-        }
-    }
-
     private class GarbageCollector {
         @Nonnull
         private final SegmentGCOptions gcOptions;
@@ -1411,6 +975,7 @@ public class FileStore implements Segmen
                                          Supplier<Boolean> cancel)
         throws IOException {
             if (gcOptions.isOffline()) {
+                BlobStore blobStore = getBlobStore();
                 SegmentWriter writer = new SegmentWriter(FileStore.this, segmentReader, blobStore, new Default(), bufferWriter, binaryReferenceConsumer);
                 return new Compactor(segmentReader, writer, blobStore, cancel, gcOptions)
                         .compact(EMPTY_NODE, head, EMPTY_NODE);

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStoreBuilder.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStoreBuilder.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStoreBuilder.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStoreBuilder.java Mon Oct 17 14:20:11 2016
@@ -36,13 +36,13 @@ import javax.annotation.CheckForNull;
 import javax.annotation.Nonnull;
 
 import com.google.common.base.Predicate;
+
 import org.apache.jackrabbit.oak.segment.RecordCache;
 import org.apache.jackrabbit.oak.segment.RecordId;
 import org.apache.jackrabbit.oak.segment.Template;
 import org.apache.jackrabbit.oak.segment.WriterCacheManager;
 import org.apache.jackrabbit.oak.segment.compaction.LoggingGCMonitor;
 import org.apache.jackrabbit.oak.segment.compaction.SegmentGCOptions;
-import org.apache.jackrabbit.oak.segment.file.FileStore.ReadOnlyStore;
 import org.apache.jackrabbit.oak.spi.blob.BlobStore;
 import org.apache.jackrabbit.oak.spi.gc.DelegatingGCMonitor;
 import org.apache.jackrabbit.oak.spi.gc.GCMonitor;
@@ -56,6 +56,9 @@ import org.slf4j.LoggerFactory;
 public class FileStoreBuilder {
     private static final Logger LOG = LoggerFactory.getLogger(FileStore.class);
 
+    private static final boolean MEMORY_MAPPING_DEFAULT =
+            "64".equals(System.getProperty("sun.arch.data.model", "32"));
+
     @Nonnull
     private final File directory;
 
@@ -76,7 +79,7 @@ public class FileStoreBuilder {
 
     private int nodeDeduplicationCacheSize = DEFAULT_NODE_CACHE_SIZE;
 
-    private boolean memoryMapping = FileStore.MEMORY_MAPPING_DEFAULT;
+    private boolean memoryMapping = MEMORY_MAPPING_DEFAULT;
 
     @Nonnull
     private StatisticsProvider statsProvider = StatisticsProvider.NOOP;
@@ -141,8 +144,7 @@ public class FileStoreBuilder {
         }
     };
 
-    @CheckForNull
-    private TarRevisions revisions;
+    private boolean built;
 
     /**
      * Create a new instance of a {@code FileStoreBuilder} for a file store.
@@ -264,7 +266,7 @@ public class FileStoreBuilder {
      */
     @Nonnull
     public FileStoreBuilder withDefaultMemoryMapping() {
-        this.memoryMapping = FileStore.MEMORY_MAPPING_DEFAULT;
+        this.memoryMapping = MEMORY_MAPPING_DEFAULT;
         return this;
     }
 
@@ -320,15 +322,16 @@ public class FileStoreBuilder {
      */
     @Nonnull
     public FileStore build() throws InvalidFileStoreVersionException, IOException {
-        checkState(revisions == null, "Cannot re-use builder");
+        checkState(!built, "Cannot re-use builder");
+        built = true;
         directory.mkdirs();
-        revisions = new TarRevisions(false, directory);
+        TarRevisions revisions = new TarRevisions(directory);
         LOG.info("Creating file store {}", this);
-        return new FileStore(this, false).bind(revisions);
+        return new FileStore(this).bind(revisions);
     }
 
     /**
-     * Create a new {@link ReadOnlyStore} instance with the settings specified in this
+     * Create a new {@link ReadOnlyFileStore} instance with the settings specified in this
      * builder. If none of the {@code with} methods have been called before calling
      * this method, a file store with the following default settings is returned:
      * <ul>
@@ -345,12 +348,13 @@ public class FileStoreBuilder {
      * @throws IOException
      */
     @Nonnull
-    public ReadOnlyStore buildReadOnly() throws InvalidFileStoreVersionException, IOException {
-        checkState(revisions == null, "Cannot re-use builder");
+    public ReadOnlyFileStore buildReadOnly() throws InvalidFileStoreVersionException, IOException {
+        checkState(!built, "Cannot re-use builder");
         checkState(directory.exists() && directory.isDirectory());
-        revisions = new TarRevisions(true, directory);
+        built = true;
+        ReadOnlyRevisions revisions = new ReadOnlyRevisions(directory);
         LOG.info("Creating file store {}", this);
-        return new ReadOnlyStore(this).bind(revisions);
+        return new ReadOnlyFileStore(this).bind(revisions);
     }
 
     @Nonnull
@@ -399,12 +403,6 @@ public class FileStoreBuilder {
     }
 
     @Nonnull
-    TarRevisions getRevisions() {
-        checkState(revisions != null, "File store not yet built");
-        return revisions;
-    }
-
-    @Nonnull
     WriterCacheManager getCacheManager() {
         if (cacheManager == null) {
             cacheManager = new EvictingWriteCacheManager(

Added: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/ReadOnlyFileStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/ReadOnlyFileStore.java?rev=1765293&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/ReadOnlyFileStore.java (added)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/ReadOnlyFileStore.java Mon Oct 17 14:20:11 2016
@@ -0,0 +1,260 @@
+/*
+ * 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.segment.file;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.collect.Lists.newArrayList;
+import static com.google.common.collect.Lists.newArrayListWithCapacity;
+import static com.google.common.collect.Maps.newHashMap;
+import static com.google.common.collect.Sets.newHashSet;
+import static java.util.Collections.emptyMap;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+
+import javax.annotation.Nonnull;
+
+import org.apache.jackrabbit.oak.segment.RecordId;
+import org.apache.jackrabbit.oak.segment.Segment;
+import org.apache.jackrabbit.oak.segment.SegmentGraph.SegmentGraphVisitor;
+import org.apache.jackrabbit.oak.segment.SegmentId;
+import org.apache.jackrabbit.oak.segment.SegmentNotFoundException;
+import org.apache.jackrabbit.oak.segment.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A read only {@link AbstractFileStore} implementation that supports going back
+ * to old revisions.
+ * <p>
+ * All write methods are no-ops.
+ */
+public class ReadOnlyFileStore extends AbstractFileStore {
+
+    private static final Logger log = LoggerFactory
+            .getLogger(ReadOnlyFileStore.class);
+
+    private final List<TarReader> readers;
+
+    private ReadOnlyRevisions revisions;
+
+    private RecordId currentHead;
+
+    ReadOnlyFileStore(FileStoreBuilder builder)
+            throws InvalidFileStoreVersionException, IOException {
+        super(builder);
+
+        Map<Integer, Map<Character, File>> map = collectFiles(directory);
+
+        if (map.size() > 0) {
+            checkManifest(openManifest());
+        }
+
+        this.readers = newArrayListWithCapacity(map.size());
+        Integer[] indices = map.keySet().toArray(new Integer[map.size()]);
+        Arrays.sort(indices);
+        for (int i = indices.length - 1; i >= 0; i--) {
+            // only try to read-only recover the latest file as that might
+            // be the *only* one still being accessed by a writer
+            boolean recover = i == indices.length - 1;
+            readers.add(TarReader.openRO(map.get(indices[i]), memoryMapping,
+                    recover));
+        }
+        log.info("TarMK ReadOnly opened: {} (mmap={})", directory,
+                memoryMapping);
+    }
+
+    ReadOnlyFileStore bind(@Nonnull ReadOnlyRevisions revisions)
+            throws IOException {
+        this.revisions = revisions;
+        this.revisions.bind(this);
+        currentHead = revisions.getHead();
+        return this;
+    }
+
+    /**
+     * Go to the specified {@code revision}
+     * 
+     * @param revision
+     */
+    public void setRevision(String revision) {
+        RecordId newHead = RecordId.fromString(this, revision);
+        if (revisions.setHead(currentHead, newHead)) {
+            currentHead = newHead;
+        }
+    }
+
+    /**
+     * Include the ids of all segments transitively reachable through forward
+     * references from {@code referencedIds}. See OAK-3864.
+     */
+    private static void includeForwardReferences(Iterable<TarReader> readers,
+            Set<UUID> referencedIds) throws IOException {
+        Set<UUID> fRefs = newHashSet(referencedIds);
+        do {
+            // Add direct forward references
+            for (TarReader reader : readers) {
+                reader.calculateForwardReferences(fRefs);
+                if (fRefs.isEmpty()) {
+                    break; // Optimisation: bail out if no references left
+                }
+            }
+            // ... as long as new forward references are found.
+        } while (referencedIds.addAll(fRefs));
+    }
+
+    /**
+     * Build the graph of segments reachable from an initial set of segments
+     * 
+     * @param roots
+     *            the initial set of segments
+     * @param visitor
+     *            visitor receiving call back while following the segment graph
+     * @throws IOException
+     */
+    public void traverseSegmentGraph(@Nonnull Set<UUID> roots,
+            @Nonnull SegmentGraphVisitor visitor) throws IOException {
+
+        List<TarReader> readers = this.readers;
+        includeForwardReferences(readers, roots);
+        for (TarReader reader : readers) {
+            reader.traverseSegmentGraph(checkNotNull(roots),
+                    checkNotNull(visitor));
+        }
+    }
+
+    @Override
+    public void writeSegment(SegmentId id, byte[] data, int offset, int length) {
+        throw new UnsupportedOperationException("Read Only Store");
+    }
+
+    @Override
+    public boolean containsSegment(SegmentId id) {
+        long msb = id.getMostSignificantBits();
+        long lsb = id.getLeastSignificantBits();
+        for (TarReader reader : readers) {
+            if (reader.containsEntry(msb, lsb)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    @Nonnull
+    public Segment readSegment(final SegmentId id) {
+        try {
+            return segmentCache.getSegment(id, new Callable<Segment>() {
+                @Override
+                public Segment call() throws Exception {
+                    long msb = id.getMostSignificantBits();
+                    long lsb = id.getLeastSignificantBits();
+
+                    for (TarReader reader : readers) {
+                        try {
+                            ByteBuffer buffer = reader.readEntry(msb, lsb);
+                            if (buffer != null) {
+                                return new Segment(ReadOnlyFileStore.this, segmentReader, id, buffer);
+                            }
+                        } catch (IOException e) {
+                            log.warn("Failed to read from tar file {}", reader, e);
+                        }
+                    }
+                    throw new SegmentNotFoundException(id);
+                }
+            });
+        } catch (ExecutionException e) {
+            throw e.getCause() instanceof SegmentNotFoundException
+                ? (SegmentNotFoundException) e.getCause()
+                : new SegmentNotFoundException(id, e);
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            revisions.close();
+            for (TarReader reader : readers) {
+                closeAndLogOnFail(reader);
+            }
+        } catch (IOException e) {
+            throw new RuntimeException("Failed to close the TarMK at "
+                    + directory, e);
+        }
+        System.gc(); // for any memory-mappings that are no longer used
+        log.info("TarMK closed: {}", directory);
+    }
+
+    @Override
+    public SegmentWriter getWriter() {
+        return null;
+    }
+
+    public Map<String, Set<UUID>> getTarReaderIndex() {
+        Map<String, Set<UUID>> index = new HashMap<String, Set<UUID>>();
+        for (TarReader reader : readers) {
+            index.put(reader.getFile().getAbsolutePath(), reader.getUUIDs());
+        }
+        return index;
+    }
+
+    public Map<UUID, List<UUID>> getTarGraph(String fileName)
+            throws IOException {
+        for (TarReader reader : readers) {
+            if (fileName.equals(reader.getFile().getName())) {
+                Map<UUID, List<UUID>> graph = newHashMap();
+                for (UUID uuid : reader.getUUIDs()) {
+                    graph.put(uuid, null);
+                }
+                Map<UUID, List<UUID>> g = reader.getGraph(false);
+                if (g != null) {
+                    graph.putAll(g);
+                }
+                return graph;
+            }
+        }
+        return emptyMap();
+    }
+
+    public Iterable<SegmentId> getSegmentIds() {
+        List<SegmentId> ids = newArrayList();
+        for (TarReader reader : readers) {
+            for (UUID uuid : reader.getUUIDs()) {
+                long msb = uuid.getMostSignificantBits();
+                long lsb = uuid.getLeastSignificantBits();
+                ids.add(newSegmentId(msb, lsb));
+            }
+        }
+        return ids;
+    }
+
+    @Override
+    public ReadOnlyRevisions getRevisions() {
+        return revisions;
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/ReadOnlyFileStore.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/ReadOnlyRevisions.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/ReadOnlyRevisions.java?rev=1765293&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/ReadOnlyRevisions.java (added)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/ReadOnlyRevisions.java Mon Oct 17 14:20:11 2016
@@ -0,0 +1,138 @@
+/*
+ * 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.segment.file;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.concurrent.atomic.AtomicReference;
+
+import javax.annotation.Nonnull;
+
+import org.apache.jackrabbit.oak.segment.RecordId;
+import org.apache.jackrabbit.oak.segment.Revisions;
+import org.apache.jackrabbit.oak.segment.SegmentStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Function;
+
+public class ReadOnlyRevisions implements Revisions, Closeable {
+
+    private static final Logger LOG = LoggerFactory
+            .getLogger(ReadOnlyRevisions.class);
+
+    public static final String JOURNAL_FILE_NAME = "journal.log";
+
+    @Nonnull
+    private final AtomicReference<RecordId> head;
+
+    @Nonnull
+    private final File directory;
+
+    @Nonnull
+    private final RandomAccessFile journalFile;
+
+    public ReadOnlyRevisions(@Nonnull File directory) throws IOException {
+        this.directory = checkNotNull(directory);
+        this.journalFile = new RandomAccessFile(new File(directory,
+                JOURNAL_FILE_NAME), "r");
+        this.journalFile.seek(journalFile.length());
+        this.head = new AtomicReference<>(null);
+    }
+
+    /**
+     * Bind this instance to a store.
+     * 
+     * @param store
+     *            store to bind to
+     * @param writeInitialNode
+     *            provider for the initial node in case the journal is empty.
+     * @throws IOException
+     */
+    synchronized void bind(@Nonnull SegmentStore store) throws IOException {
+        if (head.get() == null) {
+            RecordId persistedId = null;
+            try (JournalReader journalReader = new JournalReader(new File(
+                    directory, JOURNAL_FILE_NAME))) {
+                while (persistedId == null && journalReader.hasNext()) {
+                    String entry = journalReader.next();
+                    try {
+                        RecordId id = RecordId.fromString(store, entry);
+                        if (store.containsSegment(id.getSegmentId())) {
+                            persistedId = id;
+                        } else {
+                            LOG.warn(
+                                    "Unable to access revision {}, rewinding...",
+                                    id);
+                        }
+                    } catch (IllegalArgumentException ignore) {
+                        LOG.warn("Skipping invalid record id {}", entry);
+                    }
+                }
+            }
+
+            if (persistedId == null) {
+                throw new IllegalStateException(
+                        "Cannot start readonly store from empty journal");
+            } else {
+                head.set(persistedId);
+            }
+        }
+    }
+
+    private void checkBound() {
+        checkState(head.get() != null, "Revisions not bound to a store");
+    }
+
+    @Nonnull
+    @Override
+    public RecordId getHead() {
+        checkBound();
+        return head.get();
+    }
+
+    @Override
+    public boolean setHead(RecordId expected, RecordId head, Option... options) {
+        checkBound();
+        RecordId id = this.head.get();
+        return id.equals(expected) && this.head.compareAndSet(id, head);
+    }
+
+    @Override
+    public boolean setHead(Function<RecordId, RecordId> newHead,
+            Option... options) throws InterruptedException {
+        throw new UnsupportedOperationException("ReadOnly Revisions");
+    }
+
+    /**
+     * Close the underlying journal file.
+     * 
+     * @throws IOException
+     */
+    @Override
+    public void close() throws IOException {
+        journalFile.close();
+    }
+
+}

Propchange: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/ReadOnlyRevisions.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/TarRevisions.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/TarRevisions.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/TarRevisions.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/TarRevisions.java Mon Oct 17 14:20:11 2016
@@ -46,7 +46,6 @@ import com.google.common.base.Supplier;
 import org.apache.jackrabbit.oak.segment.RecordId;
 import org.apache.jackrabbit.oak.segment.Revisions;
 import org.apache.jackrabbit.oak.segment.SegmentStore;
-import org.apache.jackrabbit.oak.segment.file.FileStore.ReadOnlyStore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -129,21 +128,16 @@ public class TarRevisions implements Rev
         return new TimeOutOption(time, unit);
     }
 
-    // FIXME OAK-4465: Remove the read-only concern from TarRevisions: this should be possible once
-    // the ReadOnlyStore is properly separated from the FileStore. See OAK-4450.
     /**
      * Create a new instance placing the journal log file into the passed
      * {@code directory}.
-     * @param readOnly      safeguard for {@link ReadOnlyStore}: open the journal
-     *                      file in read only mode.
      * @param directory     directory of the journal file
      * @throws IOException
      */
-    public TarRevisions(boolean readOnly, @Nonnull File directory)
-    throws IOException {
+    public TarRevisions(@Nonnull File directory) throws IOException {
         this.directory = checkNotNull(directory);
-        this.journalFile = new RandomAccessFile(new File(directory, JOURNAL_FILE_NAME),
-                readOnly ? "r" : "rw");
+        this.journalFile = new RandomAccessFile(new File(directory,
+                JOURNAL_FILE_NAME), "rw");
         this.journalFile.seek(journalFile.length());
         this.head = new AtomicReference<>(null);
         this.persistedHead = new AtomicReference<>(null);

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/tooling/ConsistencyChecker.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/tooling/ConsistencyChecker.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/tooling/ConsistencyChecker.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/tooling/ConsistencyChecker.java Mon Oct 17 14:20:11 2016
@@ -42,9 +42,9 @@ import org.apache.jackrabbit.oak.api.Typ
 import org.apache.jackrabbit.oak.segment.SegmentBlob;
 import org.apache.jackrabbit.oak.segment.SegmentNodeStoreBuilders;
 import org.apache.jackrabbit.oak.segment.file.FileStore;
-import org.apache.jackrabbit.oak.segment.file.FileStore.ReadOnlyStore;
 import org.apache.jackrabbit.oak.segment.file.InvalidFileStoreVersionException;
 import org.apache.jackrabbit.oak.segment.file.JournalReader;
+import org.apache.jackrabbit.oak.segment.file.ReadOnlyFileStore;
 import org.apache.jackrabbit.oak.spi.state.ChildNodeEntry;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 import org.slf4j.Logger;
@@ -58,7 +58,7 @@ import org.slf4j.LoggerFactory;
 public class ConsistencyChecker implements Closeable {
     private static final Logger LOG = LoggerFactory.getLogger(ConsistencyChecker.class);
 
-    private final ReadOnlyStore store;
+    private final ReadOnlyFileStore store;
     private final long debugInterval;
 
     /**

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/tooling/RevisionHistory.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/tooling/RevisionHistory.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/tooling/RevisionHistory.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/tooling/RevisionHistory.java Mon Oct 17 14:20:11 2016
@@ -37,8 +37,7 @@ import com.google.common.collect.Iterato
 import org.apache.jackrabbit.oak.json.BlobSerializer;
 import org.apache.jackrabbit.oak.json.JsonSerializer;
 import org.apache.jackrabbit.oak.segment.SegmentNodeState;
-import org.apache.jackrabbit.oak.segment.file.FileStore;
-import org.apache.jackrabbit.oak.segment.file.FileStore.ReadOnlyStore;
+import org.apache.jackrabbit.oak.segment.file.ReadOnlyFileStore;
 import org.apache.jackrabbit.oak.segment.file.InvalidFileStoreVersionException;
 import org.apache.jackrabbit.oak.segment.file.JournalReader;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
@@ -47,10 +46,10 @@ import org.apache.jackrabbit.oak.spi.sta
  * Utility for tracing a node back through the revision history.
  */
 public class RevisionHistory {
-    private final ReadOnlyStore store;
+    private final ReadOnlyFileStore store;
 
     /**
-     * Create a new instance for a {@link FileStore} in the given {@code directory}.
+     * Create a new instance for a {@link ReadOnlyFileStore} in the given {@code directory}.
      *
      * @param directory
      * @throws IOException

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/Backup.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/Backup.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/Backup.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/Backup.java Mon Oct 17 14:20:11 2016
@@ -26,8 +26,8 @@ import java.io.IOException;
 
 import org.apache.jackrabbit.oak.backup.FileStoreBackup;
 import org.apache.jackrabbit.oak.backup.impl.FileStoreBackupImpl;
-import org.apache.jackrabbit.oak.segment.file.FileStore;
 import org.apache.jackrabbit.oak.segment.file.InvalidFileStoreVersionException;
+import org.apache.jackrabbit.oak.segment.file.ReadOnlyFileStore;
 
 /**
  * Perform a backup of a segment store into a specified folder.
@@ -128,14 +128,14 @@ public class Backup implements Runnable
 
     @Override
     public void run() {
-        try (FileStore fs = newFileStore()) {
+        try (ReadOnlyFileStore fs = newFileStore()) {
             fileStoreBackup.backup(fs.getReader(), fs.getRevisions(), target);
         } catch (Exception e) {
             e.printStackTrace();
         }
     }
 
-    private FileStore newFileStore() throws IOException, InvalidFileStoreVersionException {
+    private ReadOnlyFileStore newFileStore() throws IOException, InvalidFileStoreVersionException {
         if (fakeBlobStore) {
             return openReadOnlyFileStore(source, newBasicReadOnlyBlobStore());
         }

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugSegments.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugSegments.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugSegments.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugSegments.java Mon Oct 17 14:20:11 2016
@@ -35,7 +35,7 @@ import org.apache.jackrabbit.oak.json.Js
 import org.apache.jackrabbit.oak.segment.RecordId;
 import org.apache.jackrabbit.oak.segment.SegmentId;
 import org.apache.jackrabbit.oak.segment.SegmentNodeState;
-import org.apache.jackrabbit.oak.segment.file.FileStore;
+import org.apache.jackrabbit.oak.segment.file.ReadOnlyFileStore;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 
 /**
@@ -131,20 +131,20 @@ public class DebugSegments implements Ru
 
     @Override
     public void run() {
-        try (FileStore store = openReadOnlyFileStore(path)) {
+        try (ReadOnlyFileStore store = openReadOnlyFileStore(path)) {
             debugSegments(store);
         } catch (Exception e) {
             e.printStackTrace();
         }
     }
 
-    private void debugSegments(FileStore store) {
+    private void debugSegments(ReadOnlyFileStore store) {
         for (String segment : segments) {
             debugSegment(store, segment);
         }
     }
 
-    private void debugSegment(FileStore store, String segment) {
+    private void debugSegment(ReadOnlyFileStore store, String segment) {
         Matcher matcher = SEGMENT_REGEX.matcher(segment);
 
         if (!matcher.matches()) {

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugStore.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugStore.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugStore.java Mon Oct 17 14:20:11 2016
@@ -35,13 +35,14 @@ import java.util.UUID;
 
 import com.google.common.collect.Maps;
 import com.google.common.collect.Queues;
+
 import org.apache.jackrabbit.oak.segment.RecordId;
 import org.apache.jackrabbit.oak.segment.RecordType;
 import org.apache.jackrabbit.oak.segment.RecordUsageAnalyser;
 import org.apache.jackrabbit.oak.segment.Segment;
 import org.apache.jackrabbit.oak.segment.Segment.RecordConsumer;
 import org.apache.jackrabbit.oak.segment.SegmentId;
-import org.apache.jackrabbit.oak.segment.file.FileStore;
+import org.apache.jackrabbit.oak.segment.file.ReadOnlyFileStore;
 
 /**
  * Print debugging information about a segment store.
@@ -99,14 +100,14 @@ public class DebugStore implements Runna
 
     @Override
     public void run() {
-        try (FileStore store = openReadOnlyFileStore(path)) {
+        try (ReadOnlyFileStore store = openReadOnlyFileStore(path)) {
             debugFileStore(store);
         } catch (Exception e) {
             e.printStackTrace();
         }
     }
 
-    private static List<SegmentId> getReferencedSegmentIds(FileStore store, Segment segment) {
+    private static List<SegmentId> getReferencedSegmentIds(ReadOnlyFileStore store, Segment segment) {
         List<SegmentId> result = new ArrayList<>();
 
         for (int i = 0; i < segment.getReferencedSegmentIdCount(); i++) {
@@ -119,7 +120,7 @@ public class DebugStore implements Runna
         return result;
     }
 
-    private static void debugFileStore(FileStore store) {
+    private static void debugFileStore(ReadOnlyFileStore store) {
         Map<SegmentId, List<SegmentId>> idmap = Maps.newHashMap();
         int dataCount = 0;
         long dataSize = 0;

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugTars.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugTars.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugTars.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/DebugTars.java Mon Oct 17 14:20:11 2016
@@ -35,6 +35,7 @@ import java.util.UUID;
 import javax.jcr.PropertyType;
 
 import com.google.common.escape.Escapers;
+
 import org.apache.jackrabbit.oak.api.Blob;
 import org.apache.jackrabbit.oak.api.PropertyState;
 import org.apache.jackrabbit.oak.api.Type;
@@ -43,7 +44,7 @@ import org.apache.jackrabbit.oak.segment
 import org.apache.jackrabbit.oak.segment.SegmentId;
 import org.apache.jackrabbit.oak.segment.SegmentNodeState;
 import org.apache.jackrabbit.oak.segment.SegmentPropertyState;
-import org.apache.jackrabbit.oak.segment.file.FileStore;
+import org.apache.jackrabbit.oak.segment.file.ReadOnlyFileStore;
 import org.apache.jackrabbit.oak.spi.state.ChildNodeEntry;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 
@@ -128,20 +129,20 @@ public class DebugTars implements Runnab
 
     @Override
     public void run() {
-        try (FileStore store = openReadOnlyFileStore(path)) {
+        try (ReadOnlyFileStore store = openReadOnlyFileStore(path)) {
             debugTarFiles(store);
         } catch (Exception e) {
             e.printStackTrace();
         }
     }
 
-    private void debugTarFiles(FileStore store) {
+    private void debugTarFiles(ReadOnlyFileStore store) {
         for (String tar : tars) {
             debugTarFile(store, tar);
         }
     }
 
-    private void debugTarFile(FileStore store, String t) {
+    private void debugTarFile(ReadOnlyFileStore store, String t) {
         File tar = new File(path, t);
 
         if (!tar.exists()) {

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/Diff.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/Diff.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/Diff.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/Diff.java Mon Oct 17 14:20:11 2016
@@ -33,7 +33,7 @@ import java.util.List;
 
 import org.apache.jackrabbit.oak.segment.RecordId;
 import org.apache.jackrabbit.oak.segment.SegmentNotFoundException;
-import org.apache.jackrabbit.oak.segment.file.FileStore.ReadOnlyStore;
+import org.apache.jackrabbit.oak.segment.file.ReadOnlyFileStore;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 
 /**
@@ -211,7 +211,7 @@ public class Diff implements Runnable {
             return;
         }
 
-        try (ReadOnlyStore store = fileStoreBuilder(path).withBlobStore(newBasicReadOnlyBlobStore()).buildReadOnly()) {
+        try (ReadOnlyFileStore store = fileStoreBuilder(path).withBlobStore(newBasicReadOnlyBlobStore()).buildReadOnly()) {
             RecordId idL;
 
             try {
@@ -281,7 +281,7 @@ public class Diff implements Runnable {
         }
     }
 
-    private boolean diff(ReadOnlyStore store, RecordId idL, RecordId idR, PrintWriter pw) throws IOException {
+    private boolean diff(ReadOnlyFileStore store, RecordId idL, RecordId idR, PrintWriter pw) throws IOException {
         pw.println("rev " + idL + ".." + idR);
         try {
             NodeState before = store.getReader().readNode(idL).getChildNode("root");

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/GenerationGraph.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/GenerationGraph.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/GenerationGraph.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/GenerationGraph.java Mon Oct 17 14:20:11 2016
@@ -24,7 +24,8 @@ import static org.apache.jackrabbit.oak.
 import java.io.File;
 import java.io.OutputStream;
 
-import org.apache.jackrabbit.oak.segment.file.FileStore.ReadOnlyStore;
+import org.apache.jackrabbit.oak.segment.file.ReadOnlyFileStore;
+
 
 /**
  * Generates a garbage collection generation graph. The graph is written in <a
@@ -103,7 +104,7 @@ public class GenerationGraph implements
 
     @Override
     public void run() {
-        try (ReadOnlyStore store = openReadOnlyFileStore(path)) {
+        try (ReadOnlyFileStore store = openReadOnlyFileStore(path)) {
             writeGCGraph(store, out);
         } catch (Exception e) {
             e.printStackTrace();

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/SegmentGraph.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/SegmentGraph.java?rev=1765293&r1=1765292&r2=1765293&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/SegmentGraph.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/tool/SegmentGraph.java Mon Oct 17 14:20:11 2016
@@ -25,7 +25,7 @@ import java.io.File;
 import java.io.OutputStream;
 import java.util.Date;
 
-import org.apache.jackrabbit.oak.segment.file.FileStore.ReadOnlyStore;
+import org.apache.jackrabbit.oak.segment.file.ReadOnlyFileStore;
 
 /**
  * Generates a segment collection generation graph. The graph is written in <a
@@ -134,7 +134,7 @@ public class SegmentGraph implements Run
 
     @Override
     public void run() {
-        try (ReadOnlyStore store = openReadOnlyFileStore(path)) {
+        try (ReadOnlyFileStore store = openReadOnlyFileStore(path)) {
             writeSegmentGraph(store, out, epoch, filter);
         } catch (Exception e) {
             e.printStackTrace();