mirror of https://github.com/apache/lucene.git
LUCENE-7792: add try/finally to make sure semaphore is released on exceptions
This commit is contained in:
parent
f45017b2d4
commit
25f1dd2f9b
|
@ -273,6 +273,9 @@ public class OfflineSorter {
|
||||||
while (true) {
|
while (true) {
|
||||||
Partition part = readPartition(is);
|
Partition part = readPartition(is);
|
||||||
if (part.count == 0) {
|
if (part.count == 0) {
|
||||||
|
if (partitionsInRAM != null) {
|
||||||
|
partitionsInRAM.release();
|
||||||
|
}
|
||||||
assert part.exhausted;
|
assert part.exhausted;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -317,16 +320,7 @@ public class OfflineSorter {
|
||||||
result = out.getName();
|
result = out.getName();
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
try {
|
result = getPartition(segments.get(0)).fileName;
|
||||||
result = segments.get(0).get().fileName;
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
throw new ThreadInterruptedException(ie);
|
|
||||||
} catch (ExecutionException ee) {
|
|
||||||
IOUtils.reThrow(ee.getCause());
|
|
||||||
|
|
||||||
// dead code but javac disagrees:
|
|
||||||
result = null;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// We should be explicitly removing all intermediate files ourselves unless there is an exception:
|
// We should be explicitly removing all intermediate files ourselves unless there is an exception:
|
||||||
|
@ -406,52 +400,59 @@ public class OfflineSorter {
|
||||||
if (partitionsInRAM != null) {
|
if (partitionsInRAM != null) {
|
||||||
partitionsInRAM.acquire();
|
partitionsInRAM.acquire();
|
||||||
}
|
}
|
||||||
long start = System.currentTimeMillis();
|
boolean success = false;
|
||||||
SortableBytesRefArray buffer;
|
try {
|
||||||
boolean exhausted = false;
|
long start = System.currentTimeMillis();
|
||||||
int count;
|
SortableBytesRefArray buffer;
|
||||||
if (valueLength != -1) {
|
boolean exhausted = false;
|
||||||
// fixed length case
|
int count;
|
||||||
buffer = new FixedLengthBytesRefArray(valueLength);
|
if (valueLength != -1) {
|
||||||
int limit = ramBufferSize.bytes / valueLength;
|
// fixed length case
|
||||||
for(int i=0;i<limit;i++) {
|
buffer = new FixedLengthBytesRefArray(valueLength);
|
||||||
BytesRef item = null;
|
int limit = ramBufferSize.bytes / valueLength;
|
||||||
try {
|
for(int i=0;i<limit;i++) {
|
||||||
item = reader.next();
|
BytesRef item = null;
|
||||||
} catch (Throwable t) {
|
try {
|
||||||
verifyChecksum(t, reader);
|
item = reader.next();
|
||||||
|
} catch (Throwable t) {
|
||||||
|
verifyChecksum(t, reader);
|
||||||
|
}
|
||||||
|
if (item == null) {
|
||||||
|
exhausted = true;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
buffer.append(item);
|
||||||
}
|
}
|
||||||
if (item == null) {
|
} else {
|
||||||
exhausted = true;
|
Counter bufferBytesUsed = Counter.newCounter();
|
||||||
break;
|
buffer = new BytesRefArray(bufferBytesUsed);
|
||||||
|
while (true) {
|
||||||
|
BytesRef item = null;
|
||||||
|
try {
|
||||||
|
item = reader.next();
|
||||||
|
} catch (Throwable t) {
|
||||||
|
verifyChecksum(t, reader);
|
||||||
|
}
|
||||||
|
if (item == null) {
|
||||||
|
exhausted = true;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
buffer.append(item);
|
||||||
|
// Account for the created objects.
|
||||||
|
// (buffer slots do not account to buffer size.)
|
||||||
|
if (bufferBytesUsed.get() > ramBufferSize.bytes) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
buffer.append(item);
|
|
||||||
}
|
}
|
||||||
} else {
|
sortInfo.readTimeMS += System.currentTimeMillis() - start;
|
||||||
Counter bufferBytesUsed = Counter.newCounter();
|
success = true;
|
||||||
buffer = new BytesRefArray(bufferBytesUsed);
|
return new Partition(buffer, exhausted);
|
||||||
while (true) {
|
} finally {
|
||||||
BytesRef item = null;
|
if (success == false && partitionsInRAM != null) {
|
||||||
try {
|
partitionsInRAM.release();
|
||||||
item = reader.next();
|
|
||||||
} catch (Throwable t) {
|
|
||||||
verifyChecksum(t, reader);
|
|
||||||
}
|
|
||||||
if (item == null) {
|
|
||||||
exhausted = true;
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
buffer.append(item);
|
|
||||||
// Account for the created objects.
|
|
||||||
// (buffer slots do not account to buffer size.)
|
|
||||||
if (bufferBytesUsed.get() > ramBufferSize.bytes) {
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
sortInfo.readTimeMS += System.currentTimeMillis() - start;
|
|
||||||
|
|
||||||
return new Partition(buffer, exhausted);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
static class FileAndTop {
|
static class FileAndTop {
|
||||||
|
@ -616,15 +617,28 @@ public class OfflineSorter {
|
||||||
|
|
||||||
CodecUtil.writeFooter(out.out);
|
CodecUtil.writeFooter(out.out);
|
||||||
part.buffer.clear();
|
part.buffer.clear();
|
||||||
|
|
||||||
|
return new Partition(tempFile.getName(), part.count);
|
||||||
|
} finally {
|
||||||
if (partitionsInRAM != null) {
|
if (partitionsInRAM != null) {
|
||||||
partitionsInRAM.release();
|
partitionsInRAM.release();
|
||||||
}
|
}
|
||||||
|
|
||||||
return new Partition(tempFile.getName(), part.count);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Partition getPartition(Future<Partition> future) throws IOException {
|
||||||
|
try {
|
||||||
|
return future.get();
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
throw new ThreadInterruptedException(ie);
|
||||||
|
} catch (ExecutionException ee) {
|
||||||
|
IOUtils.reThrow(ee.getCause());
|
||||||
|
// oh so soon to go away:
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/** Merges multiple file-based partitions to a single on-disk partition. */
|
/** Merges multiple file-based partitions to a single on-disk partition. */
|
||||||
private class MergePartitionsTask implements Callable<Partition> {
|
private class MergePartitionsTask implements Callable<Partition> {
|
||||||
private final Directory dir;
|
private final Directory dir;
|
||||||
|
@ -636,10 +650,10 @@ public class OfflineSorter {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Partition call() throws IOException, InterruptedException, ExecutionException {
|
public Partition call() throws IOException {
|
||||||
long totalCount = 0;
|
long totalCount = 0;
|
||||||
for (Future<Partition> segment : segmentsToMerge) {
|
for (Future<Partition> segment : segmentsToMerge) {
|
||||||
totalCount += segment.get().count;
|
totalCount += getPartition(segment).count;
|
||||||
}
|
}
|
||||||
|
|
||||||
PriorityQueue<FileAndTop> queue = new PriorityQueue<FileAndTop>(segmentsToMerge.size()) {
|
PriorityQueue<FileAndTop> queue = new PriorityQueue<FileAndTop>(segmentsToMerge.size()) {
|
||||||
|
@ -660,7 +674,7 @@ public class OfflineSorter {
|
||||||
|
|
||||||
// Open streams and read the top for each file
|
// Open streams and read the top for each file
|
||||||
for (int i = 0; i < segmentsToMerge.size(); i++) {
|
for (int i = 0; i < segmentsToMerge.size(); i++) {
|
||||||
Partition segment = segmentsToMerge.get(i).get();
|
Partition segment = getPartition(segmentsToMerge.get(i));
|
||||||
streams[i] = getReader(dir.openChecksumInput(segment.fileName, IOContext.READONCE), segment.fileName);
|
streams[i] = getReader(dir.openChecksumInput(segment.fileName, IOContext.READONCE), segment.fileName);
|
||||||
|
|
||||||
BytesRef item = null;
|
BytesRef item = null;
|
||||||
|
@ -705,7 +719,7 @@ public class OfflineSorter {
|
||||||
}
|
}
|
||||||
List<String> toDelete = new ArrayList<>();
|
List<String> toDelete = new ArrayList<>();
|
||||||
for (Future<Partition> segment : segmentsToMerge) {
|
for (Future<Partition> segment : segmentsToMerge) {
|
||||||
toDelete.add(segment.get().fileName);
|
toDelete.add(getPartition(segment).fileName);
|
||||||
}
|
}
|
||||||
IOUtils.deleteFiles(dir, toDelete);
|
IOUtils.deleteFiles(dir, toDelete);
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue