Merge changes Ideecc472,I2b12788a,I6cb9382d,I12cd3326,I200baa0b,I05626f2e,I65e45422

* changes:
  Increase PackOutputStream copy buffer to 64 KiB
  Tighten object header writing in PackOutuptStream
  Skip main thread test in ThreadSafeProgressMonitor
  Declare members of PackOutputStream final
  Always allocate the PackOutputStream copyBuffer
  Disable CRC32 computation when no PackIndex will be created
  Steal work from delta threads to rebalance CPU load
This commit is contained in:
Shawn Pearce 2013-04-10 20:56:13 -04:00 committed by Gerrit Code Review @ Eclipse.org
commit b5cbfa0146
9 changed files with 236 additions and 136 deletions

View File

@ -104,9 +104,11 @@ public void testMethodsOkOnMainThread() {
assertEquals(42, mock.value); assertEquals(42, mock.value);
pm.update(1); pm.update(1);
pm.pollForUpdates();
assertEquals(43, mock.value); assertEquals(43, mock.value);
pm.update(2); pm.update(2);
pm.pollForUpdates();
assertEquals(45, mock.value); assertEquals(45, mock.value);
pm.endTask(); pm.endTask();

View File

@ -43,6 +43,8 @@
package org.eclipse.jgit.internal.storage.pack; package org.eclipse.jgit.internal.storage.pack;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import org.eclipse.jgit.lib.ObjectReader; import org.eclipse.jgit.lib.ObjectReader;
@ -50,42 +52,92 @@
import org.eclipse.jgit.storage.pack.PackConfig; import org.eclipse.jgit.storage.pack.PackConfig;
final class DeltaTask implements Callable<Object> { final class DeltaTask implements Callable<Object> {
private final PackConfig config; static final class Block {
final List<DeltaTask> tasks;
final PackConfig config;
final ObjectReader templateReader;
final DeltaCache dc;
final ThreadSafeProgressMonitor pm;
final ObjectToPack[] list;
final int beginIndex;
final int endIndex;
private final ObjectReader templateReader; Block(int threads, PackConfig config, ObjectReader reader,
DeltaCache dc, ThreadSafeProgressMonitor pm,
private final DeltaCache dc; ObjectToPack[] list, int begin, int end) {
this.tasks = new ArrayList<DeltaTask>(threads);
private final ThreadSafeProgressMonitor pm;
private final int batchSize;
private final int start;
private final ObjectToPack[] list;
DeltaTask(PackConfig config, ObjectReader reader, DeltaCache dc,
ThreadSafeProgressMonitor pm, int batchSize, int start,
ObjectToPack[] list) {
this.config = config; this.config = config;
this.templateReader = reader; this.templateReader = reader;
this.dc = dc; this.dc = dc;
this.pm = pm; this.pm = pm;
this.batchSize = batchSize;
this.start = start;
this.list = list; this.list = list;
this.beginIndex = begin;
this.endIndex = end;
} }
public Object call() throws Exception { synchronized Slice stealWork() {
final ObjectReader or = templateReader.newReader(); for (int attempts = 0; attempts < 2; attempts++) {
try { DeltaTask maxTask = null;
DeltaWindow dw; int maxWork = 0;
dw = new DeltaWindow(config, dc, or); for (DeltaTask task : tasks) {
dw.search(pm, list, start, batchSize); int r = task.remaining();
} finally { if (maxWork < r) {
or.release(); maxTask = task;
pm.endWorker(); maxWork = r;
}
}
if (maxTask == null)
return null;
Slice s = maxTask.stealWork();
if (s != null)
return s;
} }
return null; return null;
} }
} }
static final class Slice {
final int beginIndex;
final int endIndex;
Slice(int b, int e) {
beginIndex = b;
endIndex = e;
}
}
private final Block block;
private final Slice firstSlice;
private volatile DeltaWindow dw;
DeltaTask(Block b, int beginIndex, int endIndex) {
this.block = b;
this.firstSlice = new Slice(beginIndex, endIndex);
}
public Object call() throws Exception {
ObjectReader or = block.templateReader.newReader();
try {
for (Slice s = firstSlice; s != null; s = block.stealWork()) {
dw = new DeltaWindow(block.config, block.dc, or, block.pm,
block.list, s.beginIndex, s.endIndex);
dw.search();
dw = null;
}
} finally {
or.release();
block.pm.endWorker();
}
return null;
}
int remaining() {
DeltaWindow d = dw;
return d != null ? d.remaining() : 0;
}
Slice stealWork() {
DeltaWindow d = dw;
return d != null ? d.stealWork() : null;
}
}

View File

@ -56,7 +56,7 @@
import org.eclipse.jgit.storage.pack.PackConfig; import org.eclipse.jgit.storage.pack.PackConfig;
import org.eclipse.jgit.util.TemporaryBuffer; import org.eclipse.jgit.util.TemporaryBuffer;
class DeltaWindow { final class DeltaWindow {
private static final int NEXT_RES = 0; private static final int NEXT_RES = 0;
private static final int NEXT_SRC = 1; private static final int NEXT_SRC = 1;
@ -67,6 +67,8 @@ class DeltaWindow {
private final ObjectReader reader; private final ObjectReader reader;
private final ProgressMonitor monitor;
private final DeltaWindowEntry[] window; private final DeltaWindowEntry[] window;
/** Maximum number of bytes to admit to the window at once. */ /** Maximum number of bytes to admit to the window at once. */
@ -75,6 +77,12 @@ class DeltaWindow {
/** Maximum depth we should create for any delta chain. */ /** Maximum depth we should create for any delta chain. */
private final int maxDepth; private final int maxDepth;
private final ObjectToPack[] toSearch;
private int cur;
private int end;
/** Amount of memory we have loaded right now. */ /** Amount of memory we have loaded right now. */
private long loaded; private long loaded;
@ -102,10 +110,16 @@ class DeltaWindow {
/** Used to compress cached deltas. */ /** Used to compress cached deltas. */
private Deflater deflater; private Deflater deflater;
DeltaWindow(PackConfig pc, DeltaCache dc, ObjectReader or) { DeltaWindow(PackConfig pc, DeltaCache dc, ObjectReader or,
ProgressMonitor pm,
ObjectToPack[] in, int beginIndex, int endIndex) {
config = pc; config = pc;
deltaCache = dc; deltaCache = dc;
reader = or; reader = or;
monitor = pm;
toSearch = in;
cur = beginIndex;
end = endIndex;
// C Git increases the window size supplied by the user by 1. // C Git increases the window size supplied by the user by 1.
// We don't know why it does this, but if the user asks for // We don't know why it does this, but if the user asks for
@ -126,21 +140,48 @@ class DeltaWindow {
maxDepth = config.getMaxDeltaDepth(); maxDepth = config.getMaxDeltaDepth();
} }
void search(ProgressMonitor monitor, ObjectToPack[] toSearch, int off, synchronized int remaining() {
int cnt) throws IOException { return end - cur;
}
synchronized DeltaTask.Slice stealWork() {
int e = end;
int n = (e - cur) >>> 1;
if (0 == n)
return null;
int t = e - n;
int h = toSearch[t].getPathHash();
while (cur < t) {
if (h == toSearch[t - 1].getPathHash())
t--;
else
break;
}
end = t;
return new DeltaTask.Slice(t, e);
}
void search() throws IOException {
try { try {
for (int end = off + cnt; off < end; off++) { for (;;) {
ObjectToPack next;
synchronized (this) {
if (end <= cur)
break;
next = toSearch[cur++];
}
res = window[resSlot]; res = window[resSlot];
if (0 < maxMemory) { if (0 < maxMemory) {
clear(res); clear(res);
int tail = next(resSlot); int tail = next(resSlot);
final long need = estimateSize(toSearch[off]); final long need = estimateSize(next);
while (maxMemory < loaded + need && tail != resSlot) { while (maxMemory < loaded + need && tail != resSlot) {
clear(window[tail]); clear(window[tail]);
tail = next(tail); tail = next(tail);
} }
} }
res.set(toSearch[off]); res.set(next);
if (res.object.isEdge() || res.object.doNotAttemptDelta()) { if (res.object.isEdge() || res.object.doNotAttemptDelta()) {
// We don't actually want to make a delta for // We don't actually want to make a delta for
@ -152,7 +193,7 @@ void search(ProgressMonitor monitor, ObjectToPack[] toSearch, int off,
// Search for a delta for the current window slot. // Search for a delta for the current window slot.
// //
monitor.update(1); monitor.update(1);
search(); searchInWindow();
} }
} }
} finally { } finally {
@ -181,7 +222,7 @@ else if (ent.buffer != null)
ent.set(null); ent.set(null);
} }
private void search() throws IOException { private void searchInWindow() throws IOException {
// TODO(spearce) If the object is used as a base for other // TODO(spearce) If the object is used as a base for other
// objects in this pack we should limit the depth we create // objects in this pack we should limit the depth we create
// for ourselves to be the remainder of our longest dependent // for ourselves to be the remainder of our longest dependent

View File

@ -44,10 +44,13 @@
package org.eclipse.jgit.internal.storage.pack; package org.eclipse.jgit.internal.storage.pack;
import static org.eclipse.jgit.lib.Constants.OBJ_OFS_DELTA;
import static org.eclipse.jgit.lib.Constants.OBJ_REF_DELTA;
import static org.eclipse.jgit.lib.Constants.PACK_SIGNATURE;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.io.OutputStream;
import java.security.MessageDigest; import java.security.MessageDigest;
import java.util.zip.CRC32;
import org.eclipse.jgit.internal.JGitText; import org.eclipse.jgit.internal.JGitText;
import org.eclipse.jgit.lib.Constants; import org.eclipse.jgit.lib.Constants;
@ -64,18 +67,18 @@ public final class PackOutputStream extends OutputStream {
private final PackWriter packWriter; private final PackWriter packWriter;
private final CRC32 crc = new CRC32();
private final MessageDigest md = Constants.newMessageDigest(); private final MessageDigest md = Constants.newMessageDigest();
private long count; private long count;
private byte[] headerBuffer = new byte[32]; private final byte[] headerBuffer = new byte[32];
private byte[] copyBuffer; private final byte[] copyBuffer = new byte[64 << 10];
private long checkCancelAt; private long checkCancelAt;
private boolean ofsDelta;
/** /**
* Initialize a pack output stream. * Initialize a pack output stream.
* <p> * <p>
@ -99,15 +102,14 @@ public PackOutputStream(final ProgressMonitor writeMonitor,
} }
@Override @Override
public void write(final int b) throws IOException { public final void write(final int b) throws IOException {
count++; count++;
out.write(b); out.write(b);
crc.update(b);
md.update((byte) b); md.update((byte) b);
} }
@Override @Override
public void write(final byte[] b, int off, int len) public final void write(final byte[] b, int off, int len)
throws IOException { throws IOException {
while (0 < len) { while (0 < len) {
final int n = Math.min(len, BYTES_TO_WRITE_BEFORE_CANCEL_CHECK); final int n = Math.min(len, BYTES_TO_WRITE_BEFORE_CANCEL_CHECK);
@ -122,7 +124,6 @@ public void write(final byte[] b, int off, int len)
} }
out.write(b, off, n); out.write(b, off, n);
crc.update(b, off, n);
md.update(b, off, n); md.update(b, off, n);
off += n; off += n;
@ -135,11 +136,13 @@ public void flush() throws IOException {
out.flush(); out.flush();
} }
void writeFileHeader(int version, long objectCount) throws IOException { final void writeFileHeader(int version, long objectCount)
System.arraycopy(Constants.PACK_SIGNATURE, 0, headerBuffer, 0, 4); throws IOException {
System.arraycopy(PACK_SIGNATURE, 0, headerBuffer, 0, 4);
NB.encodeInt32(headerBuffer, 4, version); NB.encodeInt32(headerBuffer, 4, version);
NB.encodeInt32(headerBuffer, 8, (int) objectCount); NB.encodeInt32(headerBuffer, 8, (int) objectCount);
write(headerBuffer, 0, 12); write(headerBuffer, 0, 12);
ofsDelta = packWriter.isDeltaBaseAsOffset();
} }
/** /**
@ -157,7 +160,7 @@ void writeFileHeader(int version, long objectCount) throws IOException {
* examine the type of exception and possibly its message to * examine the type of exception and possibly its message to
* distinguish between these cases. * distinguish between these cases.
*/ */
public void writeObject(ObjectToPack otp) throws IOException { public final void writeObject(ObjectToPack otp) throws IOException {
packWriter.writeObject(this, otp); packWriter.writeObject(this, otp);
} }
@ -177,52 +180,52 @@ public void writeObject(ObjectToPack otp) throws IOException {
* @throws IOException * @throws IOException
* the underlying stream refused to accept the header. * the underlying stream refused to accept the header.
*/ */
public void writeHeader(ObjectToPack otp, long rawLength) public final void writeHeader(ObjectToPack otp, long rawLength)
throws IOException { throws IOException {
if (otp.isDeltaRepresentation()) { ObjectToPack b = otp.getDeltaBase();
if (packWriter.isDeltaBaseAsOffset()) { if (b != null && (b.isWritten() & ofsDelta)) {
ObjectToPack baseInPack = otp.getDeltaBase(); int n = objectHeader(rawLength, OBJ_OFS_DELTA, headerBuffer);
if (baseInPack != null && baseInPack.isWritten()) { n = ofsDelta(count - b.getOffset(), headerBuffer, n);
final long start = count;
int n = encodeTypeSize(Constants.OBJ_OFS_DELTA, rawLength);
write(headerBuffer, 0, n); write(headerBuffer, 0, n);
} else if (otp.isDeltaRepresentation()) {
long offsetDiff = start - baseInPack.getOffset(); int n = objectHeader(rawLength, OBJ_REF_DELTA, headerBuffer);
n = headerBuffer.length - 1;
headerBuffer[n] = (byte) (offsetDiff & 0x7F);
while ((offsetDiff >>= 7) > 0)
headerBuffer[--n] = (byte) (0x80 | (--offsetDiff & 0x7F));
write(headerBuffer, n, headerBuffer.length - n);
return;
}
}
int n = encodeTypeSize(Constants.OBJ_REF_DELTA, rawLength);
otp.getDeltaBaseId().copyRawTo(headerBuffer, n); otp.getDeltaBaseId().copyRawTo(headerBuffer, n);
write(headerBuffer, 0, n + Constants.OBJECT_ID_LENGTH); write(headerBuffer, 0, n + 20);
} else { } else {
int n = encodeTypeSize(otp.getType(), rawLength); int n = objectHeader(rawLength, otp.getType(), headerBuffer);
write(headerBuffer, 0, n); write(headerBuffer, 0, n);
} }
} }
private int encodeTypeSize(int type, long rawLength) { private static final int objectHeader(long len, int type, byte[] buf) {
long nextLength = rawLength >>> 4; byte b = (byte) ((type << 4) | (len & 0x0F));
headerBuffer[0] = (byte) ((nextLength > 0 ? 0x80 : 0x00) | (type << 4) | (rawLength & 0x0F)); int n = 0;
rawLength = nextLength; for (len >>>= 4; len != 0; len >>>= 7) {
int n = 1; buf[n++] = (byte) (0x80 | b);
while (rawLength > 0) { b = (byte) (len & 0x7F);
nextLength >>>= 7;
headerBuffer[n++] = (byte) ((nextLength > 0 ? 0x80 : 0x00) | (rawLength & 0x7F));
rawLength = nextLength;
} }
buf[n++] = b;
return n;
}
private static final int ofsDelta(long diff, byte[] buf, int p) {
p += ofsDeltaVarIntLength(diff);
int n = p;
buf[--n] = (byte) (diff & 0x7F);
while ((diff >>>= 7) != 0)
buf[--n] = (byte) (0x80 | (--diff & 0x7F));
return p;
}
private static final int ofsDeltaVarIntLength(long v) {
int n = 1;
for (; (v >>>= 7) != 0; n++)
--v;
return n; return n;
} }
/** @return a temporary buffer writers can use to copy data with. */ /** @return a temporary buffer writers can use to copy data with. */
public byte[] getCopyBuffer() { public final byte[] getCopyBuffer() {
if (copyBuffer == null)
copyBuffer = new byte[16 * 1024];
return copyBuffer; return copyBuffer;
} }
@ -231,22 +234,12 @@ void endObject() {
} }
/** @return total number of bytes written since stream start. */ /** @return total number of bytes written since stream start. */
public long length() { public final long length() {
return count; return count;
} }
/** @return obtain the current CRC32 register. */
int getCRC32() {
return (int) crc.getValue();
}
/** Reinitialize the CRC32 register for a new region. */
void resetCRC32() {
crc.reset();
}
/** @return obtain the current SHA-1 digest. */ /** @return obtain the current SHA-1 digest. */
byte[] getDigest() { final byte[] getDigest() {
return md.digest(); return md.digest();
} }
} }

View File

@ -75,6 +75,8 @@
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.zip.CRC32;
import java.util.zip.CheckedOutputStream;
import java.util.zip.Deflater; import java.util.zip.Deflater;
import java.util.zip.DeflaterOutputStream; import java.util.zip.DeflaterOutputStream;
@ -275,12 +277,16 @@ public static Iterable<PackWriter> getInstances() {
private boolean canBuildBitmaps; private boolean canBuildBitmaps;
private boolean indexDisabled;
private int depth; private int depth;
private Collection<? extends ObjectId> unshallowObjects; private Collection<? extends ObjectId> unshallowObjects;
private PackBitmapIndexBuilder writeBitmaps; private PackBitmapIndexBuilder writeBitmaps;
private CRC32 crc32;
/** /**
* Create writer for specified repository. * Create writer for specified repository.
* <p> * <p>
@ -471,6 +477,19 @@ public void setUseBitmaps(boolean useBitmaps) {
this.useBitmaps = useBitmaps; this.useBitmaps = useBitmaps;
} }
/** @return true if the index file cannot be created by this PackWriter. */
public boolean isIndexDisabled() {
return indexDisabled || !cachedPacks.isEmpty();
}
/**
* @param noIndex
* true to disable creation of the index file.
*/
public void setIndexDisabled(boolean noIndex) {
this.indexDisabled = noIndex;
}
/** /**
* @return true to ignore objects that are uninteresting and also not found * @return true to ignore objects that are uninteresting and also not found
* on local disk; false to throw a {@link MissingObjectException} * on local disk; false to throw a {@link MissingObjectException}
@ -855,7 +874,7 @@ public int getIndexVersion() {
* the index data could not be written to the supplied stream. * the index data could not be written to the supplied stream.
*/ */
public void writeIndex(final OutputStream indexStream) throws IOException { public void writeIndex(final OutputStream indexStream) throws IOException {
if (!cachedPacks.isEmpty()) if (isIndexDisabled())
throw new IOException(JGitText.get().cachedPacksPreventsIndexCreation); throw new IOException(JGitText.get().cachedPacksPreventsIndexCreation);
long writeStart = System.currentTimeMillis(); long writeStart = System.currentTimeMillis();
@ -996,8 +1015,13 @@ public void writePack(ProgressMonitor compressMonitor,
if (config.isDeltaCompress()) if (config.isDeltaCompress())
searchForDeltas(compressMonitor); searchForDeltas(compressMonitor);
final PackOutputStream out = new PackOutputStream(writeMonitor, crc32 = new CRC32();
packStream, this); final PackOutputStream out = new PackOutputStream(
writeMonitor,
isIndexDisabled()
? packStream
: new CheckedOutputStream(packStream, crc32),
this);
long objCnt = getObjectCount(); long objCnt = getObjectCount();
stats.totalObjects = objCnt; stats.totalObjects = objCnt;
@ -1305,68 +1329,57 @@ private void searchForDeltas(final ProgressMonitor monitor,
threads = Runtime.getRuntime().availableProcessors(); threads = Runtime.getRuntime().availableProcessors();
if (threads <= 1 || cnt <= 2 * config.getDeltaSearchWindowSize()) { if (threads <= 1 || cnt <= 2 * config.getDeltaSearchWindowSize()) {
DeltaCache dc = new DeltaCache(config); new DeltaWindow(config, new DeltaCache(config), reader, monitor,
DeltaWindow dw = new DeltaWindow(config, dc, reader); list, 0, cnt).search();
dw.search(monitor, list, 0, cnt);
return; return;
} }
final DeltaCache dc = new ThreadSafeDeltaCache(config); final DeltaCache dc = new ThreadSafeDeltaCache(config);
final ThreadSafeProgressMonitor pm = new ThreadSafeProgressMonitor(monitor); final ThreadSafeProgressMonitor pm = new ThreadSafeProgressMonitor(monitor);
// Guess at the size of batch we want. Because we don't really int estSize = cnt / threads;
// have a way for a thread to steal work from another thread if if (estSize < config.getDeltaSearchWindowSize())
// it ends early, we over partition slightly so the work units estSize = config.getDeltaSearchWindowSize();
// are a bit smaller.
//
int estSize = cnt / (threads * 2);
if (estSize < 2 * config.getDeltaSearchWindowSize())
estSize = 2 * config.getDeltaSearchWindowSize();
final List<DeltaTask> myTasks = new ArrayList<DeltaTask>(threads * 2); DeltaTask.Block taskBlock = new DeltaTask.Block(threads, config,
reader, dc, pm,
list, 0, cnt);
for (int i = 0; i < cnt;) { for (int i = 0; i < cnt;) {
final int start = i; final int start = i;
final int batchSize; int end;
if (cnt - i < estSize) { if (cnt - i < estSize) {
// If we don't have enough to fill the remaining block, // If we don't have enough to fill the remaining block,
// schedule what is left over as a single block. // schedule what is left over as a single block.
// end = cnt;
batchSize = cnt - i;
} else { } else {
// Try to split the block at the end of a path. // Try to split the block at the end of a path.
// end = start + estSize;
int end = start + estSize; int h = list[end - 1].getPathHash();
while (end < cnt) { while (end < cnt) {
ObjectToPack a = list[end - 1]; if (h == list[end].getPathHash())
ObjectToPack b = list[end];
if (a.getPathHash() == b.getPathHash())
end++; end++;
else else
break; break;
} }
batchSize = end - start;
} }
i += batchSize; i = end;
myTasks.add(new DeltaTask(config, reader, dc, pm, batchSize, start, list)); taskBlock.tasks.add(new DeltaTask(taskBlock, start, end));
} }
pm.startWorkers(myTasks.size()); pm.startWorkers(taskBlock.tasks.size());
final Executor executor = config.getExecutor(); final Executor executor = config.getExecutor();
final List<Throwable> errors = Collections final List<Throwable> errors = Collections
.synchronizedList(new ArrayList<Throwable>()); .synchronizedList(new ArrayList<Throwable>());
if (executor instanceof ExecutorService) { if (executor instanceof ExecutorService) {
// Caller supplied us a service, use it directly. // Caller supplied us a service, use it directly.
// runTasks((ExecutorService) executor, pm, taskBlock, errors);
runTasks((ExecutorService) executor, pm, myTasks, errors);
} else if (executor == null) { } else if (executor == null) {
// Caller didn't give us a way to run the tasks, spawn up a // Caller didn't give us a way to run the tasks, spawn up a
// temporary thread pool and make sure it tears down cleanly. // temporary thread pool and make sure it tears down cleanly.
//
ExecutorService pool = Executors.newFixedThreadPool(threads); ExecutorService pool = Executors.newFixedThreadPool(threads);
try { try {
runTasks(pool, pm, myTasks, errors); runTasks(pool, pm, taskBlock, errors);
} finally { } finally {
pool.shutdown(); pool.shutdown();
for (;;) { for (;;) {
@ -1383,8 +1396,7 @@ private void searchForDeltas(final ProgressMonitor monitor,
// The caller gave us an executor, but it might not do // The caller gave us an executor, but it might not do
// asynchronous execution. Wrap everything and hope it // asynchronous execution. Wrap everything and hope it
// can schedule these for us. // can schedule these for us.
// for (final DeltaTask task : taskBlock.tasks) {
for (final DeltaTask task : myTasks) {
executor.execute(new Runnable() { executor.execute(new Runnable() {
public void run() { public void run() {
try { try {
@ -1426,9 +1438,9 @@ public void run() {
private static void runTasks(ExecutorService pool, private static void runTasks(ExecutorService pool,
ThreadSafeProgressMonitor pm, ThreadSafeProgressMonitor pm,
List<DeltaTask> tasks, List<Throwable> errors) throws IOException { DeltaTask.Block tb, List<Throwable> errors) throws IOException {
List<Future<?>> futures = new ArrayList<Future<?>>(tasks.size()); List<Future<?>> futures = new ArrayList<Future<?>>(tb.tasks.size());
for (DeltaTask task : tasks) for (DeltaTask task : tb.tasks)
futures.add(pool.submit(task)); futures.add(pool.submit(task));
try { try {
@ -1496,12 +1508,12 @@ private void writeObjectImpl(PackOutputStream out, ObjectToPack otp)
if (otp.isWritten()) if (otp.isWritten())
return; // Delta chain cycle caused this to write already. return; // Delta chain cycle caused this to write already.
out.resetCRC32(); crc32.reset();
otp.setOffset(out.length()); otp.setOffset(out.length());
try { try {
reuseSupport.copyObjectAsIs(out, otp, reuseValidate); reuseSupport.copyObjectAsIs(out, otp, reuseValidate);
out.endObject(); out.endObject();
otp.setCRC(out.getCRC32()); otp.setCRC((int) crc32.getValue());
typeStats.reusedObjects++; typeStats.reusedObjects++;
if (otp.isDeltaRepresentation()) { if (otp.isDeltaRepresentation()) {
typeStats.reusedDeltas++; typeStats.reusedDeltas++;
@ -1535,7 +1547,7 @@ private void writeObjectImpl(PackOutputStream out, ObjectToPack otp)
else else
writeWholeObjectDeflate(out, otp); writeWholeObjectDeflate(out, otp);
out.endObject(); out.endObject();
otp.setCRC(out.getCRC32()); otp.setCRC((int) crc32.getValue());
} }
private void writeBase(PackOutputStream out, ObjectToPack base) private void writeBase(PackOutputStream out, ObjectToPack base)
@ -1549,7 +1561,7 @@ private void writeWholeObjectDeflate(PackOutputStream out,
final Deflater deflater = deflater(); final Deflater deflater = deflater();
final ObjectLoader ldr = reader.open(otp, otp.getType()); final ObjectLoader ldr = reader.open(otp, otp.getType());
out.resetCRC32(); crc32.reset();
otp.setOffset(out.length()); otp.setOffset(out.length());
out.writeHeader(otp, ldr.getSize()); out.writeHeader(otp, ldr.getSize());
@ -1563,7 +1575,7 @@ private void writeDeltaObjectDeflate(PackOutputStream out,
final ObjectToPack otp) throws IOException { final ObjectToPack otp) throws IOException {
writeBase(out, otp.getDeltaBase()); writeBase(out, otp.getDeltaBase());
out.resetCRC32(); crc32.reset();
otp.setOffset(out.length()); otp.setOffset(out.length());
DeltaCache.Ref ref = otp.popCachedDelta(); DeltaCache.Ref ref = otp.popCachedDelta();

View File

@ -157,10 +157,7 @@ private void doUpdates() {
} }
public void update(int completed) { public void update(int completed) {
int old = pendingUpdates.getAndAdd(completed); if (0 == pendingUpdates.getAndAdd(completed))
if (isMainThread())
doUpdates();
else if (old == 0)
process.release(); process.release();
} }

View File

@ -291,6 +291,7 @@ private void writePack(final Map<String, RemoteRefUpdate> refUpdates,
newObjects.add(r.getNewObjectId()); newObjects.add(r.getNewObjectId());
} }
writer.setIndexDisabled(true);
writer.setUseCachedPacks(true); writer.setUseCachedPacks(true);
writer.setUseBitmaps(true); writer.setUseBitmaps(true);
writer.setThin(thinPack); writer.setThin(thinPack);

View File

@ -200,6 +200,7 @@ public void writeBundle(ProgressMonitor monitor, OutputStream os)
inc.addAll(include.values()); inc.addAll(include.values());
for (final RevCommit r : assume) for (final RevCommit r : assume)
exc.add(r.getId()); exc.add(r.getId());
packWriter.setIndexDisabled(true);
packWriter.setDeltaBaseAsOffset(true); packWriter.setDeltaBaseAsOffset(true);
packWriter.setThin(exc.size() > 0); packWriter.setThin(exc.size() > 0);
packWriter.setReuseValidatingObjects(false); packWriter.setReuseValidatingObjects(false);

View File

@ -1116,6 +1116,7 @@ private void sendPack(final boolean sideband) throws IOException {
cfg = new PackConfig(db); cfg = new PackConfig(db);
final PackWriter pw = new PackWriter(cfg, walk.getObjectReader()); final PackWriter pw = new PackWriter(cfg, walk.getObjectReader());
try { try {
pw.setIndexDisabled(true);
pw.setUseCachedPacks(true); pw.setUseCachedPacks(true);
pw.setUseBitmaps(true); pw.setUseBitmaps(true);
pw.setReuseDeltaCommits(true); pw.setReuseDeltaCommits(true);