Browse Source

DfsBlockCache to lock while loading object references

We see the same index being loaded by multiple threads. Each is
hundreds of MB and takes several seconds to load, causing server to
run out of memory.  This change introduces a lock to avoid these
duplicate works.  It uses a new set of locks similar in implementation
to the loadLocks for getOrLoad of blocks.  The locks are kept separate
to prevent long-running index loading from blocking out fast block
loading. The cache instance can be configured with a consumer to
monitor the wait time of the new locks.

Change-Id: I44962fe84093456962d5981545e3f7851ecb6e43
Signed-off-by: Minh Thai <mthai@google.com>
stable-5.3
Minh Thai 6 years ago
parent
commit
8bc9acf264
  1. 141
      org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsBlockCache.java
  2. 24
      org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsBlockCacheConfig.java
  3. 268
      org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsPackFile.java

141
org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsBlockCache.java

@ -49,6 +49,7 @@ import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.atomic.AtomicReferenceArray;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Consumer;
import java.util.stream.LongStream;
import org.eclipse.jgit.annotations.Nullable;
@ -128,9 +129,18 @@ public final class DfsBlockCache {
/** Hash bucket directory; entries are chained below. */
private final AtomicReferenceArray<HashEntry> table;
/** Locks to prevent concurrent loads for same (PackFile,position). */
/**
* Locks to prevent concurrent loads for same (PackFile,position) block. The
* number of locks is {@link DfsBlockCacheConfig#getConcurrencyLevel()} to
* cap the overall concurrent block loads.
*/
private final ReentrantLock[] loadLocks;
/**
* A separate pool of locks to prevent concurrent loads for same index or bitmap from PackFile.
*/
private final ReentrantLock[] refLocks;
/** Maximum number of bytes the cache should hold. */
private final long maxBytes;
@ -177,19 +187,30 @@ public final class DfsBlockCache {
/** Protects the clock and its related data. */
private final ReentrantLock clockLock;
/**
* A consumer of object reference lock wait time milliseconds. May be used to build a metric.
*/
private final Consumer<Long> refLockWaitTime;
/** Current position of the clock. */
private Ref clockHand;
@SuppressWarnings("unchecked")
private DfsBlockCache(DfsBlockCacheConfig cfg) {
tableSize = tableSize(cfg);
if (tableSize < 1)
if (tableSize < 1) {
throw new IllegalArgumentException(JGitText.get().tSizeMustBeGreaterOrEqual1);
}
table = new AtomicReferenceArray<>(tableSize);
loadLocks = new ReentrantLock[cfg.getConcurrencyLevel()];
for (int i = 0; i < loadLocks.length; i++)
for (int i = 0; i < loadLocks.length; i++) {
loadLocks[i] = new ReentrantLock(true /* fair */);
}
refLocks = new ReentrantLock[cfg.getConcurrencyLevel()];
for (int i = 0; i < refLocks.length; i++) {
refLocks[i] = new ReentrantLock(true /* fair */);
}
maxBytes = cfg.getBlockLimit();
maxStreamThroughCache = (long) (maxBytes * cfg.getStreamRatio());
@ -207,6 +228,8 @@ public final class DfsBlockCache {
statMiss = new AtomicReference<>(newCounters());
statEvict = new AtomicReference<>(newCounters());
liveBytes = new AtomicReference<>(newCounters());
refLockWaitTime = cfg.getRefLockWaitTimeConsumer();
}
boolean shouldCopyThroughCache(long length) {
@ -333,15 +356,17 @@ public final class DfsBlockCache {
private static int tableSize(DfsBlockCacheConfig cfg) {
final int wsz = cfg.getBlockSize();
final long limit = cfg.getBlockLimit();
if (wsz <= 0)
if (wsz <= 0) {
throw new IllegalArgumentException(JGitText.get().invalidWindowSize);
if (limit < wsz)
}
if (limit < wsz) {
throw new IllegalArgumentException(JGitText.get().windowSizeMustBeLesserThanLimit);
}
return (int) Math.min(5 * (limit / wsz) / 2, Integer.MAX_VALUE);
}
/**
* Lookup a cached object, creating and loading it if it doesn't exist.
* Look up a cached object, creating and loading it if it doesn't exist.
*
* @param file
* the pack that "contains" the cached object.
@ -391,8 +416,9 @@ public final class DfsBlockCache {
v = file.readOneBlock(requestedPosition, ctx, fileChannel);
credit = false;
} finally {
if (credit)
if (credit) {
creditSpace(blockSize, key);
}
}
if (position != v.start) {
// The file discovered its blockSize and adjusted.
@ -405,8 +431,9 @@ public final class DfsBlockCache {
ref.hot = true;
for (;;) {
HashEntry n = new HashEntry(clean(e2), ref);
if (table.compareAndSet(slot, e2, n))
if (table.compareAndSet(slot, e2, n)) {
break;
}
e2 = table.get(slot);
}
addToClock(ref, blockSize - v.size());
@ -416,8 +443,9 @@ public final class DfsBlockCache {
// If the block size changed from the default, it is possible the block
// that was loaded is the wrong block for the requested position.
if (v.contains(file.key, requestedPosition))
if (v.contains(file.key, requestedPosition)) {
return v;
}
return getOrLoad(file, requestedPosition, ctx, fileChannel);
}
@ -488,6 +516,63 @@ public final class DfsBlockCache {
put(v.stream, v.start, v.size(), v);
}
/**
* Look up a cached object, creating and loading it if it doesn't exist.
*
* @param key
* the stream key of the pack.
* @param loader
* the function to load the reference.
* @return the object reference.
* @throws IOException
* the reference was not in the cache and could not be loaded.
*/
<T> Ref<T> getOrLoadRef(DfsStreamKey key, RefLoader<T> loader)
throws IOException {
int slot = slot(key, 0);
HashEntry e1 = table.get(slot);
Ref<T> ref = scanRef(e1, key, 0);
if (ref != null) {
getStat(statHit, key).incrementAndGet();
return ref;
}
ReentrantLock regionLock = lockForRef(key);
long lockStart = System.currentTimeMillis();
regionLock.lock();
try {
HashEntry e2 = table.get(slot);
if (e2 != e1) {
ref = scanRef(e2, key, 0);
if (ref != null) {
getStat(statHit, key).incrementAndGet();
return ref;
}
}
if (refLockWaitTime != null) {
refLockWaitTime.accept(
Long.valueOf(System.currentTimeMillis() - lockStart));
}
getStat(statMiss, key).incrementAndGet();
ref = loader.load();
ref.hot = true;
// Reserve after loading to get the size of the object
reserveSpace(ref.size, key);
for (;;) {
HashEntry n = new HashEntry(clean(e2), ref);
if (table.compareAndSet(slot, e2, n)) {
break;
}
e2 = table.get(slot);
}
addToClock(ref, 0);
} finally {
regionLock.unlock();
}
return ref;
}
<T> Ref<T> putRef(DfsStreamKey key, long size, T v) {
return put(key, 0, (int) Math.min(size, Integer.MAX_VALUE), v);
}
@ -496,8 +581,9 @@ public final class DfsBlockCache {
int slot = slot(key, pos);
HashEntry e1 = table.get(slot);
Ref<T> ref = scanRef(e1, key, pos);
if (ref != null)
if (ref != null) {
return ref;
}
reserveSpace(size, key);
ReentrantLock regionLock = lockFor(key, pos);
@ -516,8 +602,9 @@ public final class DfsBlockCache {
ref.hot = true;
for (;;) {
HashEntry n = new HashEntry(clean(e2), ref);
if (table.compareAndSet(slot, e2, n))
if (table.compareAndSet(slot, e2, n)) {
break;
}
e2 = table.get(slot);
}
addToClock(ref, 0);
@ -534,10 +621,11 @@ public final class DfsBlockCache {
@SuppressWarnings("unchecked")
<T> T get(DfsStreamKey key, long position) {
T val = (T) scan(table.get(slot(key, position)), key, position);
if (val == null)
if (val == null) {
getStat(statMiss, key).incrementAndGet();
else
} else {
getStat(statHit, key).incrementAndGet();
}
return val;
}
@ -546,21 +634,13 @@ public final class DfsBlockCache {
return r != null ? r.get() : null;
}
<T> Ref<T> getRef(DfsStreamKey key) {
Ref<T> r = scanRef(table.get(slot(key, 0)), key, 0);
if (r != null)
getStat(statHit, key).incrementAndGet();
else
getStat(statMiss, key).incrementAndGet();
return r;
}
@SuppressWarnings("unchecked")
private <T> Ref<T> scanRef(HashEntry n, DfsStreamKey key, long position) {
for (; n != null; n = n.next) {
Ref<T> r = n.ref;
if (r.position == position && r.key.equals(key))
if (r.position == position && r.key.equals(key)) {
return r.get() != null ? r : null;
}
}
return null;
}
@ -573,6 +653,10 @@ public final class DfsBlockCache {
return loadLocks[(hash(key.hash, position) >>> 1) % loadLocks.length];
}
private ReentrantLock lockForRef(DfsStreamKey key) {
return refLocks[(key.hash >>> 1) % refLocks.length];
}
private static AtomicLong[] newCounters() {
AtomicLong[] ret = new AtomicLong[PackExt.values().length];
for (int i = 0; i < ret.length; i++) {
@ -613,8 +697,9 @@ public final class DfsBlockCache {
private static HashEntry clean(HashEntry top) {
while (top != null && top.ref.next == null)
top = top.next;
if (top == null)
if (top == null) {
return null;
}
HashEntry n = clean(top.next);
return n == top.next ? top : new HashEntry(n, top.ref);
}
@ -649,8 +734,9 @@ public final class DfsBlockCache {
T get() {
T v = value;
if (v != null)
if (v != null) {
hot = true;
}
return v;
}
@ -658,4 +744,9 @@ public final class DfsBlockCache {
return value != null;
}
}
@FunctionalInterface
interface RefLoader<T> {
Ref<T> load() throws IOException;
}
}

24
org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsBlockCacheConfig.java

@ -51,6 +51,7 @@ import static org.eclipse.jgit.lib.ConfigConstants.CONFIG_KEY_CONCURRENCY_LEVEL;
import static org.eclipse.jgit.lib.ConfigConstants.CONFIG_KEY_STREAM_RATIO;
import java.text.MessageFormat;
import java.util.function.Consumer;
import org.eclipse.jgit.internal.JGitText;
import org.eclipse.jgit.lib.Config;
@ -71,6 +72,8 @@ public class DfsBlockCacheConfig {
private double streamRatio;
private int concurrencyLevel;
private Consumer<Long> refLock;
/**
* Create a default configuration.
*/
@ -193,6 +196,27 @@ public class DfsBlockCacheConfig {
return this;
}
/**
* Get the consumer of the object reference lock wait time in milliseconds.
*
* @return consumer of wait time in milliseconds.
*/
public Consumer<Long> getRefLockWaitTimeConsumer() {
return refLock;
}
/**
* Set the consumer for lock wait time.
*
* @param c
* consumer of wait time in milliseconds.
* @return {@code this}
*/
public DfsBlockCacheConfig setReflockWaitTimeConsumer(Consumer<Long> c) {
refLock = c;
return this;
}
/**
* Update properties by setting fields from the configuration.
* <p>

268
org.eclipse.jgit/src/org/eclipse/jgit/internal/storage/dfs/DfsPackFile.java

@ -88,6 +88,8 @@ import org.eclipse.jgit.util.LongList;
* objects are similar.
*/
public final class DfsPackFile extends BlockBasedFile {
private static final int REC_SIZE = Constants.OBJECT_ID_LENGTH + 8;
/**
* Lock for initialization of {@link #index} and {@link #corruptObjects}.
* <p>
@ -177,64 +179,71 @@ public final class DfsPackFile extends BlockBasedFile {
DfsBlockCache.Ref<PackIndex> idxref = index;
if (idxref != null) {
PackIndex idx = idxref.get();
if (idx != null)
if (idx != null) {
return idx;
}
}
if (invalid)
if (invalid) {
throw new PackInvalidException(getFileName());
}
Repository.getGlobalListenerList()
.dispatch(new BeforeDfsPackIndexLoadedEvent(this));
synchronized (initLock) {
idxref = index;
if (idxref != null) {
PackIndex idx = idxref.get();
if (idx != null)
return idx;
}
DfsStreamKey idxKey = desc.getStreamKey(INDEX);
idxref = cache.getRef(idxKey);
if (idxref != null) {
PackIndex idx = idxref.get();
if (idx != null) {
index = idxref;
return idx;
}
}
PackIndex idx;
DfsStreamKey idxKey = desc.getStreamKey(INDEX);
try {
ctx.stats.readIdx++;
long start = System.nanoTime();
try (ReadableChannel rc = ctx.db.openFile(desc, INDEX)) {
InputStream in = Channels.newInputStream(rc);
int wantSize = 8192;
int bs = rc.blockSize();
if (0 < bs && bs < wantSize)
bs = (wantSize / bs) * bs;
else if (bs <= 0)
bs = wantSize;
idx = PackIndex.read(new BufferedInputStream(in, bs));
ctx.stats.readIdxBytes += rc.position();
} finally {
ctx.stats.readIdxMicros += elapsedMicros(start);
}
} catch (EOFException e) {
invalid = true;
throw new IOException(MessageFormat.format(
DfsText.get().shortReadOfIndex,
desc.getFileName(INDEX)), e);
idxref = cache.getOrLoadRef(idxKey, () -> {
try {
ctx.stats.readIdx++;
long start = System.nanoTime();
try (ReadableChannel rc = ctx.db.openFile(desc,
INDEX)) {
InputStream in = Channels.newInputStream(rc);
int wantSize = 8192;
int bs = rc.blockSize();
if (0 < bs && bs < wantSize) {
bs = (wantSize / bs) * bs;
} else if (bs <= 0) {
bs = wantSize;
}
PackIndex idx = PackIndex
.read(new BufferedInputStream(in, bs));
int sz = (int) Math.min(
idx.getObjectCount() * REC_SIZE,
Integer.MAX_VALUE);
ctx.stats.readIdxBytes += rc.position();
return new DfsBlockCache.Ref<>(idxKey, 0, sz, idx);
} finally {
ctx.stats.readIdxMicros += elapsedMicros(start);
}
} catch (EOFException e) {
throw new IOException(MessageFormat.format(
DfsText.get().shortReadOfIndex,
desc.getFileName(INDEX)), e);
} catch (IOException e) {
throw new IOException(MessageFormat.format(
DfsText.get().cannotReadIndex,
desc.getFileName(INDEX)), e);
}
});
} catch (IOException e) {
invalid = true;
throw new IOException(MessageFormat.format(
DfsText.get().cannotReadIndex,
desc.getFileName(INDEX)), e);
throw e;
}
PackIndex idx = idxref.get();
if (idx != null) {
index = idxref;
}
setPackIndex(idx);
return idx;
}
}
@ -244,67 +253,71 @@ public final class DfsPackFile extends BlockBasedFile {
}
PackBitmapIndex getBitmapIndex(DfsReader ctx) throws IOException {
if (invalid || isGarbage() || !desc.hasFileExt(BITMAP_INDEX))
if (invalid || isGarbage() || !desc.hasFileExt(BITMAP_INDEX)) {
return null;
}
DfsBlockCache.Ref<PackBitmapIndex> idxref = bitmapIndex;
if (idxref != null) {
PackBitmapIndex idx = idxref.get();
if (idx != null)
return idx;
PackBitmapIndex bmidx = idxref.get();
if (bmidx != null) {
return bmidx;
}
}
synchronized (initLock) {
idxref = bitmapIndex;
if (idxref != null) {
PackBitmapIndex idx = idxref.get();
if (idx != null)
return idx;
}
DfsStreamKey bitmapKey = desc.getStreamKey(BITMAP_INDEX);
idxref = cache.getRef(bitmapKey);
if (idxref != null) {
PackBitmapIndex idx = idxref.get();
if (idx != null) {
bitmapIndex = idxref;
return idx;
PackBitmapIndex bmidx = idxref.get();
if (bmidx != null) {
return bmidx;
}
}
long size;
PackBitmapIndex idx;
ctx.stats.readBitmap++;
long start = System.nanoTime();
try (ReadableChannel rc = ctx.db.openFile(desc, BITMAP_INDEX)) {
try {
InputStream in = Channels.newInputStream(rc);
int wantSize = 8192;
int bs = rc.blockSize();
if (0 < bs && bs < wantSize)
bs = (wantSize / bs) * bs;
else if (bs <= 0)
bs = wantSize;
in = new BufferedInputStream(in, bs);
idx = PackBitmapIndex.read(
in, idx(ctx), getReverseIdx(ctx));
} finally {
size = rc.position();
ctx.stats.readIdxBytes += size;
ctx.stats.readIdxMicros += elapsedMicros(start);
PackIndex idx = idx(ctx);
PackReverseIndex revidx = getReverseIdx(ctx);
DfsStreamKey bitmapKey = desc.getStreamKey(BITMAP_INDEX);
idxref = cache.getOrLoadRef(bitmapKey, () -> {
ctx.stats.readBitmap++;
long start = System.nanoTime();
try (ReadableChannel rc = ctx.db.openFile(desc, BITMAP_INDEX)) {
long size;
PackBitmapIndex bmidx;
try {
InputStream in = Channels.newInputStream(rc);
int wantSize = 8192;
int bs = rc.blockSize();
if (0 < bs && bs < wantSize) {
bs = (wantSize / bs) * bs;
} else if (bs <= 0) {
bs = wantSize;
}
in = new BufferedInputStream(in, bs);
bmidx = PackBitmapIndex.read(in, idx, revidx);
} finally {
size = rc.position();
ctx.stats.readIdxBytes += size;
ctx.stats.readIdxMicros += elapsedMicros(start);
}
int sz = (int) Math.min(size, Integer.MAX_VALUE);
return new DfsBlockCache.Ref<>(bitmapKey, 0, sz, bmidx);
} catch (EOFException e) {
throw new IOException(
MessageFormat.format(DfsText.get().shortReadOfIndex,
desc.getFileName(BITMAP_INDEX)),
e);
} catch (IOException e) {
throw new IOException(
MessageFormat.format(DfsText.get().cannotReadIndex,
desc.getFileName(BITMAP_INDEX)),
e);
}
} catch (EOFException e) {
throw new IOException(MessageFormat.format(
DfsText.get().shortReadOfIndex,
desc.getFileName(BITMAP_INDEX)), e);
} catch (IOException e) {
throw new IOException(MessageFormat.format(
DfsText.get().cannotReadIndex,
desc.getFileName(BITMAP_INDEX)), e);
});
PackBitmapIndex bmidx = idxref.get();
if (bmidx != null) {
bitmapIndex = idxref;
}
bitmapIndex = cache.putRef(bitmapKey, size, idx);
return idx;
return bmidx;
}
}
@ -312,33 +325,33 @@ public final class DfsPackFile extends BlockBasedFile {
DfsBlockCache.Ref<PackReverseIndex> revref = reverseIndex;
if (revref != null) {
PackReverseIndex revidx = revref.get();
if (revidx != null)
if (revidx != null) {
return revidx;
}
}
synchronized (initLock) {
revref = reverseIndex;
if (revref != null) {
PackReverseIndex revidx = revref.get();
if (revidx != null)
if (revidx != null) {
return revidx;
}
DfsStreamKey revKey =
new DfsStreamKey.ForReverseIndex(desc.getStreamKey(INDEX));
revref = cache.getRef(revKey);
if (revref != null) {
PackReverseIndex idx = revref.get();
if (idx != null) {
reverseIndex = revref;
return idx;
}
}
PackIndex idx = idx(ctx);
PackReverseIndex revidx = new PackReverseIndex(idx);
long cnt = idx.getObjectCount();
reverseIndex = cache.putRef(revKey, cnt * 8, revidx);
DfsStreamKey revKey = new DfsStreamKey.ForReverseIndex(
desc.getStreamKey(INDEX));
revref = cache.getOrLoadRef(revKey, () -> {
PackReverseIndex revidx = new PackReverseIndex(idx);
int sz = (int) Math.min(idx.getObjectCount() * 8,
Integer.MAX_VALUE);
return new DfsBlockCache.Ref<>(revKey, 0, sz, revidx);
});
PackReverseIndex revidx = revref.get();
if (revidx != null) {
reverseIndex = revref;
}
return revidx;
}
}
@ -417,10 +430,11 @@ public final class DfsPackFile extends BlockBasedFile {
return null;
}
if (ctx.inflate(this, position, dstbuf, false) != sz)
if (ctx.inflate(this, position, dstbuf, false) != sz) {
throw new EOFException(MessageFormat.format(
JGitText.get().shortCompressedStreamAt,
Long.valueOf(position)));
}
return dstbuf;
}
@ -431,10 +445,11 @@ public final class DfsPackFile extends BlockBasedFile {
ctx.pin(this, 0);
ctx.unpin();
}
if (cache.shouldCopyThroughCache(length))
if (cache.shouldCopyThroughCache(length)) {
copyPackThroughCache(out, ctx);
else
} else {
copyPackBypassCache(out, ctx);
}
}
private void copyPackThroughCache(PackOutputStream out, DfsReader ctx)
@ -477,8 +492,9 @@ public final class DfsPackFile extends BlockBasedFile {
throws IOException {
try (ReadableChannel rc = ctx.db.openFile(desc, PACK)) {
ByteBuffer buf = newCopyBuffer(out, rc);
if (ctx.getOptions().getStreamPackBufferSize() > 0)
if (ctx.getOptions().getStreamPackBufferSize() > 0) {
rc.setReadAheadBytes(ctx.getOptions().getStreamPackBufferSize());
}
long position = 12;
long remaining = length - (12 + 20);
boolean packHeadSkipped = false;
@ -497,10 +513,11 @@ public final class DfsPackFile extends BlockBasedFile {
buf.position(0);
int n = read(rc, buf);
if (n <= 0)
if (n <= 0) {
throw packfileIsTruncated();
else if (n > remaining)
} else if (n > remaining) {
n = (int) remaining;
}
if (!packHeadSkipped) {
// Need skip the 'PACK' header for the first read
@ -519,8 +536,9 @@ public final class DfsPackFile extends BlockBasedFile {
private ByteBuffer newCopyBuffer(PackOutputStream out, ReadableChannel rc) {
int bs = blockSize(rc);
byte[] copyBuf = out.getCopyBuffer();
if (bs > copyBuf.length)
if (bs > copyBuf.length) {
copyBuf = new byte[bs];
}
return ByteBuffer.wrap(copyBuf, 0, bs);
}
@ -632,8 +650,9 @@ public final class DfsPackFile extends BlockBasedFile {
readFully(pos, buf, 0, n, ctx);
crc1.update(buf, 0, n);
inf.setInput(buf, 0, n);
while (inf.inflate(tmp, 0, tmp.length) > 0)
while (inf.inflate(tmp, 0, tmp.length) > 0) {
continue;
}
pos += n;
cnt -= n;
}
@ -765,8 +784,9 @@ public final class DfsPackFile extends BlockBasedFile {
if (sz < ctx.getStreamFileThreshold()) {
data = decompress(pos + p, (int) sz, ctx);
if (data != null)
if (data != null) {
return new ObjectLoader.SmallObject(typeCode, data);
}
}
return new LargePackedWholeObject(typeCode, sz, pos, p, this, ctx.db);
}
@ -782,8 +802,9 @@ public final class DfsPackFile extends BlockBasedFile {
}
base = pos - base;
delta = new Delta(delta, pos, (int) sz, p, base);
if (sz != delta.deltaSize)
if (sz != delta.deltaSize) {
break SEARCH;
}
DeltaBaseCache.Entry e = ctx.getDeltaBaseCache().get(key, base);
if (e != null) {
@ -800,8 +821,9 @@ public final class DfsPackFile extends BlockBasedFile {
readFully(pos + p, ib, 0, 20, ctx);
long base = findDeltaBase(ctx, ObjectId.fromRaw(ib));
delta = new Delta(delta, pos, (int) sz, p + 20, base);
if (sz != delta.deltaSize)
if (sz != delta.deltaSize) {
break SEARCH;
}
DeltaBaseCache.Entry e = ctx.getDeltaBaseCache().get(key, base);
if (e != null) {
@ -829,10 +851,11 @@ public final class DfsPackFile extends BlockBasedFile {
assert(delta != null);
do {
// Cache only the base immediately before desired object.
if (cached)
if (cached) {
cached = false;
else if (delta.next == null)
} else if (delta.next == null) {
ctx.getDeltaBaseCache().put(key, delta.basePos, type, data);
}
pos = delta.deltaPos;
@ -843,8 +866,9 @@ public final class DfsPackFile extends BlockBasedFile {
}
final long sz = BinaryDelta.getResultSize(cmds);
if (Integer.MAX_VALUE <= sz)
if (Integer.MAX_VALUE <= sz) {
throw new LargeObjectException.ExceedsByteArrayLimit();
}
final byte[] result;
try {
@ -874,9 +898,10 @@ public final class DfsPackFile extends BlockBasedFile {
private long findDeltaBase(DfsReader ctx, ObjectId baseId)
throws IOException, MissingObjectException {
long ofs = idx(ctx).findOffset(baseId);
if (ofs < 0)
if (ofs < 0) {
throw new MissingObjectException(baseId,
JGitText.get().missingDeltaBase);
}
return ofs;
}
@ -933,8 +958,9 @@ public final class DfsPackFile extends BlockBasedFile {
case Constants.OBJ_OFS_DELTA: {
int p = 1;
while ((c & 0x80) != 0)
while ((c & 0x80) != 0) {
c = ib[p++] & 0xff;
}
c = ib[p++] & 0xff;
long ofs = c & 127;
while ((c & 128) != 0) {
@ -949,8 +975,9 @@ public final class DfsPackFile extends BlockBasedFile {
case Constants.OBJ_REF_DELTA: {
int p = 1;
while ((c & 0x80) != 0)
while ((c & 0x80) != 0) {
c = ib[p++] & 0xff;
}
readFully(pos + p, ib, 0, 20, ctx);
pos = findDeltaBase(ctx, ObjectId.fromRaw(ib));
continue;
@ -993,8 +1020,9 @@ public final class DfsPackFile extends BlockBasedFile {
case Constants.OBJ_OFS_DELTA:
c = ib[p++] & 0xff;
while ((c & 128) != 0)
while ((c & 128) != 0) {
c = ib[p++] & 0xff;
}
deltaAt = pos + p;
break;
@ -1027,8 +1055,9 @@ public final class DfsPackFile extends BlockBasedFile {
int c = ib[0] & 0xff;
int p = 1;
final int typeCode = (c >> 4) & 7;
while ((c & 0x80) != 0)
while ((c & 0x80) != 0) {
c = ib[p++] & 0xff;
}
long len = rev.findNextOffset(pos, length - 20) - pos;
switch (typeCode) {
@ -1072,8 +1101,9 @@ public final class DfsPackFile extends BlockBasedFile {
boolean isCorrupt(long offset) {
LongList list = corruptObjects;
if (list == null)
if (list == null) {
return false;
}
synchronized (list) {
return list.contains(offset);
}

Loading…
Cancel
Save