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);
pm.update(1);
pm.pollForUpdates();
assertEquals(43, mock.value);
pm.update(2);
pm.pollForUpdates();
assertEquals(45, mock.value);
pm.endTask();

View File

@ -43,6 +43,8 @@
package org.eclipse.jgit.internal.storage.pack;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Callable;
import org.eclipse.jgit.lib.ObjectReader;
@ -50,42 +52,92 @@
import org.eclipse.jgit.storage.pack.PackConfig;
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,
ObjectToPack[] list, int begin, int end) {
this.tasks = new ArrayList<DeltaTask>(threads);
this.config = config;
this.templateReader = reader;
this.dc = dc;
this.pm = pm;
this.list = list;
this.beginIndex = begin;
this.endIndex = end;
}
private final DeltaCache dc;
synchronized Slice stealWork() {
for (int attempts = 0; attempts < 2; attempts++) {
DeltaTask maxTask = null;
int maxWork = 0;
for (DeltaTask task : tasks) {
int r = task.remaining();
if (maxWork < r) {
maxTask = task;
maxWork = r;
}
}
if (maxTask == null)
return null;
Slice s = maxTask.stealWork();
if (s != null)
return s;
}
return null;
}
}
private final ThreadSafeProgressMonitor pm;
static final class Slice {
final int beginIndex;
final int endIndex;
private final int batchSize;
Slice(int b, int e) {
beginIndex = b;
endIndex = e;
}
}
private final int start;
private final Block block;
private final Slice firstSlice;
private volatile DeltaWindow dw;
private final ObjectToPack[] list;
DeltaTask(PackConfig config, ObjectReader reader, DeltaCache dc,
ThreadSafeProgressMonitor pm, int batchSize, int start,
ObjectToPack[] list) {
this.config = config;
this.templateReader = reader;
this.dc = dc;
this.pm = pm;
this.batchSize = batchSize;
this.start = start;
this.list = list;
DeltaTask(Block b, int beginIndex, int endIndex) {
this.block = b;
this.firstSlice = new Slice(beginIndex, endIndex);
}
public Object call() throws Exception {
final ObjectReader or = templateReader.newReader();
ObjectReader or = block.templateReader.newReader();
try {
DeltaWindow dw;
dw = new DeltaWindow(config, dc, or);
dw.search(pm, list, start, batchSize);
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();
pm.endWorker();
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.util.TemporaryBuffer;
class DeltaWindow {
final class DeltaWindow {
private static final int NEXT_RES = 0;
private static final int NEXT_SRC = 1;
@ -67,6 +67,8 @@ class DeltaWindow {
private final ObjectReader reader;
private final ProgressMonitor monitor;
private final DeltaWindowEntry[] window;
/** 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. */
private final int maxDepth;
private final ObjectToPack[] toSearch;
private int cur;
private int end;
/** Amount of memory we have loaded right now. */
private long loaded;
@ -102,10 +110,16 @@ class DeltaWindow {
/** Used to compress cached deltas. */
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;
deltaCache = dc;
reader = or;
monitor = pm;
toSearch = in;
cur = beginIndex;
end = endIndex;
// 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
@ -126,21 +140,48 @@ class DeltaWindow {
maxDepth = config.getMaxDeltaDepth();
}
void search(ProgressMonitor monitor, ObjectToPack[] toSearch, int off,
int cnt) throws IOException {
synchronized int remaining() {
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 {
for (int end = off + cnt; off < end; off++) {
for (;;) {
ObjectToPack next;
synchronized (this) {
if (end <= cur)
break;
next = toSearch[cur++];
}
res = window[resSlot];
if (0 < maxMemory) {
clear(res);
int tail = next(resSlot);
final long need = estimateSize(toSearch[off]);
final long need = estimateSize(next);
while (maxMemory < loaded + need && tail != resSlot) {
clear(window[tail]);
tail = next(tail);
}
}
res.set(toSearch[off]);
res.set(next);
if (res.object.isEdge() || res.object.doNotAttemptDelta()) {
// 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.
//
monitor.update(1);
search();
searchInWindow();
}
}
} finally {
@ -181,7 +222,7 @@ else if (ent.buffer != 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
// objects in this pack we should limit the depth we create
// for ourselves to be the remainder of our longest dependent

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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