]> source.dussan.org Git - jgit.git/commitdiff
reftable: split off generic code from DFS code 01/147301/49
authorHan-Wen Nienhuys <hanwen@google.com>
Thu, 26 Sep 2019 11:40:26 +0000 (13:40 +0200)
committerHan-Wen Nienhuys <hanwen@google.com>
Wed, 16 Oct 2019 17:44:30 +0000 (19:44 +0200)
This introduces ReftableBatchRefUpdate and ReftableDatabase, as
generic classes, with some code moved to DfsReftableBatchRefUpdate and
DfsReftableDatabase.

Clarify thread-safety requirements by asserting locked status in
accessors, and acquiring locks in callers. This does not fix threading
problems, because ReftableBatchRefUpdate already wraps the whole
transaction in a lock.

This also fixes a number of bugs in ReftableBatchRefUpdate:

* non-atomic updates should not bail on first failure

* isNameConflicting should also check for conflicts between names that
  are added and removed in the BatchRefUpdate.

Change-Id: I5ec91173ea9a0aa19da444c8c0b2e0f4e8f88798
Signed-off-by: Han-Wen Nienhuys <hanwen@google.com>
Signed-off-by: Matthias Sohn <matthias.sohn@sap.com>
org.eclipse.jgit.http.test/src/org/eclipse/jgit/http/test/RefsUnreadableInMemoryRepository.java
org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsReftableBatchRefUpdate.java [new file with mode: 0644]
org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsReftableDatabase.java
org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/ReftableBatchRefUpdate.java [deleted file]
org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/reftable/ReftableBatchRefUpdate.java [new file with mode: 0644]
org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/reftable/ReftableDatabase.java [new file with mode: 0644]

index 78f909eeac612abdce0d1dcd8431446f23d909a6..6f8597978554f68026eb40b25dc007dddc6eae7a 100644 (file)
 package org.eclipse.jgit.http.test;
 
 import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import org.eclipse.jgit.internal.storage.dfs.DfsRepositoryDescription;
 import org.eclipse.jgit.internal.storage.dfs.InMemoryRepository;
-import org.eclipse.jgit.internal.storage.reftable.Reftable;
+import org.eclipse.jgit.lib.ObjectId;
+import org.eclipse.jgit.lib.Ref;
 import org.eclipse.jgit.lib.RefDatabase;
 
 /**
@@ -82,12 +86,43 @@ class RefsUnreadableInMemoryRepository extends InMemoryRepository {
        }
 
        private class RefsUnreadableRefDatabase extends MemRefDatabase {
+
+               /** {@inheritDoc} */
+               @Override
+               public Ref exactRef(String name) throws IOException {
+                       if (failing) {
+                               throw new IOException("disk failed, no refs found");
+                       }
+                       return super.exactRef(name);
+               }
+
+               /** {@inheritDoc} */
+               @Override
+               public Map<String, Ref> getRefs(String prefix) throws IOException {
+                       if (failing) {
+                               throw new IOException("disk failed, no refs found");
+                       }
+
+                       return super.getRefs(prefix);
+               }
+
+               /** {@inheritDoc} */
+               @Override
+               public List<Ref> getRefsByPrefix(String prefix) throws IOException {
+                       if (failing) {
+                               throw new IOException("disk failed, no refs found");
+                       }
+
+                       return super.getRefsByPrefix(prefix);
+               }
+
+               /** {@inheritDoc} */
                @Override
-               protected Reftable reader() throws IOException {
+               public Set<Ref> getTipsWithSha1(ObjectId id) throws IOException {
                        if (failing) {
                                throw new IOException("disk failed, no refs found");
                        }
-                       return super.reader();
+                       return super.getTipsWithSha1(id);
                }
        }
 }
diff --git a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsReftableBatchRefUpdate.java b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsReftableBatchRefUpdate.java
new file mode 100644 (file)
index 0000000..124630e
--- /dev/null
@@ -0,0 +1,196 @@
+/*
+ * Copyright (C) 2019, Google Inc.
+ * and other copyright owners as documented in the project's IP log.
+ *
+ * This program and the accompanying materials are made available
+ * under the terms of the Eclipse Distribution License v1.0 which
+ * accompanies this distribution, is reproduced below, and is
+ * available at http://www.eclipse.org/org/documents/edl-v10.php
+ *
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or
+ * without modification, are permitted provided that the following
+ * conditions are met:
+ *
+ * - Redistributions of source code must retain the above copyright
+ *   notice, this list of conditions and the following disclaimer.
+ *
+ * - Redistributions in binary form must reproduce the above
+ *   copyright notice, this list of conditions and the following
+ *   disclaimer in the documentation and/or other materials provided
+ *   with the distribution.
+ *
+ * - Neither the name of the Eclipse Foundation, Inc. nor the
+ *   names of its contributors may be used to endorse or promote
+ *   products derived from this software without specific prior
+ *   written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND
+ * CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES,
+ * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
+ * CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+ * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF
+ * ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+package org.eclipse.jgit.internal.storage.dfs;
+
+import org.eclipse.jgit.internal.storage.dfs.DfsObjDatabase.PackSource;
+import org.eclipse.jgit.internal.storage.io.BlockSource;
+import org.eclipse.jgit.internal.storage.pack.PackExt;
+import org.eclipse.jgit.internal.storage.reftable.ReftableBatchRefUpdate;
+import org.eclipse.jgit.internal.storage.reftable.ReftableCompactor;
+import org.eclipse.jgit.internal.storage.reftable.ReftableConfig;
+import org.eclipse.jgit.internal.storage.reftable.ReftableReader;
+import org.eclipse.jgit.internal.storage.reftable.ReftableWriter;
+import org.eclipse.jgit.lib.Ref;
+import org.eclipse.jgit.transport.ReceiveCommand;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import static org.eclipse.jgit.internal.storage.pack.PackExt.REFTABLE;
+
+/**
+ * {@link org.eclipse.jgit.lib.BatchRefUpdate} for
+ * {@link org.eclipse.jgit.internal.storage.dfs.DfsReftableDatabase}.
+ */
+public class DfsReftableBatchRefUpdate extends ReftableBatchRefUpdate {
+       private static final int AVG_BYTES = 36;
+
+       private final DfsReftableDatabase refdb;
+
+       private final DfsObjDatabase odb;
+
+       /**
+        * Initialize batch update.
+        *
+        * @param refdb
+        *            database the update will modify.
+        * @param odb
+        *            object database to store the reftable.
+        */
+       protected DfsReftableBatchRefUpdate(DfsReftableDatabase refdb,
+                       DfsObjDatabase odb) {
+               super(refdb, refdb.reftableDatabase, refdb.getLock(), refdb.getRepository());
+               this.refdb = refdb;
+               this.odb = odb;
+       }
+
+       @Override
+       protected void applyUpdates(List<Ref> newRefs, List<ReceiveCommand> pending)
+                       throws IOException {
+               Set<DfsPackDescription> prune = Collections.emptySet();
+               DfsPackDescription pack = odb.newPack(PackSource.INSERT);
+               try (DfsOutputStream out = odb.writeFile(pack, REFTABLE)) {
+                       ReftableConfig cfg = DfsPackCompactor
+                                       .configureReftable(refdb.getReftableConfig(), out);
+
+                       ReftableWriter.Stats stats;
+                       if (refdb.compactDuringCommit()
+                                       && newRefs.size() * AVG_BYTES <= cfg.getRefBlockSize()
+                                       && canCompactTopOfStack(cfg)) {
+                               ByteArrayOutputStream tmp = new ByteArrayOutputStream();
+                               ReftableWriter rw = new ReftableWriter(cfg, tmp);
+                               write(rw, newRefs, pending);
+                               rw.finish();
+                               stats = compactTopOfStack(out, cfg, tmp.toByteArray());
+                               prune = toPruneTopOfStack();
+                       } else {
+                               ReftableWriter rw = new ReftableWriter(cfg, out);
+                               write(rw, newRefs, pending);
+                               rw.finish();
+                               stats = rw.getStats();
+                       }
+                       pack.addFileExt(REFTABLE);
+                       pack.setReftableStats(stats);
+               }
+
+               odb.commitPack(Collections.singleton(pack), prune);
+               odb.addReftable(pack, prune);
+               refdb.clearCache();
+       }
+
+       private boolean canCompactTopOfStack(ReftableConfig cfg)
+                       throws IOException {
+               refdb.getLock().lock();
+               try {
+                       DfsReftableStack stack = refdb.stack();
+                       List<ReftableReader> readers = stack.readers();
+                       if (readers.isEmpty()) {
+                               return false;
+                       }
+
+                       int lastIdx = readers.size() - 1;
+                       DfsReftable last = stack.files().get(lastIdx);
+                       DfsPackDescription desc = last.getPackDescription();
+                       if (desc.getPackSource() != PackSource.INSERT
+                               || !packOnlyContainsReftable(desc)) {
+                               return false;
+                       }
+
+                       ReftableReader table = readers.get(lastIdx);
+                       int bs = cfg.getRefBlockSize();
+                       return table.size() <= 3 * bs;
+               } finally {
+                       refdb.getLock().unlock();
+               }
+       }
+
+       private ReftableWriter.Stats compactTopOfStack(OutputStream out,
+                       ReftableConfig cfg, byte[] newTable) throws IOException {
+               refdb.getLock().lock();
+               try {
+                       List<ReftableReader> stack = refdb.stack().readers();
+
+                       ReftableReader last = stack.get(stack.size() - 1);
+
+                       List<ReftableReader> tables = new ArrayList<>(2);
+                       tables.add(last);
+                       tables.add(new ReftableReader(BlockSource.from(newTable)));
+
+                       ReftableCompactor compactor = new ReftableCompactor(out);
+                       compactor.setConfig(cfg);
+                       compactor.setIncludeDeletes(true);
+                       compactor.addAll(tables);
+                       compactor.compact();
+                       return compactor.getStats();
+               } finally {
+                       refdb.getLock().unlock();
+               }
+       }
+
+       private Set<DfsPackDescription> toPruneTopOfStack() throws IOException {
+               refdb.getLock().lock();
+               try {
+                       List<DfsReftable> stack = refdb.stack().files();
+
+                       DfsReftable last = stack.get(stack.size() - 1);
+                       return Collections.singleton(last.getPackDescription());
+               } finally {
+                       refdb.getLock().unlock();
+               }
+       }
+
+       private boolean packOnlyContainsReftable(DfsPackDescription desc) {
+               for (PackExt ext : PackExt.values()) {
+                       if (ext != REFTABLE && desc.hasFileExt(ext)) {
+                               return false;
+                       }
+               }
+               return true;
+       }
+}
index cf3c71d431bc7ab605847a9067471d925558c453..2497719196127cccab010c4f91215a61921c7359 100644 (file)
@@ -45,19 +45,17 @@ package org.eclipse.jgit.internal.storage.dfs;
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeSet;
 import java.util.concurrent.locks.ReentrantLock;
 
 import org.eclipse.jgit.annotations.Nullable;
 import org.eclipse.jgit.internal.storage.reftable.MergedReftable;
-import org.eclipse.jgit.internal.storage.reftable.RefCursor;
-import org.eclipse.jgit.internal.storage.reftable.Reftable;
 import org.eclipse.jgit.internal.storage.reftable.ReftableConfig;
+import org.eclipse.jgit.internal.storage.reftable.ReftableDatabase;
 import org.eclipse.jgit.lib.BatchRefUpdate;
 import org.eclipse.jgit.lib.NullProgressMonitor;
 import org.eclipse.jgit.lib.ObjectId;
@@ -81,13 +79,10 @@ import org.eclipse.jgit.util.RefMap;
  * and one will fail.
  */
 public class DfsReftableDatabase extends DfsRefDatabase {
-       private final ReentrantLock lock = new ReentrantLock(true);
+       final ReftableDatabase reftableDatabase;
 
        private DfsReader ctx;
-
-       private DfsReftableStack tableStack;
-
-       private MergedReftable mergedTables;
+       private DfsReftableStack stack;
 
        /**
         * Initialize the reference database for a repository.
@@ -97,6 +92,18 @@ public class DfsReftableDatabase extends DfsRefDatabase {
         */
        protected DfsReftableDatabase(DfsRepository repo) {
                super(repo);
+               reftableDatabase = new ReftableDatabase() {
+                       @Override
+                       public MergedReftable openMergedReftable() throws IOException {
+                               DfsReftableDatabase.this.getLock().lock();
+                               try {
+                                       return new MergedReftable(stack().readers());
+                               } finally {
+                                       DfsReftableDatabase.this.getLock().unlock();
+                               }
+                       }
+               };
+               stack = null;
        }
 
        /** {@inheritDoc} */
@@ -115,7 +122,7 @@ public class DfsReftableDatabase extends DfsRefDatabase {
        @Override
        public BatchRefUpdate newBatchUpdate() {
                DfsObjDatabase odb = getRepository().getObjectDatabase();
-               return new ReftableBatchRefUpdate(this, odb);
+               return new DfsReftableBatchRefUpdate(this, odb);
        }
 
        /**
@@ -133,7 +140,7 @@ public class DfsReftableDatabase extends DfsRefDatabase {
         * @return the lock protecting this instance's state.
         */
        protected ReentrantLock getLock() {
-               return lock;
+               return reftableDatabase.getLock();
        }
 
        /**
@@ -147,134 +154,55 @@ public class DfsReftableDatabase extends DfsRefDatabase {
                return true;
        }
 
-       /**
-        * Obtain a handle to the merged reader.
-        *
-        * @return (possibly cached) handle to the merged reader.
-        * @throws java.io.IOException
-        *             if tables cannot be opened.
-        */
-       protected Reftable reader() throws IOException {
-               lock.lock();
-               try {
-                       if (mergedTables == null) {
-                               mergedTables = new MergedReftable(stack().readers());
-                       }
-                       return mergedTables;
-               } finally {
-                       lock.unlock();
-               }
-       }
 
        /**
-        * Obtain a handle to the stack of reftables.
+        * Obtain a handle to the stack of reftables. Must hold lock.
         *
         * @return (possibly cached) handle to the stack.
         * @throws java.io.IOException
         *             if tables cannot be opened.
         */
        protected DfsReftableStack stack() throws IOException {
-               lock.lock();
-               try {
-                       if (tableStack == null) {
-                               DfsObjDatabase odb = getRepository().getObjectDatabase();
-                               if (ctx == null) {
-                                       ctx = odb.newReader();
-                               }
-                               tableStack = DfsReftableStack.open(ctx,
-                                               Arrays.asList(odb.getReftables()));
-                       }
-                       return tableStack;
-               } finally {
-                       lock.unlock();
+               assert getLock().isLocked();
+               DfsObjDatabase odb = getRepository().getObjectDatabase();
+
+               if (ctx == null) {
+                       ctx = odb.newReader();
                }
+               if (stack == null) {
+                       stack = DfsReftableStack.open(ctx, Arrays.asList(odb.getReftables()));
+               }
+               return stack;
        }
 
-       /** {@inheritDoc} */
        @Override
        public boolean isNameConflicting(String refName) throws IOException {
-               lock.lock();
-               try {
-                       Reftable table = reader();
-
-                       // Cannot be nested within an existing reference.
-                       int lastSlash = refName.lastIndexOf('/');
-                       while (0 < lastSlash) {
-                               if (table.hasRef(refName.substring(0, lastSlash))) {
-                                       return true;
-                               }
-                               lastSlash = refName.lastIndexOf('/', lastSlash - 1);
-                       }
-
-                       // Cannot be the container of an existing reference.
-                       return table.hasRefsWithPrefix(refName + '/');
-               } finally {
-                       lock.unlock();
-               }
+               return reftableDatabase.isNameConflicting(refName, new TreeSet<>(), new HashSet<>());
        }
 
        /** {@inheritDoc} */
        @Override
        public Ref exactRef(String name) throws IOException {
-               lock.lock();
-               try {
-                       Reftable table = reader();
-                       Ref ref = table.exactRef(name);
-                       if (ref != null && ref.isSymbolic()) {
-                               return table.resolve(ref);
-                       }
-                       return ref;
-               } finally {
-                       lock.unlock();
-               }
+               return reftableDatabase.exactRef(name);
        }
 
        /** {@inheritDoc} */
        @Override
        public Map<String, Ref> getRefs(String prefix) throws IOException {
-               RefList.Builder<Ref> all = new RefList.Builder<>();
-               lock.lock();
-               try {
-                       Reftable table = reader();
-                       try (RefCursor rc = ALL.equals(prefix) ? table.allRefs()
-                                       : (prefix.endsWith("/") ? table.seekRefsWithPrefix(prefix) //$NON-NLS-1$
-                                                       : table.seekRef(prefix))) {
-                               while (rc.next()) {
-                                       Ref ref = table.resolve(rc.getRef());
-                                       if (ref != null && ref.getObjectId() != null) {
-                                               all.add(ref);
-                                       }
-                               }
-                       }
-               } finally {
-                       lock.unlock();
+               List<Ref> refs = reftableDatabase.getRefsByPrefix(prefix);
+               RefList.Builder<Ref> builder = new RefList.Builder<>(refs.size());
+               for (Ref r : refs) {
+                       builder.add(r);
                }
-
-               RefList<Ref> none = RefList.emptyList();
-               return new RefMap(prefix, all.toRefList(), none, none);
+               return new RefMap(prefix, builder.toRefList(), RefList.emptyList(),
+                       RefList.emptyList());
        }
 
        /** {@inheritDoc} */
        @Override
        public List<Ref> getRefsByPrefix(String prefix) throws IOException {
-               List<Ref> all = new ArrayList<>();
-               lock.lock();
-               try {
-                       Reftable table = reader();
-                       try (RefCursor rc = ALL.equals(prefix) ? table.allRefs()
-                                       : table.seekRefsWithPrefix(prefix)) {
-                               while (rc.next()) {
-                                       Ref ref = table.resolve(rc.getRef());
-                                       if (ref != null && ref.getObjectId() != null) {
-                                               all.add(ref);
-                                       }
-                               }
-                       }
-               } finally {
-                       lock.unlock();
-               }
 
-               return Collections.unmodifiableList(all);
+               return reftableDatabase.getRefsByPrefix(prefix);
        }
 
        /** {@inheritDoc} */
@@ -283,17 +211,7 @@ public class DfsReftableDatabase extends DfsRefDatabase {
                if (!getReftableConfig().isIndexObjects()) {
                        return super.getTipsWithSha1(id);
                }
-               lock.lock();
-               try {
-                       RefCursor cursor = reader().byObjectId(id);
-                       Set<Ref> refs = new HashSet<>();
-                       while (cursor.next()) {
-                               refs.add(cursor.getRef());
-                       }
-                       return refs;
-               } finally {
-                       lock.unlock();
-               }
+               return reftableDatabase.getTipsWithSha1(id);
        }
 
        /** {@inheritDoc} */
@@ -314,19 +232,19 @@ public class DfsReftableDatabase extends DfsRefDatabase {
 
        @Override
        void clearCache() {
-               lock.lock();
+               getLock().lock();
                try {
-                       if (tableStack != null) {
-                               tableStack.close();
-                               tableStack = null;
-                       }
                        if (ctx != null) {
                                ctx.close();
                                ctx = null;
                        }
-                       mergedTables = null;
+                       reftableDatabase.clearCache();
+                       if (stack != null) {
+                               stack.close();
+                               stack = null;
+                       }
                } finally {
-                       lock.unlock();
+                       getLock().unlock();
                }
        }
 
@@ -334,7 +252,7 @@ public class DfsReftableDatabase extends DfsRefDatabase {
        @Override
        protected boolean compareAndPut(Ref oldRef, @Nullable Ref newRef)
                        throws IOException {
-               ReceiveCommand cmd = toCommand(oldRef, newRef);
+               ReceiveCommand cmd = ReftableDatabase.toCommand(oldRef, newRef);
                try (RevWalk rw = new RevWalk(getRepository())) {
                        rw.setRetainBody(false);
                        newBatchUpdate().setAllowNonFastForwards(true).addCommand(cmd)
@@ -351,58 +269,6 @@ public class DfsReftableDatabase extends DfsRefDatabase {
                }
        }
 
-       private static ReceiveCommand toCommand(Ref oldRef, Ref newRef) {
-               ObjectId oldId = toId(oldRef);
-               ObjectId newId = toId(newRef);
-               String name = toName(oldRef, newRef);
-
-               if (oldRef != null && oldRef.isSymbolic()) {
-                       if (newRef != null) {
-                               if (newRef.isSymbolic()) {
-                                       return ReceiveCommand.link(oldRef.getTarget().getName(),
-                                                       newRef.getTarget().getName(), name);
-                               } else {
-                                       return ReceiveCommand.unlink(oldRef.getTarget().getName(),
-                                                       newId, name);
-                               }
-                       } else {
-                               return ReceiveCommand.unlink(oldRef.getTarget().getName(),
-                                               ObjectId.zeroId(), name);
-                       }
-               }
-
-               if (newRef != null && newRef.isSymbolic()) {
-                       if (oldRef != null) {
-                               if (oldRef.isSymbolic()) {
-                                       return ReceiveCommand.link(oldRef.getTarget().getName(),
-                                                       newRef.getTarget().getName(), name);
-                               } else {
-                                       return ReceiveCommand.link(oldId,
-                                                       newRef.getTarget().getName(), name);
-                               }
-                       } else {
-                               return ReceiveCommand.link(ObjectId.zeroId(),
-                                               newRef.getTarget().getName(), name);
-                       }
-               }
-
-               return new ReceiveCommand(oldId, newId, name);
-       }
-
-       private static ObjectId toId(Ref ref) {
-               if (ref != null) {
-                       ObjectId id = ref.getObjectId();
-                       if (id != null) {
-                               return id;
-                       }
-               }
-               return ObjectId.zeroId();
-       }
-
-       private static String toName(Ref oldRef, Ref newRef) {
-               return oldRef != null ? oldRef.getName() : newRef.getName();
-       }
-
        /** {@inheritDoc} */
        @Override
        protected boolean compareAndRemove(Ref oldRef) throws IOException {
@@ -417,12 +283,12 @@ public class DfsReftableDatabase extends DfsRefDatabase {
 
        @Override
        void stored(Ref ref) {
-               // Unnecessary; ReftableBatchRefUpdate calls clearCache().
+               // Unnecessary; DfsReftableBatchRefUpdate calls clearCache().
        }
 
        @Override
        void removed(String refName) {
-               // Unnecessary; ReftableBatchRefUpdate calls clearCache().
+               // Unnecessary; DfsReftableBatchRefUpdate calls clearCache().
        }
 
        /** {@inheritDoc} */
@@ -430,4 +296,5 @@ public class DfsReftableDatabase extends DfsRefDatabase {
        protected void cachePeeledState(Ref oldLeaf, Ref newLeaf) {
                // Do not cache peeled state in reftable.
        }
+
 }
diff --git a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/ReftableBatchRefUpdate.java b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/ReftableBatchRefUpdate.java
deleted file mode 100644 (file)
index b19ffff..0000000
+++ /dev/null
@@ -1,459 +0,0 @@
-/*
- * Copyright (C) 2017, Google Inc.
- * and other copyright owners as documented in the project's IP log.
- *
- * This program and the accompanying materials are made available
- * under the terms of the Eclipse Distribution License v1.0 which
- * accompanies this distribution, is reproduced below, and is
- * available at http://www.eclipse.org/org/documents/edl-v10.php
- *
- * All rights reserved.
- *
- * Redistribution and use in source and binary forms, with or
- * without modification, are permitted provided that the following
- * conditions are met:
- *
- * - Redistributions of source code must retain the above copyright
- *   notice, this list of conditions and the following disclaimer.
- *
- * - Redistributions in binary form must reproduce the above
- *   copyright notice, this list of conditions and the following
- *   disclaimer in the documentation and/or other materials provided
- *   with the distribution.
- *
- * - Neither the name of the Eclipse Foundation, Inc. nor the
- *   names of its contributors may be used to endorse or promote
- *   products derived from this software without specific prior
- *   written permission.
- *
- * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND
- * CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES,
- * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
- * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
- * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
- * CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
- * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
- * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
- * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
- * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
- * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF
- * ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
- */
-
-package org.eclipse.jgit.internal.storage.dfs;
-
-import static org.eclipse.jgit.internal.storage.pack.PackExt.REFTABLE;
-import static org.eclipse.jgit.lib.Ref.Storage.NEW;
-import static org.eclipse.jgit.lib.Ref.Storage.PACKED;
-import static org.eclipse.jgit.transport.ReceiveCommand.Result.LOCK_FAILURE;
-import static org.eclipse.jgit.transport.ReceiveCommand.Result.NOT_ATTEMPTED;
-import static org.eclipse.jgit.transport.ReceiveCommand.Result.OK;
-import static org.eclipse.jgit.transport.ReceiveCommand.Result.REJECTED_MISSING_OBJECT;
-import static org.eclipse.jgit.transport.ReceiveCommand.Result.REJECTED_NONFASTFORWARD;
-import static org.eclipse.jgit.transport.ReceiveCommand.Type.UPDATE_NONFASTFORWARD;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.eclipse.jgit.annotations.Nullable;
-import org.eclipse.jgit.errors.MissingObjectException;
-import org.eclipse.jgit.internal.storage.dfs.DfsObjDatabase.PackSource;
-import org.eclipse.jgit.internal.storage.io.BlockSource;
-import org.eclipse.jgit.internal.storage.pack.PackExt;
-import org.eclipse.jgit.internal.storage.reftable.Reftable;
-import org.eclipse.jgit.internal.storage.reftable.ReftableCompactor;
-import org.eclipse.jgit.internal.storage.reftable.ReftableConfig;
-import org.eclipse.jgit.internal.storage.reftable.ReftableReader;
-import org.eclipse.jgit.internal.storage.reftable.ReftableWriter;
-import org.eclipse.jgit.lib.AnyObjectId;
-import org.eclipse.jgit.lib.BatchRefUpdate;
-import org.eclipse.jgit.lib.ObjectId;
-import org.eclipse.jgit.lib.ObjectIdRef;
-import org.eclipse.jgit.lib.PersonIdent;
-import org.eclipse.jgit.lib.ProgressMonitor;
-import org.eclipse.jgit.lib.Ref;
-import org.eclipse.jgit.lib.ReflogEntry;
-import org.eclipse.jgit.lib.SymbolicRef;
-import org.eclipse.jgit.revwalk.RevObject;
-import org.eclipse.jgit.revwalk.RevTag;
-import org.eclipse.jgit.revwalk.RevWalk;
-import org.eclipse.jgit.transport.ReceiveCommand;
-
-/**
- * {@link org.eclipse.jgit.lib.BatchRefUpdate} for
- * {@link org.eclipse.jgit.internal.storage.dfs.DfsReftableDatabase}.
- */
-public class ReftableBatchRefUpdate extends BatchRefUpdate {
-       private static final int AVG_BYTES = 36;
-
-       private final DfsReftableDatabase refdb;
-
-       private final DfsObjDatabase odb;
-
-       private final ReentrantLock lock;
-
-       private final ReftableConfig reftableConfig;
-
-       /**
-        * Initialize batch update.
-        *
-        * @param refdb
-        *            database the update will modify.
-        * @param odb
-        *            object database to store the reftable.
-        */
-       protected ReftableBatchRefUpdate(DfsReftableDatabase refdb,
-                       DfsObjDatabase odb) {
-               super(refdb);
-               this.refdb = refdb;
-               this.odb = odb;
-               lock = refdb.getLock();
-               reftableConfig = refdb.getReftableConfig();
-       }
-
-       /** {@inheritDoc} */
-       @Override
-       public void execute(RevWalk rw, ProgressMonitor pm, List<String> options) {
-               List<ReceiveCommand> pending = getPending();
-               if (pending.isEmpty()) {
-                       return;
-               }
-               if (options != null) {
-                       setPushOptions(options);
-               }
-               try {
-                       if (!checkObjectExistence(rw, pending)) {
-                               return;
-                       }
-                       if (!checkNonFastForwards(rw, pending)) {
-                               return;
-                       }
-
-                       lock.lock();
-                       try {
-                               if (!checkExpected(pending)) {
-                                       return;
-                               }
-                               if (!checkConflicting(pending)) {
-                                       return;
-                               }
-                               if (!blockUntilTimestamps(MAX_WAIT)) {
-                                       return;
-                               }
-                               applyUpdates(rw, pending);
-                               for (ReceiveCommand cmd : pending) {
-                                       cmd.setResult(OK);
-                               }
-                       } finally {
-                               lock.unlock();
-                       }
-               } catch (IOException e) {
-                       pending.get(0).setResult(LOCK_FAILURE, "io error"); //$NON-NLS-1$
-                       ReceiveCommand.abort(pending);
-               }
-       }
-
-       private List<ReceiveCommand> getPending() {
-               return ReceiveCommand.filter(getCommands(), NOT_ATTEMPTED);
-       }
-
-       private boolean checkObjectExistence(RevWalk rw,
-                       List<ReceiveCommand> pending) throws IOException {
-               for (ReceiveCommand cmd : pending) {
-                       try {
-                               if (!cmd.getNewId().equals(ObjectId.zeroId())) {
-                                       rw.parseAny(cmd.getNewId());
-                               }
-                       } catch (MissingObjectException e) {
-                               // ReceiveCommand#setResult(Result) converts REJECTED to
-                               // REJECTED_NONFASTFORWARD, even though that result is also
-                               // used for a missing object. Eagerly handle this case so we
-                               // can set the right result.
-                               cmd.setResult(REJECTED_MISSING_OBJECT);
-                               ReceiveCommand.abort(pending);
-                               return false;
-                       }
-               }
-               return true;
-       }
-
-       private boolean checkNonFastForwards(RevWalk rw,
-                       List<ReceiveCommand> pending) throws IOException {
-               if (isAllowNonFastForwards()) {
-                       return true;
-               }
-               for (ReceiveCommand cmd : pending) {
-                       cmd.updateType(rw);
-                       if (cmd.getType() == UPDATE_NONFASTFORWARD) {
-                               cmd.setResult(REJECTED_NONFASTFORWARD);
-                               ReceiveCommand.abort(pending);
-                               return false;
-                       }
-               }
-               return true;
-       }
-
-       private boolean checkConflicting(List<ReceiveCommand> pending)
-                       throws IOException {
-               Set<String> names = new HashSet<>();
-               for (ReceiveCommand cmd : pending) {
-                       names.add(cmd.getRefName());
-               }
-
-               boolean ok = true;
-               for (ReceiveCommand cmd : pending) {
-                       String name = cmd.getRefName();
-                       if (refdb.isNameConflicting(name)) {
-                               cmd.setResult(LOCK_FAILURE);
-                               ok = false;
-                       } else {
-                               int s = name.lastIndexOf('/');
-                               while (0 < s) {
-                                       if (names.contains(name.substring(0, s))) {
-                                               cmd.setResult(LOCK_FAILURE);
-                                               ok = false;
-                                               break;
-                                       }
-                                       s = name.lastIndexOf('/', s - 1);
-                               }
-                       }
-               }
-               if (!ok && isAtomic()) {
-                       ReceiveCommand.abort(pending);
-                       return false;
-               }
-               return ok;
-       }
-
-       private boolean checkExpected(List<ReceiveCommand> pending)
-                       throws IOException {
-               for (ReceiveCommand cmd : pending) {
-                       if (!matchOld(cmd, refdb.exactRef(cmd.getRefName()))) {
-                               cmd.setResult(LOCK_FAILURE);
-                               if (isAtomic()) {
-                                       ReceiveCommand.abort(pending);
-                                       return false;
-                               }
-                       }
-               }
-               return true;
-       }
-
-       private static boolean matchOld(ReceiveCommand cmd, @Nullable Ref ref) {
-               if (ref == null) {
-                       return AnyObjectId.isEqual(ObjectId.zeroId(), cmd.getOldId())
-                                       && cmd.getOldSymref() == null;
-               } else if (ref.isSymbolic()) {
-                       return ref.getTarget().getName().equals(cmd.getOldSymref());
-               }
-               ObjectId id = ref.getObjectId();
-               if (id == null) {
-                       id = ObjectId.zeroId();
-               }
-               return cmd.getOldId().equals(id);
-       }
-
-       private void applyUpdates(RevWalk rw, List<ReceiveCommand> pending)
-                       throws IOException {
-               List<Ref> newRefs = toNewRefs(rw, pending);
-               long updateIndex = nextUpdateIndex();
-               Set<DfsPackDescription> prune = Collections.emptySet();
-               DfsPackDescription pack = odb.newPack(PackSource.INSERT);
-               try (DfsOutputStream out = odb.writeFile(pack, REFTABLE)) {
-                       ReftableConfig cfg = DfsPackCompactor
-                                       .configureReftable(reftableConfig, out);
-
-                       ReftableWriter.Stats stats;
-                       if (refdb.compactDuringCommit()
-                                       && newRefs.size() * AVG_BYTES <= cfg.getRefBlockSize()
-                                       && canCompactTopOfStack(cfg)) {
-                               ByteArrayOutputStream tmp = new ByteArrayOutputStream();
-                               write(tmp, cfg, updateIndex, newRefs, pending);
-                               stats = compactTopOfStack(out, cfg, tmp.toByteArray());
-                               prune = toPruneTopOfStack();
-                       } else {
-                               stats = write(out, cfg, updateIndex, newRefs, pending);
-                       }
-                       pack.addFileExt(REFTABLE);
-                       pack.setReftableStats(stats);
-               }
-
-               odb.commitPack(Collections.singleton(pack), prune);
-               odb.addReftable(pack, prune);
-               refdb.clearCache();
-       }
-
-       private ReftableWriter.Stats write(OutputStream os, ReftableConfig cfg,
-                       long updateIndex, List<Ref> newRefs, List<ReceiveCommand> pending)
-                       throws IOException {
-               ReftableWriter writer = new ReftableWriter(cfg, os)
-                               .setMinUpdateIndex(updateIndex).setMaxUpdateIndex(updateIndex)
-                               .begin().sortAndWriteRefs(newRefs);
-               if (!isRefLogDisabled()) {
-                       writeLog(writer, updateIndex, pending);
-               }
-               writer.finish();
-               return writer.getStats();
-       }
-
-       private void writeLog(ReftableWriter writer, long updateIndex,
-                       List<ReceiveCommand> pending) throws IOException {
-               Map<String, ReceiveCommand> cmds = new HashMap<>();
-               List<String> byName = new ArrayList<>(pending.size());
-               for (ReceiveCommand cmd : pending) {
-                       cmds.put(cmd.getRefName(), cmd);
-                       byName.add(cmd.getRefName());
-               }
-               Collections.sort(byName);
-
-               PersonIdent ident = getRefLogIdent();
-               if (ident == null) {
-                       ident = new PersonIdent(refdb.getRepository());
-               }
-               for (String name : byName) {
-                       ReceiveCommand cmd = cmds.get(name);
-                       if (isRefLogDisabled(cmd)) {
-                               continue;
-                       }
-                       String msg = getRefLogMessage(cmd);
-                       if (isRefLogIncludingResult(cmd)) {
-                               String strResult = toResultString(cmd);
-                               if (strResult != null) {
-                                       msg = msg.isEmpty() ? strResult : msg + ": " + strResult; //$NON-NLS-1$
-                               }
-                       }
-                       writer.writeLog(name, updateIndex, ident, cmd.getOldId(),
-                                       cmd.getNewId(), msg);
-               }
-       }
-
-       private String toResultString(ReceiveCommand cmd) {
-               switch (cmd.getType()) {
-               case CREATE:
-                       return ReflogEntry.PREFIX_CREATED;
-               case UPDATE:
-                       // Match the behavior of a single RefUpdate. In that case, setting
-                       // the force bit completely bypasses the potentially expensive
-                       // isMergedInto check, by design, so the reflog message may be
-                       // inaccurate.
-                       //
-                       // Similarly, this class bypasses the isMergedInto checks when the
-                       // force bit is set, meaning we can't actually distinguish between
-                       // UPDATE and UPDATE_NONFASTFORWARD when isAllowNonFastForwards()
-                       // returns true.
-                       return isAllowNonFastForwards() ? ReflogEntry.PREFIX_FORCED_UPDATE
-                                       : ReflogEntry.PREFIX_FAST_FORWARD;
-               case UPDATE_NONFASTFORWARD:
-                       return ReflogEntry.PREFIX_FORCED_UPDATE;
-               default:
-                       return null;
-               }
-       }
-
-       private static List<Ref> toNewRefs(RevWalk rw, List<ReceiveCommand> pending)
-                       throws IOException {
-               List<Ref> refs = new ArrayList<>(pending.size());
-               for (ReceiveCommand cmd : pending) {
-                       String name = cmd.getRefName();
-                       ObjectId newId = cmd.getNewId();
-                       String newSymref = cmd.getNewSymref();
-                       if (AnyObjectId.isEqual(ObjectId.zeroId(), newId)
-                                       && newSymref == null) {
-                               refs.add(new ObjectIdRef.Unpeeled(NEW, name, null));
-                               continue;
-                       } else if (newSymref != null) {
-                               refs.add(new SymbolicRef(name,
-                                               new ObjectIdRef.Unpeeled(NEW, newSymref, null)));
-                               continue;
-                       }
-
-                       RevObject obj = rw.parseAny(newId);
-                       RevObject peel = null;
-                       if (obj instanceof RevTag) {
-                               peel = rw.peel(obj);
-                       }
-                       if (peel != null) {
-                               refs.add(new ObjectIdRef.PeeledTag(PACKED, name, newId,
-                                               peel.copy()));
-                       } else {
-                               refs.add(new ObjectIdRef.PeeledNonTag(PACKED, name, newId));
-                       }
-               }
-               return refs;
-       }
-
-       private long nextUpdateIndex() throws IOException {
-               long updateIndex = 0;
-               for (Reftable r : refdb.stack().readers()) {
-                       if (r instanceof ReftableReader) {
-                               updateIndex = Math.max(updateIndex,
-                                               ((ReftableReader) r).maxUpdateIndex());
-                       }
-               }
-               return updateIndex + 1;
-       }
-
-       private boolean canCompactTopOfStack(ReftableConfig cfg)
-                       throws IOException {
-               DfsReftableStack stack = refdb.stack();
-               List<ReftableReader> readers = stack.readers();
-               if (readers.isEmpty()) {
-                       return false;
-               }
-
-               int lastIdx = readers.size() - 1;
-               DfsReftable last = stack.files().get(lastIdx);
-               DfsPackDescription desc = last.getPackDescription();
-               if (desc.getPackSource() != PackSource.INSERT
-                               || !packOnlyContainsReftable(desc)) {
-                       return false;
-               }
-
-               Reftable table = readers.get(lastIdx);
-               int bs = cfg.getRefBlockSize();
-               return table instanceof ReftableReader
-                               && ((ReftableReader) table).size() <= 3 * bs;
-       }
-
-       private ReftableWriter.Stats compactTopOfStack(OutputStream out,
-                       ReftableConfig cfg, byte[] newTable) throws IOException {
-               List<ReftableReader> stack = refdb.stack().readers();
-               ReftableReader last = stack.get(stack.size() - 1);
-
-               List<ReftableReader> tables = new ArrayList<>(2);
-               tables.add(last);
-               tables.add(new ReftableReader(BlockSource.from(newTable)));
-
-               ReftableCompactor compactor = new ReftableCompactor(out);
-               compactor.setConfig(cfg);
-               compactor.setIncludeDeletes(true);
-               compactor.addAll(tables);
-               compactor.compact();
-               return compactor.getStats();
-       }
-
-       private Set<DfsPackDescription> toPruneTopOfStack() throws IOException {
-               List<DfsReftable> stack = refdb.stack().files();
-               DfsReftable last = stack.get(stack.size() - 1);
-               return Collections.singleton(last.getPackDescription());
-       }
-
-       private boolean packOnlyContainsReftable(DfsPackDescription desc) {
-               for (PackExt ext : PackExt.values()) {
-                       if (ext != REFTABLE && desc.hasFileExt(ext)) {
-                               return false;
-                       }
-               }
-               return true;
-       }
-}
diff --git a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/reftable/ReftableBatchRefUpdate.java b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/reftable/ReftableBatchRefUpdate.java
new file mode 100644 (file)
index 0000000..592120d
--- /dev/null
@@ -0,0 +1,415 @@
+/*
+ * Copyright (C) 2019, Google Inc.
+ * and other copyright owners as documented in the project's IP log.
+ *
+ * This program and the accompanying materials are made available
+ * under the terms of the Eclipse Distribution License v1.0 which
+ * accompanies this distribution, is reproduced below, and is
+ * available at http://www.eclipse.org/org/documents/edl-v10.php
+ *
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or
+ * without modification, are permitted provided that the following
+ * conditions are met:
+ *
+ * - Redistributions of source code must retain the above copyright
+ *   notice, this list of conditions and the following disclaimer.
+ *
+ * - Redistributions in binary form must reproduce the above
+ *   copyright notice, this list of conditions and the following
+ *   disclaimer in the documentation and/or other materials provided
+ *   with the distribution.
+ *
+ * - Neither the name of the Eclipse Foundation, Inc. nor the
+ *   names of its contributors may be used to endorse or promote
+ *   products derived from this software without specific prior
+ *   written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND
+ * CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES,
+ * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
+ * CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+ * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF
+ * ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+package org.eclipse.jgit.internal.storage.reftable;
+
+import org.eclipse.jgit.annotations.Nullable;
+import org.eclipse.jgit.errors.MissingObjectException;
+import org.eclipse.jgit.internal.JGitText;
+import org.eclipse.jgit.lib.AnyObjectId;
+import org.eclipse.jgit.lib.BatchRefUpdate;
+import org.eclipse.jgit.lib.ObjectId;
+import org.eclipse.jgit.lib.ObjectIdRef;
+import org.eclipse.jgit.lib.PersonIdent;
+import org.eclipse.jgit.lib.ProgressMonitor;
+import org.eclipse.jgit.lib.Ref;
+import org.eclipse.jgit.lib.RefDatabase;
+import org.eclipse.jgit.lib.ReflogEntry;
+import org.eclipse.jgit.lib.Repository;
+import org.eclipse.jgit.lib.SymbolicRef;
+import org.eclipse.jgit.revwalk.RevObject;
+import org.eclipse.jgit.revwalk.RevTag;
+import org.eclipse.jgit.revwalk.RevWalk;
+import org.eclipse.jgit.transport.ReceiveCommand;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.locks.Lock;
+import java.util.stream.Collectors;
+
+import static org.eclipse.jgit.lib.Ref.Storage.NEW;
+import static org.eclipse.jgit.lib.Ref.Storage.PACKED;
+
+import static org.eclipse.jgit.transport.ReceiveCommand.Result.LOCK_FAILURE;
+import static org.eclipse.jgit.transport.ReceiveCommand.Result.NOT_ATTEMPTED;
+import static org.eclipse.jgit.transport.ReceiveCommand.Result.OK;
+import static org.eclipse.jgit.transport.ReceiveCommand.Result.REJECTED_MISSING_OBJECT;
+import static org.eclipse.jgit.transport.ReceiveCommand.Result.REJECTED_NONFASTFORWARD;
+import static org.eclipse.jgit.transport.ReceiveCommand.Type.DELETE;
+import static org.eclipse.jgit.transport.ReceiveCommand.Type.UPDATE_NONFASTFORWARD;
+
+/**
+ * {@link org.eclipse.jgit.lib.BatchRefUpdate} for Reftable based RefDatabase.
+ */
+public abstract class ReftableBatchRefUpdate extends BatchRefUpdate {
+       private final Lock lock;
+
+       private final ReftableDatabase refDb;
+
+       private final Repository repository;
+
+       /**
+        * Initialize.
+        *
+        * @param refdb
+        *            The RefDatabase
+        * @param reftableDb
+        *            The ReftableDatabase
+        * @param lock
+        *            A lock protecting the refdatabase's state
+        * @param repository
+        *            The repository on which this update will run
+        */
+       protected ReftableBatchRefUpdate(RefDatabase refdb, ReftableDatabase reftableDb, Lock lock,
+                       Repository repository) {
+               super(refdb);
+               this.refDb = reftableDb;
+               this.lock = lock;
+               this.repository = repository;
+       }
+
+       /** {@inheritDoc} */
+       @Override
+       public void execute(RevWalk rw, ProgressMonitor pm, List<String> options) {
+               List<ReceiveCommand> pending = getPending();
+               if (pending.isEmpty()) {
+                       return;
+               }
+               if (options != null) {
+                       setPushOptions(options);
+               }
+               try {
+                       if (!checkObjectExistence(rw, pending)) {
+                               return;
+                       }
+                       // if we are here, checkObjectExistence might have flagged some problems
+                       // but the transaction is not atomic, so we should proceed with the other
+                       // pending commands.
+                       pending = getPending();
+                       if (!checkNonFastForwards(rw, pending)) {
+                               return;
+                       }
+                       pending = getPending();
+
+                       lock.lock();
+                       try {
+                               if (!checkExpected(pending)) {
+                                       return;
+                               }
+                               pending = getPending();
+                               if (!checkConflicting(pending)) {
+                                       return;
+                               }
+                               pending = getPending();
+                               if (!blockUntilTimestamps(MAX_WAIT)) {
+                                       return;
+                               }
+
+                               List<Ref> newRefs = toNewRefs(rw, pending);
+                               applyUpdates(newRefs, pending);
+                               for (ReceiveCommand cmd : pending) {
+                                       if (cmd.getResult() == NOT_ATTEMPTED) {
+                                               // XXX this is a bug in DFS ?
+                                               cmd.setResult(OK);
+                                       }
+                               }
+                       } finally {
+                               lock.unlock();
+                       }
+               } catch (IOException e) {
+                       pending.get(0).setResult(LOCK_FAILURE, "io error"); //$NON-NLS-1$
+                       ReceiveCommand.abort(pending);
+               }
+       }
+
+       /**
+        * Implements the storage-specific part of the update.
+        *
+        * @param newRefs
+        *            the new refs to create
+        * @param pending
+        *            the pending receive commands to be executed
+        * @throws IOException
+        *             if any of the writes fail.
+        */
+       protected abstract void applyUpdates(List<Ref> newRefs,
+                       List<ReceiveCommand> pending) throws IOException;
+
+       private List<ReceiveCommand> getPending() {
+               return ReceiveCommand.filter(getCommands(), NOT_ATTEMPTED);
+       }
+
+       private boolean checkObjectExistence(RevWalk rw,
+                       List<ReceiveCommand> pending) throws IOException {
+               for (ReceiveCommand cmd : pending) {
+                       try {
+                               if (!cmd.getNewId().equals(ObjectId.zeroId())) {
+                                       rw.parseAny(cmd.getNewId());
+                               }
+                       } catch (MissingObjectException e) {
+                               // ReceiveCommand#setResult(Result) converts REJECTED to
+                               // REJECTED_NONFASTFORWARD, even though that result is also
+                               // used for a missing object. Eagerly handle this case so we
+                               // can set the right result.
+                               cmd.setResult(REJECTED_MISSING_OBJECT);
+                               if (isAtomic()) {
+                                       ReceiveCommand.abort(pending);
+                                       return false;
+                               }
+                       }
+               }
+               return true;
+       }
+
+       private boolean checkNonFastForwards(RevWalk rw,
+                       List<ReceiveCommand> pending) throws IOException {
+               if (isAllowNonFastForwards()) {
+                       return true;
+               }
+               for (ReceiveCommand cmd : pending) {
+                       cmd.updateType(rw);
+                       if (cmd.getType() == UPDATE_NONFASTFORWARD) {
+                               cmd.setResult(REJECTED_NONFASTFORWARD);
+                               if (isAtomic()) {
+                                       ReceiveCommand.abort(pending);
+                                       return false;
+                               }
+                       }
+               }
+               return true;
+       }
+
+       private boolean checkConflicting(List<ReceiveCommand> pending)
+                       throws IOException {
+               TreeSet<String> added = new TreeSet<>();
+               Set<String> deleted =
+                               pending.stream()
+                                               .filter(cmd -> cmd.getType() == DELETE)
+                                               .map(c -> c.getRefName())
+                                               .collect(Collectors.toSet());
+
+               boolean ok = true;
+               for (ReceiveCommand cmd : pending) {
+                       if (cmd.getType() == DELETE) {
+                               continue;
+                       }
+
+                       String name = cmd.getRefName();
+                       if (refDb.isNameConflicting(name, added, deleted)) {
+                               if (isAtomic()) {
+                                       cmd.setResult(
+                                                       ReceiveCommand.Result.REJECTED_OTHER_REASON, JGitText.get().transactionAborted);
+                               } else {
+                                       cmd.setResult(LOCK_FAILURE);
+                               }
+
+                               ok = false;
+                       }
+                       added.add(name);
+               }
+
+               if (isAtomic()) {
+                       if (!ok) {
+                               pending.stream()
+                                               .filter(cmd -> cmd.getResult() == NOT_ATTEMPTED)
+                                               .forEach(cmd -> cmd.setResult(LOCK_FAILURE));
+                       }
+                       return ok;
+               }
+
+               for (ReceiveCommand cmd : pending) {
+                       if (cmd.getResult() == NOT_ATTEMPTED) {
+                               return true;
+                       }
+               }
+
+               return false;
+       }
+
+       private boolean checkExpected(List<ReceiveCommand> pending)
+                       throws IOException {
+               for (ReceiveCommand cmd : pending) {
+                       if (!matchOld(cmd, refDb.exactRef(cmd.getRefName()))) {
+                               cmd.setResult(LOCK_FAILURE);
+                               if (isAtomic()) {
+                                       ReceiveCommand.abort(pending);
+                                       return false;
+                               }
+                       }
+               }
+               return true;
+       }
+
+       private static boolean matchOld(ReceiveCommand cmd, @Nullable Ref ref) {
+               if (ref == null) {
+                       return AnyObjectId.isEqual(ObjectId.zeroId(), cmd.getOldId())
+                               && cmd.getOldSymref() == null;
+               } else if (ref.isSymbolic()) {
+                       return ref.getTarget().getName().equals(cmd.getOldSymref());
+               }
+               ObjectId id = ref.getObjectId();
+               if (id == null) {
+                       id = ObjectId.zeroId();
+               }
+               return cmd.getOldId().equals(id);
+       }
+
+       /**
+        * Writes the refs to the writer, and calls finish.
+        *
+        * @param writer
+        *            the writer on which we should write.
+        * @param newRefs
+        *            the ref data to write..
+        * @param pending
+        *            the log data to write.
+        * @throws IOException
+        *            in case of problems.
+        */
+       protected void write(ReftableWriter writer, List<Ref> newRefs,
+                       List<ReceiveCommand> pending) throws IOException {
+               long updateIndex = refDb.nextUpdateIndex();
+               writer.setMinUpdateIndex(updateIndex).setMaxUpdateIndex(updateIndex)
+                               .begin().sortAndWriteRefs(newRefs);
+               if (!isRefLogDisabled()) {
+                       writeLog(writer, updateIndex, pending);
+               }
+       }
+
+       private void writeLog(ReftableWriter writer, long updateIndex,
+                       List<ReceiveCommand> pending) throws IOException {
+               Map<String, ReceiveCommand> cmds = new HashMap<>();
+               List<String> byName = new ArrayList<>(pending.size());
+               for (ReceiveCommand cmd : pending) {
+                       cmds.put(cmd.getRefName(), cmd);
+                       byName.add(cmd.getRefName());
+               }
+               Collections.sort(byName);
+
+               PersonIdent ident = getRefLogIdent();
+               if (ident == null) {
+                       ident = new PersonIdent(repository);
+               }
+               for (String name : byName) {
+                       ReceiveCommand cmd = cmds.get(name);
+                       if (isRefLogDisabled(cmd)) {
+                               continue;
+                       }
+                       String msg = getRefLogMessage(cmd);
+                       if (isRefLogIncludingResult(cmd)) {
+                               String strResult = toResultString(cmd);
+                               if (strResult != null) {
+                                       msg = msg.isEmpty() ? strResult : msg + ": " + strResult; //$NON-NLS-1$
+                               }
+                       }
+                       writer.writeLog(name, updateIndex, ident, cmd.getOldId(),
+                                       cmd.getNewId(), msg);
+               }
+       }
+
+       private String toResultString(ReceiveCommand cmd) {
+               switch (cmd.getType()) {
+               case CREATE:
+                       return ReflogEntry.PREFIX_CREATED;
+               case UPDATE:
+                       // Match the behavior of a single RefUpdate. In that case, setting
+                       // the force bit completely bypasses the potentially expensive
+                       // isMergedInto check, by design, so the reflog message may be
+                       // inaccurate.
+                       //
+                       // Similarly, this class bypasses the isMergedInto checks when the
+                       // force bit is set, meaning we can't actually distinguish between
+                       // UPDATE and UPDATE_NONFASTFORWARD when isAllowNonFastForwards()
+                       // returns true.
+                       return isAllowNonFastForwards() ? ReflogEntry.PREFIX_FORCED_UPDATE
+                                       : ReflogEntry.PREFIX_FAST_FORWARD;
+               case UPDATE_NONFASTFORWARD:
+                       return ReflogEntry.PREFIX_FORCED_UPDATE;
+               default:
+                       return null;
+               }
+       }
+
+       // Extracts and peels the refs out of the ReceiveCommands
+       private static List<Ref> toNewRefs(RevWalk rw, List<ReceiveCommand> pending)
+               throws IOException {
+               List<Ref> refs = new ArrayList<>(pending.size());
+               for (ReceiveCommand cmd : pending) {
+                       if (cmd.getResult() != NOT_ATTEMPTED) {
+                               continue;
+                       }
+
+                       String name = cmd.getRefName();
+                       ObjectId newId = cmd.getNewId();
+                       String newSymref = cmd.getNewSymref();
+                       if (AnyObjectId.isEqual(ObjectId.zeroId(), newId)
+                               && newSymref == null) {
+                               refs.add(new ObjectIdRef.Unpeeled(NEW, name, null));
+                               continue;
+                       } else if (newSymref != null) {
+                               refs.add(new SymbolicRef(name,
+                                       new ObjectIdRef.Unpeeled(NEW, newSymref, null)));
+                               continue;
+                       }
+
+                       RevObject obj = rw.parseAny(newId);
+                       RevObject peel = null;
+                       if (obj instanceof RevTag) {
+                               peel = rw.peel(obj);
+                       }
+                       if (peel != null) {
+                               refs.add(new ObjectIdRef.PeeledTag(PACKED, name, newId,
+                                       peel.copy()));
+                       } else {
+                               refs.add(new ObjectIdRef.PeeledNonTag(PACKED, name, newId));
+                       }
+               }
+               return refs;
+       }
+}
diff --git a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/reftable/ReftableDatabase.java b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/reftable/ReftableDatabase.java
new file mode 100644 (file)
index 0000000..6c8d4fc
--- /dev/null
@@ -0,0 +1,295 @@
+package org.eclipse.jgit.internal.storage.reftable;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.eclipse.jgit.annotations.Nullable;
+import org.eclipse.jgit.lib.ObjectId;
+import org.eclipse.jgit.lib.Ref;
+import org.eclipse.jgit.lib.RefDatabase;
+import org.eclipse.jgit.lib.ReflogReader;
+import org.eclipse.jgit.transport.ReceiveCommand;
+
+/**
+ * Operations on {@link MergedReftable} that is common to various reftable-using
+ * subclasses of {@link RefDatabase}. See
+ * {@link org.eclipse.jgit.internal.storage.dfs.DfsReftableDatabase} for an
+ * example.
+ */
+public abstract class ReftableDatabase {
+       // Protects mergedTables.
+       private final ReentrantLock lock = new ReentrantLock(true);
+
+       private Reftable mergedTables;
+
+       /**
+        * ReftableDatabase lazily initializes its merged reftable on the first read after
+        * construction or clearCache() call. This function should always instantiate a new
+        * MergedReftable based on the list of reftables specified by the underlying storage.
+        *
+        * @return the ReftableStack for this instance
+        * @throws IOException
+        *             on I/O problems.
+        */
+       abstract protected MergedReftable openMergedReftable() throws IOException;
+
+       /**
+        * @return the next available logical timestamp for an additional reftable
+        *         in the stack.
+        * @throws java.io.IOException
+        *             on I/O problems.
+        */
+       public long nextUpdateIndex() throws IOException {
+               lock.lock();
+               try {
+                       return reader().maxUpdateIndex() + 1;
+               } finally {
+                       lock.unlock();
+               }
+       }
+
+       /**
+        * @return a ReflogReader for the given ref
+        * @param refname
+        *            the name of the ref.
+        * @throws IOException
+        *             on I/O problems
+        */
+       public ReflogReader getReflogReader(String refname) throws IOException {
+               lock.lock();
+               try {
+                       return new ReftableReflogReader(lock, reader(), refname);
+               } finally {
+                       lock.unlock();
+               }
+       }
+
+       /**
+        * @return a ReceiveCommand for the change from oldRef to newRef
+        * @param oldRef
+        *            a ref
+        * @param newRef
+        *            a ref
+        */
+       public static ReceiveCommand toCommand(Ref oldRef, Ref newRef) {
+               ObjectId oldId = toId(oldRef);
+               ObjectId newId = toId(newRef);
+               String name = oldRef != null ? oldRef.getName() : newRef.getName();
+
+               if (oldRef != null && oldRef.isSymbolic()) {
+                       if (newRef != null) {
+                               if (newRef.isSymbolic()) {
+                                       return ReceiveCommand.link(oldRef.getTarget().getName(),
+                                                       newRef.getTarget().getName(), name);
+                               } else {
+                                       // This should pass in oldId for compat with
+                                       // RefDirectoryUpdate
+                                       return ReceiveCommand.unlink(oldRef.getTarget().getName(),
+                                                       newId, name);
+                               }
+                       } else {
+                               return ReceiveCommand.unlink(oldRef.getTarget().getName(),
+                                               ObjectId.zeroId(), name);
+                       }
+               }
+
+               if (newRef != null && newRef.isSymbolic()) {
+                       if (oldRef != null) {
+                               if (oldRef.isSymbolic()) {
+                                       return ReceiveCommand.link(oldRef.getTarget().getName(),
+                                                       newRef.getTarget().getName(), name);
+                               } else {
+                                       return ReceiveCommand.link(oldId,
+                                                       newRef.getTarget().getName(), name);
+                               }
+                       } else {
+                               return ReceiveCommand.link(ObjectId.zeroId(),
+                                               newRef.getTarget().getName(), name);
+                       }
+               }
+
+               return new ReceiveCommand(oldId, newId, name);
+       }
+
+       private static ObjectId toId(Ref ref) {
+               if (ref != null) {
+                       ObjectId id = ref.getObjectId();
+                       if (id != null) {
+                               return id;
+                       }
+               }
+               return ObjectId.zeroId();
+       }
+
+       /**
+        * @return the lock protecting underlying ReftableReaders against concurrent
+        *         reads.
+        */
+       public ReentrantLock getLock() {
+               return lock;
+       }
+
+       /**
+        * @return the merged reftable that is implemented by the stack of
+        *         reftables. Return value must be accessed under lock.
+        * @throws IOException
+        *             on I/O problems
+        */
+       private Reftable reader() throws IOException {
+               assert lock.isLocked();
+               if (mergedTables == null) {
+                       mergedTables = openMergedReftable();
+               }
+               return mergedTables;
+       }
+
+       /**
+        * @return whether the given refName would be illegal in a repository that
+        *         uses loose refs.
+        * @param refName
+        *            the name to check
+        * @param added
+        *            a sorted set of refs we pretend have been added to the
+        *            database.
+        * @param deleted
+        *            a set of refs we pretend have been removed from the database.
+        * @throws IOException
+        *             on I/O problems
+        */
+       public boolean isNameConflicting(String refName, TreeSet<String> added,
+                       Set<String> deleted) throws IOException {
+               lock.lock();
+               try {
+                       Reftable table = reader();
+
+                       // Cannot be nested within an existing reference.
+                       int lastSlash = refName.lastIndexOf('/');
+                       while (0 < lastSlash) {
+                               String prefix = refName.substring(0, lastSlash);
+                               if (!deleted.contains(prefix)
+                                               && (table.hasRef(prefix) || added.contains(prefix))) {
+                                       return true;
+                               }
+                               lastSlash = refName.lastIndexOf('/', lastSlash - 1);
+                       }
+
+                       // Cannot be the container of an existing reference.
+                       String prefix = refName + '/';
+                       RefCursor c = table.seekRefsWithPrefix(prefix);
+                       while (c.next()) {
+                               if (!deleted.contains(c.getRef().getName())) {
+                                       return true;
+                               }
+                       }
+
+                       String it = added.ceiling(refName + '/');
+                       if (it != null && it.startsWith(prefix)) {
+                               return true;
+                       }
+                       return false;
+               } finally {
+                       lock.unlock();
+               }
+       }
+
+       /**
+        * Read a single reference.
+        * <p>
+        * This method expects an unshortened reference name and does not search
+        * using the standard search path.
+        *
+        * @param name
+        *            the unabbreviated name of the reference.
+        * @return the reference (if it exists); else {@code null}.
+        * @throws java.io.IOException
+        *             the reference space cannot be accessed.
+        */
+       @Nullable
+       public Ref exactRef(String name) throws IOException {
+               lock.lock();
+               try {
+                       Reftable table = reader();
+                       Ref ref = table.exactRef(name);
+                       if (ref != null && ref.isSymbolic()) {
+                               return table.resolve(ref);
+                       }
+                       return ref;
+               } finally {
+                       lock.unlock();
+               }
+       }
+
+       /**
+        * Returns refs whose names start with a given prefix.
+        *
+        * @param prefix
+        *            string that names of refs should start with; may be empty (to
+        *            return all refs).
+        * @return immutable list of refs whose names start with {@code prefix}.
+        * @throws java.io.IOException
+        *             the reference space cannot be accessed.
+        */
+       public List<Ref> getRefsByPrefix(String prefix) throws IOException {
+               List<Ref> all = new ArrayList<>();
+               lock.lock();
+               try {
+                       Reftable table = reader();
+                       try (RefCursor rc = RefDatabase.ALL.equals(prefix) ? table.allRefs()
+                                       : table.seekRefsWithPrefix(prefix)) {
+                               while (rc.next()) {
+                                       Ref ref = table.resolve(rc.getRef());
+                                       if (ref != null && ref.getObjectId() != null) {
+                                               all.add(ref);
+                                       }
+                               }
+                       }
+               } finally {
+                       lock.unlock();
+               }
+
+               return Collections.unmodifiableList(all);
+       }
+
+       /**
+        * Returns all refs that resolve directly to the given {@link ObjectId}.
+        * Includes peeled {@linkObjectId}s.
+        *
+        * @param id
+        *            {@link ObjectId} to resolve
+        * @return a {@link Set} of {@link Ref}s whose tips point to the provided
+        *         id.
+        * @throws java.io.IOException
+        *             on I/O errors.
+        */
+       public Set<Ref> getTipsWithSha1(ObjectId id) throws IOException {
+               lock.lock();
+               try {
+                       RefCursor cursor = reader().byObjectId(id);
+                       Set<Ref> refs = new HashSet<>();
+                       while (cursor.next()) {
+                               refs.add(cursor.getRef());
+                       }
+                       return refs;
+               } finally {
+                       lock.unlock();
+               }
+       }
+
+       /**
+        * Drops all data that might be cached in memory.
+        */
+       public void clearCache() {
+               lock.lock();
+               try {
+                       mergedTables = null;
+               } finally {
+                       lock.unlock();
+               }
+       }
+}