summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorHan-Wen Nienhuys <hanwen@google.com>2019-09-26 13:40:26 +0200
committerHan-Wen Nienhuys <hanwen@google.com>2019-10-16 19:44:30 +0200
commit2b1e942729617c45d2cb03b7556ab3d63253f64f (patch)
tree902bcb2c59895ef5fcb9e5c5221f551b5a95a0ba
parentd1d8bc30c213e6c416534a02921edada9d50cb25 (diff)
downloadjgit-2b1e942729617c45d2cb03b7556ab3d63253f64f.tar.gz
jgit-2b1e942729617c45d2cb03b7556ab3d63253f64f.zip
reftable: split off generic code from DFS code
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>
-rw-r--r--org.eclipse.jgit.http.test/src/org/eclipse/jgit/http/test/RefsUnreadableInMemoryRepository.java41
-rw-r--r--org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsReftableBatchRefUpdate.java196
-rw-r--r--org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsReftableDatabase.java231
-rw-r--r--org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/reftable/ReftableBatchRefUpdate.java (renamed from org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/ReftableBatchRefUpdate.java)330
-rw-r--r--org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/reftable/ReftableDatabase.java295
5 files changed, 721 insertions, 372 deletions
diff --git a/org.eclipse.jgit.http.test/src/org/eclipse/jgit/http/test/RefsUnreadableInMemoryRepository.java b/org.eclipse.jgit.http.test/src/org/eclipse/jgit/http/test/RefsUnreadableInMemoryRepository.java
index 78f909eeac..6f85979785 100644
--- a/org.eclipse.jgit.http.test/src/org/eclipse/jgit/http/test/RefsUnreadableInMemoryRepository.java
+++ b/org.eclipse.jgit.http.test/src/org/eclipse/jgit/http/test/RefsUnreadableInMemoryRepository.java
@@ -43,10 +43,14 @@
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
index 0000000000..124630edb7
--- /dev/null
+++ b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsReftableBatchRefUpdate.java
@@ -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;
+ }
+}
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
index cf3c71d431..2497719196 100644
--- 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
@@ -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/reftable/ReftableBatchRefUpdate.java
index b19ffffba9..592120d89b 100644
--- a/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/ReftableBatchRefUpdate.java
+++ b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/reftable/ReftableBatchRefUpdate.java
@@ -1,5 +1,5 @@
/*
- * Copyright (C) 2017, Google Inc.
+ * 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
@@ -41,40 +41,11 @@
* 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;
+package org.eclipse.jgit.internal.storage.reftable;
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.internal.JGitText;
import org.eclipse.jgit.lib.AnyObjectId;
import org.eclipse.jgit.lib.BatchRefUpdate;
import org.eclipse.jgit.lib.ObjectId;
@@ -82,43 +53,65 @@ 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;
-/**
- * {@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;
+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;
- private final DfsReftableDatabase refdb;
+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;
- private final DfsObjDatabase odb;
+/**
+ * {@link org.eclipse.jgit.lib.BatchRefUpdate} for Reftable based RefDatabase.
+ */
+public abstract class ReftableBatchRefUpdate extends BatchRefUpdate {
+ private final Lock lock;
- private final ReentrantLock lock;
+ private final ReftableDatabase refDb;
- private final ReftableConfig reftableConfig;
+ private final Repository repository;
/**
- * Initialize batch update.
+ * Initialize.
*
* @param refdb
- * database the update will modify.
- * @param odb
- * object database to store the reftable.
+ * 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(DfsReftableDatabase refdb,
- DfsObjDatabase odb) {
+ protected ReftableBatchRefUpdate(RefDatabase refdb, ReftableDatabase reftableDb, Lock lock,
+ Repository repository) {
super(refdb);
- this.refdb = refdb;
- this.odb = odb;
- lock = refdb.getLock();
- reftableConfig = refdb.getReftableConfig();
+ this.refDb = reftableDb;
+ this.lock = lock;
+ this.repository = repository;
}
/** {@inheritDoc} */
@@ -135,24 +128,36 @@ public class ReftableBatchRefUpdate extends BatchRefUpdate {
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;
}
- applyUpdates(rw, pending);
+
+ List<Ref> newRefs = toNewRefs(rw, pending);
+ applyUpdates(newRefs, pending);
for (ReceiveCommand cmd : pending) {
- cmd.setResult(OK);
+ if (cmd.getResult() == NOT_ATTEMPTED) {
+ // XXX this is a bug in DFS ?
+ cmd.setResult(OK);
+ }
}
} finally {
lock.unlock();
@@ -163,6 +168,19 @@ public class ReftableBatchRefUpdate extends BatchRefUpdate {
}
}
+ /**
+ * 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);
}
@@ -180,8 +198,10 @@ public class ReftableBatchRefUpdate extends BatchRefUpdate {
// 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;
+ if (isAtomic()) {
+ ReceiveCommand.abort(pending);
+ return false;
+ }
}
}
return true;
@@ -196,8 +216,10 @@ public class ReftableBatchRefUpdate extends BatchRefUpdate {
cmd.updateType(rw);
if (cmd.getType() == UPDATE_NONFASTFORWARD) {
cmd.setResult(REJECTED_NONFASTFORWARD);
- ReceiveCommand.abort(pending);
- return false;
+ if (isAtomic()) {
+ ReceiveCommand.abort(pending);
+ return false;
+ }
}
}
return true;
@@ -205,40 +227,55 @@ public class ReftableBatchRefUpdate extends BatchRefUpdate {
private boolean checkConflicting(List<ReceiveCommand> pending)
throws IOException {
- Set<String> names = new HashSet<>();
- for (ReceiveCommand cmd : pending) {
- names.add(cmd.getRefName());
- }
+ 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)) {
- 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 (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 (!ok && isAtomic()) {
- ReceiveCommand.abort(pending);
- return false;
+
+ if (isAtomic()) {
+ if (!ok) {
+ pending.stream()
+ .filter(cmd -> cmd.getResult() == NOT_ATTEMPTED)
+ .forEach(cmd -> cmd.setResult(LOCK_FAILURE));
+ }
+ return ok;
}
- 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()))) {
+ if (!matchOld(cmd, refDb.exactRef(cmd.getRefName()))) {
cmd.setResult(LOCK_FAILURE);
if (isAtomic()) {
ReceiveCommand.abort(pending);
@@ -252,7 +289,7 @@ public class ReftableBatchRefUpdate extends BatchRefUpdate {
private static boolean matchOld(ReceiveCommand cmd, @Nullable Ref ref) {
if (ref == null) {
return AnyObjectId.isEqual(ObjectId.zeroId(), cmd.getOldId())
- && cmd.getOldSymref() == null;
+ && cmd.getOldSymref() == null;
} else if (ref.isSymbolic()) {
return ref.getTarget().getName().equals(cmd.getOldSymref());
}
@@ -263,47 +300,26 @@ public class ReftableBatchRefUpdate extends BatchRefUpdate {
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)
+ /**
+ * 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);
}
- writer.finish();
- return writer.getStats();
}
private void writeLog(ReftableWriter writer, long updateIndex,
@@ -318,7 +334,7 @@ public class ReftableBatchRefUpdate extends BatchRefUpdate {
PersonIdent ident = getRefLogIdent();
if (ident == null) {
- ident = new PersonIdent(refdb.getRepository());
+ ident = new PersonIdent(repository);
}
for (String name : byName) {
ReceiveCommand cmd = cmds.get(name);
@@ -360,20 +376,25 @@ public class ReftableBatchRefUpdate extends BatchRefUpdate {
}
}
+ // Extracts and peels the refs out of the ReceiveCommands
private static List<Ref> toNewRefs(RevWalk rw, List<ReceiveCommand> pending)
- throws IOException {
+ 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) {
+ && 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)));
+ new ObjectIdRef.Unpeeled(NEW, newSymref, null)));
continue;
}
@@ -384,76 +405,11 @@ public class ReftableBatchRefUpdate extends BatchRefUpdate {
}
if (peel != null) {
refs.add(new ObjectIdRef.PeeledTag(PACKED, name, newId,
- peel.copy()));
+ 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/ReftableDatabase.java b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/reftable/ReftableDatabase.java
new file mode 100644
index 0000000000..6c8d4fc03e
--- /dev/null
+++ b/org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/reftable/ReftableDatabase.java
@@ -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();
+ }
+ }
+}