You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2020/09/18 16:05:34 UTC
[lucene-solr] branch branch_8x updated: LUCENE-9534: Ensure
DWPT#ramBytesUsed is only called unter lock (#1889)
This is an automated email from the ASF dual-hosted git repository.
simonw pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git
The following commit(s) were added to refs/heads/branch_8x by this push:
new 3577ff4 LUCENE-9534: Ensure DWPT#ramBytesUsed is only called unter lock (#1889)
3577ff4 is described below
commit 3577ff4a6d298483f7dbc967c5e004eb72842270
Author: Simon Willnauer <si...@apache.org>
AuthorDate: Fri Sep 18 17:59:05 2020 +0200
LUCENE-9534: Ensure DWPT#ramBytesUsed is only called unter lock (#1889)
Consumers of the used RAM of a DWPT should use it's committed bytesUsed
value that's threadsafe.
---
.../java/org/apache/lucene/index/DocumentsWriterFlushControl.java | 2 +-
.../src/java/org/apache/lucene/index/DocumentsWriterPerThread.java | 6 ++++--
2 files changed, 5 insertions(+), 3 deletions(-)
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
index 9e3be43..3262f50 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
@@ -669,7 +669,7 @@ final class DocumentsWriterFlushControl implements Accountable, Closeable {
int count = 0;
for (DocumentsWriterPerThread next : perThreadPool) {
if (next.isFlushPending() == false && next.getNumDocsInRAM() > 0) {
- final long nextRam = next.ramBytesUsed();
+ final long nextRam = next.getLastCommittedBytesUsed();
if (infoStream.isEnabled("FP")) {
infoStream.message("FP", "thread state has " + nextRam + " bytes; docInRAM=" + next.getNumDocsInRAM());
}
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
index 6790819..8b919a7 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
@@ -311,8 +311,8 @@ final class DocumentsWriterPerThread implements Accountable {
assert deleteSlice.isEmpty() : "all deletes must be applied in prepareFlush";
segmentInfo.setMaxDoc(numDocsInRAM);
final SegmentWriteState flushState = new SegmentWriteState(infoStream, directory, segmentInfo, fieldInfos.finish(),
- pendingUpdates, new IOContext(new FlushInfo(numDocsInRAM, ramBytesUsed())));
- final double startMBUsed = ramBytesUsed() / 1024. / 1024.;
+ pendingUpdates, new IOContext(new FlushInfo(numDocsInRAM, lastCommittedBytesUsed)));
+ final double startMBUsed = lastCommittedBytesUsed / 1024. / 1024.;
// Apply delete-by-docID now (delete-byDocID only
// happens when an exception is hit processing that
@@ -518,11 +518,13 @@ final class DocumentsWriterPerThread implements Accountable {
@Override
public long ramBytesUsed() {
+ assert lock.isHeldByCurrentThread();
return (deleteDocIDs.length * Integer.BYTES)+ pendingUpdates.ramBytesUsed() + consumer.ramBytesUsed();
}
@Override
public Collection<Accountable> getChildResources() {
+ assert lock.isHeldByCurrentThread();
return Collections.unmodifiableList(Arrays.asList(pendingUpdates, consumer));
}