dfs: reftable backed DfsRefDatabase

DfsReftableDatabase is a new alternative for DfsRefDatabase that
handles more operations for the implementor by delegating through
reftable. All reftable files are stored in sibling DfsObjDatabase
using PackExt.REFTABLE and PackSource.INSERT.

Its assumed the DfsObjDatabase periodically runs compactions and GCs
using DfsPackCompactor and DfsGarbageCollector.  Those passes are
essential to collapsing the stack of reftables.

Change-Id: Ia03196ff6fd9ae2d0623c3747cfa84357c6d0c79
diff --git a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsGarbageCollector.java b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsGarbageCollector.java
index 87312ad..ff24edb 100644
--- a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsGarbageCollector.java
+++ b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsGarbageCollector.java
@@ -352,6 +352,11 @@
 				packGarbage(pm);
 				objdb.commitPack(newPackDesc, toPrune());
 				rollback = false;
+
+				if (refdb instanceof DfsReftableDatabase) {
+					objdb.clearCache();
+					((DfsReftableDatabase) refdb).clearCache();
+				}
 				return true;
 			} finally {
 				if (rollback)
diff --git a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsObjDatabase.java b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsObjDatabase.java
index 9439822..6e9d7e0 100644
--- a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsObjDatabase.java
+++ b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsObjDatabase.java
@@ -52,6 +52,7 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.eclipse.jgit.internal.storage.pack.PackExt;
@@ -457,6 +458,31 @@
 		} while (!packList.compareAndSet(o, n));
 	}
 
+	void addReftable(DfsPackDescription add, Set<DfsPackDescription> remove)
+			throws IOException {
+		PackList o, n;
+		do {
+			o = packList.get();
+			if (o == NO_PACKS) {
+				o = scanPacks(o);
+				for (DfsReftable t : o.reftables) {
+					if (t.getPackDescription().equals(add)) {
+						return;
+					}
+				}
+			}
+
+			List<DfsReftable> tables = new ArrayList<>(1 + o.reftables.length);
+			for (DfsReftable t : o.reftables) {
+				if (!remove.contains(t.getPackDescription())) {
+					tables.add(t);
+				}
+			}
+			tables.add(new DfsReftable(add));
+			n = new PackListImpl(o.packs, tables.toArray(new DfsReftable[0]));
+		} while (!packList.compareAndSet(o, n));
+	}
+
 	PackList scanPacks(final PackList original) throws IOException {
 		PackList o, n;
 		synchronized (packList) {
diff --git a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsPackCompactor.java b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsPackCompactor.java
index 963ba8a..ddb5380 100644
--- a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsPackCompactor.java
+++ b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsPackCompactor.java
@@ -72,6 +72,7 @@
 import org.eclipse.jgit.lib.ObjectId;
 import org.eclipse.jgit.lib.ObjectIdSet;
 import org.eclipse.jgit.lib.ProgressMonitor;
+import org.eclipse.jgit.lib.RefDatabase;
 import org.eclipse.jgit.revwalk.RevFlag;
 import org.eclipse.jgit.revwalk.RevObject;
 import org.eclipse.jgit.revwalk.RevWalk;
@@ -258,6 +259,12 @@
 			if (!commit.isEmpty() || !remove.isEmpty()) {
 				objdb.commitPack(commit, remove);
 			}
+
+			RefDatabase refdb = repo.getRefDatabase();
+			if (refdb instanceof DfsReftableDatabase) {
+				objdb.clearCache();
+				((DfsReftableDatabase) refdb).clearCache();
+			}
 		} finally {
 			rw = null;
 		}
diff --git a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsRefDatabase.java b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsRefDatabase.java
index b41c18b..d11286a 100644
--- a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsRefDatabase.java
+++ b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsRefDatabase.java
@@ -180,7 +180,7 @@
 		return recreate(ref, newLeaf);
 	}
 
-	private Ref doPeel(final Ref leaf) throws MissingObjectException,
+	Ref doPeel(Ref leaf) throws MissingObjectException,
 			IOException {
 		try (RevWalk rw = new RevWalk(repository)) {
 			RevObject obj = rw.parseAny(leaf.getObjectId());
@@ -199,7 +199,7 @@
 		}
 	}
 
-	private static Ref recreate(Ref old, Ref leaf) {
+	static Ref recreate(Ref old, Ref leaf) {
 		if (old.isSymbolic()) {
 			Ref dst = recreate(old.getTarget(), leaf);
 			return new SymbolicRef(old.getName(), dst);
diff --git a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsReftableDatabase.java b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsReftableDatabase.java
new file mode 100644
index 0000000..76255b7
--- /dev/null
+++ b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsReftableDatabase.java
@@ -0,0 +1,374 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.eclipse.jgit.annotations.Nullable;
+import org.eclipse.jgit.internal.storage.pack.PackExt;
+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.lib.BatchRefUpdate;
+import org.eclipse.jgit.lib.NullProgressMonitor;
+import org.eclipse.jgit.lib.ObjectId;
+import org.eclipse.jgit.lib.Ref;
+import org.eclipse.jgit.revwalk.RevWalk;
+import org.eclipse.jgit.transport.ReceiveCommand;
+import org.eclipse.jgit.util.RefList;
+import org.eclipse.jgit.util.RefMap;
+
+/**
+ * A {@link DfsRefDatabase} that uses reftable for storage.
+ * <p>
+ * A {@code DfsRefDatabase} instance is thread-safe.
+ * <p>
+ * Implementors may wish to use {@link DfsPackDescription#getMaxUpdateIndex()}
+ * as the primary key identifier for a {@link PackExt#REFTABLE} only pack
+ * description, ensuring that when there are competing transactions one wins,
+ * and one will fail.
+ */
+public class DfsReftableDatabase extends DfsRefDatabase {
+	private final ReentrantLock lock = new ReentrantLock(true);
+	private DfsReader ctx;
+	private ReftableStack tableStack;
+	private MergedReftable mergedTables;
+
+	/**
+	 * Initialize the reference database for a repository.
+	 *
+	 * @param repo
+	 *            the repository this database instance manages references for.
+	 */
+	protected DfsReftableDatabase(DfsRepository repo) {
+		super(repo);
+	}
+
+	@Override
+	public boolean performsAtomicTransactions() {
+		return true;
+	}
+
+	@Override
+	public BatchRefUpdate newBatchUpdate() {
+		DfsObjDatabase odb = getRepository().getObjectDatabase();
+		return new ReftableBatchRefUpdate(this, odb);
+	}
+
+	/** @return the lock protecting this instance's state. */
+	protected ReentrantLock getLock() {
+		return lock;
+	}
+
+	/**
+	 * @return {@code true} if commit of a new small reftable should try to
+	 *         replace a prior small reftable by performing a compaction,
+	 *         instead of extending the stack depth.
+	 */
+	protected boolean compactDuringCommit() {
+		return true;
+	}
+
+	/**
+	 * Obtain a handle to the merged reader.
+	 *
+	 * @return (possibly cached) handle to the merged reader.
+	 * @throws IOException
+	 *             if tables cannot be opened.
+	 */
+	protected Reftable read() 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.
+	 *
+	 * @return (possibly cached) handle to the stack.
+	 * @throws IOException
+	 *             if tables cannot be opened.
+	 */
+	protected ReftableStack stack() throws IOException {
+		lock.lock();
+		try {
+			if (tableStack == null) {
+				DfsObjDatabase odb = getRepository().getObjectDatabase();
+				if (ctx == null) {
+					ctx = odb.newReader();
+				}
+				if (tableStack == null) {
+					tableStack = ReftableStack.open(ctx,
+							Arrays.asList(odb.getReftables()));
+				}
+			}
+			return tableStack;
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	public boolean isNameConflicting(String refName) throws IOException {
+		lock.lock();
+		try {
+			Reftable table = read();
+
+			// 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.hasRef(refName + '/');
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	public Ref exactRef(String name) throws IOException {
+		lock.lock();
+		try {
+			Reftable table = read();
+			Ref ref = table.exactRef(name);
+			if (ref != null && ref.isSymbolic()) {
+				return table.resolve(ref);
+			}
+			return ref;
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	public Ref getRef(String needle) throws IOException {
+		for (String prefix : SEARCH_PATH) {
+			Ref ref = exactRef(prefix + needle);
+			if (ref != null) {
+				return ref;
+			}
+		}
+		return null;
+	}
+
+	@Override
+	public Map<String, Ref> getRefs(String prefix) throws IOException {
+		RefList.Builder<Ref> all = new RefList.Builder<>();
+		lock.lock();
+		try {
+			Reftable table = read();
+			try (RefCursor rc = ALL.equals(prefix)
+					? table.allRefs()
+					: table.seekRef(prefix)) {
+				while (rc.next()) {
+					Ref ref = table.resolve(rc.getRef());
+					if (ref != null) {
+						all.add(ref);
+					}
+				}
+			}
+		} finally {
+			lock.unlock();
+		}
+
+		RefList<Ref> none = RefList.emptyList();
+		return new RefMap(prefix, all.toRefList(), none, none);
+	}
+
+	@Override
+	public Ref peel(Ref ref) throws IOException {
+		Ref oldLeaf = ref.getLeaf();
+		if (oldLeaf.isPeeled() || oldLeaf.getObjectId() == null) {
+			return ref;
+		}
+		return recreate(ref, doPeel(oldLeaf));
+	}
+
+	@Override
+	boolean exists() throws IOException {
+		DfsObjDatabase odb = getRepository().getObjectDatabase();
+		return odb.getReftables().length > 0;
+	}
+
+	@Override
+	void clearCache() {
+		lock.lock();
+		try {
+			if (tableStack != null) {
+				tableStack.close();
+				tableStack = null;
+			}
+			if (ctx != null) {
+				ctx.close();
+				ctx = null;
+			}
+			mergedTables = null;
+		} finally {
+			lock.unlock();
+		}
+	}
+
+	@Override
+	protected RefCache scanAllRefs() throws IOException {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	protected boolean compareAndPut(Ref oldRef, @Nullable Ref newRef)
+			throws IOException {
+		ReceiveCommand cmd = toCommand(oldRef, newRef);
+		try (RevWalk rw = new RevWalk(getRepository())) {
+			newBatchUpdate()
+				.setAllowNonFastForwards(true)
+				.addCommand(cmd)
+				.execute(rw, NullProgressMonitor.INSTANCE);
+		}
+		switch (cmd.getResult()) {
+		case OK:
+			return true;
+		case REJECTED_OTHER_REASON:
+			throw new IOException(cmd.getMessage());
+		case LOCK_FAILURE:
+		default:
+			return false;
+		}
+	}
+
+	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) {
+		if (oldRef != null) {
+			return oldRef.getName();
+		} else if (newRef != null) {
+			return newRef.getName();
+		}
+		return null;
+	}
+
+	@Override
+	protected boolean compareAndRemove(Ref oldRef) throws IOException {
+		return compareAndPut(oldRef, null);
+	}
+
+	@Override
+	void stored(Ref ref) {
+		// Unnecessary; ReftableBatchRefUpdate calls clearCache().
+	}
+
+	@Override
+	void removed(String refName) {
+		// Unnecessary; ReftableBatchRefUpdate calls clearCache().
+	}
+
+	@Override
+	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
new file mode 100644
index 0000000..fa2e8a9
--- /dev/null
+++ b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/ReftableBatchRefUpdate.java
@@ -0,0 +1,465 @@
+/*
+ * 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.RefCursor;
+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.Config;
+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 BatchRefUpdate} for {@link 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();
+
+		Config cfg = refdb.getRepository().getConfig();
+		reftableConfig = new ReftableConfig();
+		reftableConfig.fromConfig(cfg);
+	}
+
+	@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 {
+				Reftable table = refdb.read();
+				if (!checkExpected(table, 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(Reftable table, List<ReceiveCommand> pending)
+			throws IOException {
+		for (ReceiveCommand cmd : pending) {
+			Ref ref;
+			try (RefCursor rc = table.seekRef(cmd.getRefName())) {
+				ref = rc.next() ? rc.getRef() : null;
+			}
+			if (!matchOld(cmd, ref)) {
+				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.equals(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)
+				.setMinUpdateIndex(updateIndex)
+				.setMaxUpdateIndex(updateIndex)
+				.begin(os)
+				.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.equals(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 {
+		ReftableStack stack = refdb.stack();
+		List<Reftable> 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) {
+			return false;
+		}
+		for (PackExt ext : PackExt.values()) {
+			if (ext != REFTABLE && desc.hasFileExt(ext)) {
+				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<Reftable> stack = refdb.stack().readers();
+		Reftable last = stack.get(stack.size() - 1);
+
+		List<Reftable> tables = new ArrayList<>(2);
+		tables.add(last);
+		tables.add(new ReftableReader(BlockSource.from(newTable)));
+
+		ReftableCompactor compactor = new ReftableCompactor();
+		compactor.setConfig(cfg);
+		compactor.addAll(tables);
+		compactor.compact(out);
+		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());
+	}
+}
diff --git a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/ReftableStack.java b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/ReftableStack.java
index 8d1cc98..3656884 100644
--- a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/ReftableStack.java
+++ b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/ReftableStack.java
@@ -58,18 +58,19 @@
 	 * @param ctx
 	 *            context to read the tables with. This {@code ctx} will be
 	 *            retained by the stack and each of the table readers.
-	 * @param tables
+	 * @param files
 	 *            the tables to open.
 	 * @return stack reference to close the tables.
 	 * @throws IOException
 	 *             a table could not be opened
 	 */
-	public static ReftableStack open(DfsReader ctx, List<DfsReftable> tables)
+	public static ReftableStack open(DfsReader ctx, List<DfsReftable> files)
 			throws IOException {
-		ReftableStack stack = new ReftableStack(tables.size());
+		ReftableStack stack = new ReftableStack(files.size());
 		boolean close = true;
 		try {
-			for (DfsReftable t : tables) {
+			for (DfsReftable t : files) {
+				stack.files.add(t);
 				stack.tables.add(t.open(ctx));
 			}
 			close = false;
@@ -81,13 +82,23 @@
 		}
 	}
 
+	private final List<DfsReftable> files;
 	private final List<Reftable> tables;
 
 	private ReftableStack(int tableCnt) {
+		this.files = new ArrayList<>(tableCnt);
 		this.tables = new ArrayList<>(tableCnt);
 	}
 
 	/**
+	 * @return unmodifiable list of DfsRefatble files, in the same order the
+	 *         files were passed to {@link #open(DfsReader, List)}.
+	 */
+	public List<DfsReftable> files() {
+		return Collections.unmodifiableList(files);
+	}
+
+	/**
 	 * @return unmodifiable list of tables, in the same order the files were
 	 *         passed to {@link #open(DfsReader, List)}.
 	 */