Skip to content

Commit 535dbbe

Browse files
authored
Add coordinating object to track bytes (#122460)
This commit adds a new coordinating object to reflect reduce the need to introduce a new releasable for every network byte reference. Additionally, it will allow us to more specifically track bytes while they are buffered.
1 parent a0f3b24 commit 535dbbe

File tree

5 files changed

+226
-111
lines changed

5 files changed

+226
-111
lines changed

server/src/main/java/org/elasticsearch/action/bulk/IncrementalBulkService.java

Lines changed: 23 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -91,19 +91,18 @@ public static class Handler implements Releasable {
9191
public static final BulkRequest.IncrementalState EMPTY_STATE = new BulkRequest.IncrementalState(Collections.emptyMap(), true);
9292

9393
private final Client client;
94-
private final IndexingPressure indexingPressure;
9594
private final ActiveShardCount waitForActiveShards;
9695
private final TimeValue timeout;
9796
private final String refresh;
9897

9998
private final ArrayList<Releasable> releasables = new ArrayList<>(4);
10099
private final ArrayList<BulkResponse> responses = new ArrayList<>(2);
100+
private final IndexingPressure.Incremental incrementalOperation;
101101
private boolean closed = false;
102102
private boolean globalFailure = false;
103103
private boolean incrementalRequestSubmitted = false;
104104
private boolean bulkInProgress = false;
105105
private Exception bulkActionLevelFailure = null;
106-
private long currentBulkSize = 0L;
107106
private BulkRequest bulkRequest = null;
108107

109108
protected Handler(
@@ -114,13 +113,17 @@ protected Handler(
114113
@Nullable String refresh
115114
) {
116115
this.client = client;
117-
this.indexingPressure = indexingPressure;
118116
this.waitForActiveShards = waitForActiveShards != null ? ActiveShardCount.parseString(waitForActiveShards) : null;
119117
this.timeout = timeout;
120118
this.refresh = refresh;
119+
this.incrementalOperation = indexingPressure.startIncrementalCoordinating(0, 0, false);
121120
createNewBulkRequest(EMPTY_STATE);
122121
}
123122

123+
public IndexingPressure.Incremental getIncrementalOperation() {
124+
return incrementalOperation;
125+
}
126+
124127
public void addItems(List<DocWriteRequest<?>> items, Releasable releasable, Runnable nextItems) {
125128
assert closed == false;
126129
assert bulkInProgress == false;
@@ -130,7 +133,8 @@ public void addItems(List<DocWriteRequest<?>> items, Releasable releasable, Runn
130133
} else {
131134
assert bulkRequest != null;
132135
if (internalAddItems(items, releasable)) {
133-
if (shouldBackOff()) {
136+
if (incrementalOperation.shouldSplit()) {
137+
IndexingPressure.Coordinating coordinating = incrementalOperation.split();
134138
final boolean isFirstRequest = incrementalRequestSubmitted == false;
135139
incrementalRequestSubmitted = true;
136140
final ArrayList<Releasable> toRelease = new ArrayList<>(releasables);
@@ -152,6 +156,7 @@ public void onFailure(Exception e) {
152156
}
153157
}, () -> {
154158
bulkInProgress = false;
159+
coordinating.close();
155160
toRelease.forEach(Releasable::close);
156161
nextItems.run();
157162
}));
@@ -164,10 +169,6 @@ public void onFailure(Exception e) {
164169
}
165170
}
166171

167-
private boolean shouldBackOff() {
168-
return indexingPressure.shouldSplitBulk(currentBulkSize);
169-
}
170-
171172
public void lastItems(List<DocWriteRequest<?>> items, Releasable releasable, ActionListener<BulkResponse> listener) {
172173
assert bulkInProgress == false;
173174
if (bulkActionLevelFailure != null) {
@@ -176,6 +177,7 @@ public void lastItems(List<DocWriteRequest<?>> items, Releasable releasable, Act
176177
} else {
177178
assert bulkRequest != null;
178179
if (internalAddItems(items, releasable)) {
180+
IndexingPressure.Coordinating coordinating = incrementalOperation.split();
179181
final ArrayList<Releasable> toRelease = new ArrayList<>(releasables);
180182
releasables.clear();
181183
// We do not need to set this back to false as this will be the last request.
@@ -195,7 +197,10 @@ public void onFailure(Exception e) {
195197
handleBulkFailure(isFirstRequest, e);
196198
errorResponse(listener);
197199
}
198-
}, () -> toRelease.forEach(Releasable::close)));
200+
}, () -> {
201+
coordinating.close();
202+
toRelease.forEach(Releasable::close);
203+
}));
199204
} else {
200205
errorResponse(listener);
201206
}
@@ -204,13 +209,17 @@ public void onFailure(Exception e) {
204209

205210
@Override
206211
public void close() {
207-
closed = true;
208-
releasables.forEach(Releasable::close);
209-
releasables.clear();
212+
if (closed == false) {
213+
closed = true;
214+
incrementalOperation.close();
215+
releasables.forEach(Releasable::close);
216+
releasables.clear();
217+
}
210218
}
211219

212220
private void shortCircuitDueToTopLevelFailure(List<DocWriteRequest<?>> items, Releasable releasable) {
213221
assert releasables.isEmpty();
222+
assert incrementalOperation.currentOperationsSize() == 0;
214223
assert bulkRequest == null;
215224
if (globalFailure == false) {
216225
addItemLevelFailures(items);
@@ -228,7 +237,6 @@ private void errorResponse(ActionListener<BulkResponse> listener) {
228237

229238
private void handleBulkSuccess(BulkResponse bulkResponse) {
230239
responses.add(bulkResponse);
231-
currentBulkSize = 0L;
232240
bulkRequest = null;
233241
}
234242

@@ -237,7 +245,6 @@ private void handleBulkFailure(boolean isFirstRequest, Exception e) {
237245
globalFailure = isFirstRequest;
238246
bulkActionLevelFailure = e;
239247
addItemLevelFailures(bulkRequest.requests());
240-
currentBulkSize = 0;
241248
bulkRequest = null;
242249
}
243250

@@ -257,19 +264,18 @@ private boolean internalAddItems(List<DocWriteRequest<?>> items, Releasable rele
257264
bulkRequest.add(items);
258265
releasables.add(releasable);
259266
long size = items.stream().mapToLong(Accountable::ramBytesUsed).sum();
260-
releasables.add(indexingPressure.markCoordinatingOperationStarted(items.size(), size, false));
261-
currentBulkSize += size;
267+
incrementalOperation.increment(items.size(), size);
262268
return true;
263269
} catch (EsRejectedExecutionException e) {
264270
handleBulkFailure(incrementalRequestSubmitted == false, e);
271+
incrementalOperation.split().close();
265272
releasables.forEach(Releasable::close);
266273
releasables.clear();
267274
return false;
268275
}
269276
}
270277

271278
private void createNewBulkRequest(BulkRequest.IncrementalState incrementalState) {
272-
assert currentBulkSize == 0L;
273279
assert bulkRequest == null;
274280
bulkRequest = new BulkRequest();
275281
bulkRequest.incrementalState(incrementalState);

server/src/main/java/org/elasticsearch/index/IndexingPressure.java

Lines changed: 160 additions & 56 deletions
Original file line numberDiff line numberDiff line change
@@ -138,48 +138,167 @@ private static Releasable wrapReleasable(Releasable releasable) {
138138
};
139139
}
140140

141-
public Releasable markCoordinatingOperationStarted(int operations, long bytes, boolean forceExecution) {
142-
long combinedBytes = this.currentCombinedCoordinatingAndPrimaryBytes.addAndGet(bytes);
143-
long replicaWriteBytes = this.currentReplicaBytes.get();
144-
long totalBytes = combinedBytes + replicaWriteBytes;
145-
if (forceExecution == false && totalBytes > coordinatingLimit) {
146-
long bytesWithoutOperation = combinedBytes - bytes;
147-
long totalBytesWithoutOperation = totalBytes - bytes;
148-
this.currentCombinedCoordinatingAndPrimaryBytes.getAndAdd(-bytes);
149-
this.coordinatingRejections.getAndIncrement();
150-
throw new EsRejectedExecutionException(
151-
"rejected execution of coordinating operation ["
152-
+ "coordinating_and_primary_bytes="
153-
+ bytesWithoutOperation
154-
+ ", "
155-
+ "replica_bytes="
156-
+ replicaWriteBytes
157-
+ ", "
158-
+ "all_bytes="
159-
+ totalBytesWithoutOperation
160-
+ ", "
161-
+ "coordinating_operation_bytes="
162-
+ bytes
163-
+ ", "
164-
+ "max_coordinating_bytes="
165-
+ coordinatingLimit
166-
+ "]",
167-
false
168-
);
141+
public Incremental startIncrementalCoordinating(int operations, long bytes, boolean forceExecution) {
142+
Incremental coordinating = new Incremental(forceExecution);
143+
coordinating.coordinating.increment(operations, bytes);
144+
return coordinating;
145+
}
146+
147+
public Coordinating markCoordinatingOperationStarted(int operations, long bytes, boolean forceExecution) {
148+
Coordinating coordinating = new Coordinating(forceExecution);
149+
coordinating.increment(operations, bytes);
150+
return coordinating;
151+
}
152+
153+
public class Incremental implements Releasable {
154+
155+
private final AtomicBoolean closed = new AtomicBoolean();
156+
private final boolean forceExecution;
157+
private long currentUnparsedSize = 0;
158+
private long totalParsedBytes = 0;
159+
private Coordinating coordinating;
160+
161+
public Incremental(boolean forceExecution) {
162+
this.forceExecution = forceExecution;
163+
this.coordinating = new Coordinating(forceExecution);
164+
}
165+
166+
public long totalParsedBytes() {
167+
return totalParsedBytes;
168+
}
169+
170+
public void incrementUnparsedBytes(long bytes) {
171+
assert closed.get() == false;
172+
// TODO: Implement integration with IndexingPressure for unparsed bytes
173+
currentUnparsedSize += bytes;
174+
}
175+
176+
public void transferUnparsedBytesToParsed(long bytes) {
177+
assert closed.get() == false;
178+
assert currentUnparsedSize >= bytes;
179+
currentUnparsedSize -= bytes;
180+
totalParsedBytes += bytes;
181+
}
182+
183+
public void increment(int operations, long bytes) {
184+
// TODO: Eventually most of the memory will already be accounted for in unparsed.
185+
coordinating.increment(operations, bytes);
186+
}
187+
188+
public long currentOperationsSize() {
189+
return coordinating.currentOperationsSize;
190+
}
191+
192+
public boolean shouldSplit() {
193+
long currentUsage = (currentCombinedCoordinatingAndPrimaryBytes.get() + currentReplicaBytes.get());
194+
long currentOperationsSize = coordinating.currentOperationsSize;
195+
if (currentUsage >= highWatermark && currentOperationsSize >= highWatermarkSize) {
196+
highWaterMarkSplits.getAndIncrement();
197+
logger.trace(
198+
() -> Strings.format(
199+
"Split bulk due to high watermark: current bytes [%d] and size [%d]",
200+
currentUsage,
201+
currentOperationsSize
202+
)
203+
);
204+
return true;
205+
}
206+
if (currentUsage >= lowWatermark && currentOperationsSize >= lowWatermarkSize) {
207+
lowWaterMarkSplits.getAndIncrement();
208+
logger.trace(
209+
() -> Strings.format(
210+
"Split bulk due to low watermark: current bytes [%d] and size [%d]",
211+
currentUsage,
212+
currentOperationsSize
213+
)
214+
);
215+
return true;
216+
}
217+
return false;
218+
}
219+
220+
public Coordinating split() {
221+
Coordinating toReturn = coordinating;
222+
coordinating = new Coordinating(forceExecution);
223+
return toReturn;
224+
}
225+
226+
@Override
227+
public void close() {
228+
coordinating.close();
229+
}
230+
}
231+
232+
// TODO: Maybe this should be re-named and used for primary operations too. Eventually we will need to account for: ingest pipeline
233+
// expansions, reading updates, etc. This could just be a generic OP that could be expanded as appropriate
234+
public class Coordinating implements Releasable {
235+
236+
private final AtomicBoolean closed = new AtomicBoolean();
237+
private final boolean forceExecution;
238+
private int currentOperations = 0;
239+
private long currentOperationsSize = 0;
240+
241+
public Coordinating(boolean forceExecution) {
242+
this.forceExecution = forceExecution;
243+
}
244+
245+
private void increment(int operations, long bytes) {
246+
assert closed.get() == false;
247+
long combinedBytes = currentCombinedCoordinatingAndPrimaryBytes.addAndGet(bytes);
248+
long replicaWriteBytes = currentReplicaBytes.get();
249+
long totalBytes = combinedBytes + replicaWriteBytes;
250+
if (forceExecution == false && totalBytes > coordinatingLimit) {
251+
long bytesWithoutOperation = combinedBytes - bytes;
252+
long totalBytesWithoutOperation = totalBytes - bytes;
253+
currentCombinedCoordinatingAndPrimaryBytes.getAndAdd(-bytes);
254+
coordinatingRejections.getAndIncrement();
255+
throw new EsRejectedExecutionException(
256+
"rejected execution of coordinating operation ["
257+
+ "coordinating_and_primary_bytes="
258+
+ bytesWithoutOperation
259+
+ ", "
260+
+ "replica_bytes="
261+
+ replicaWriteBytes
262+
+ ", "
263+
+ "all_bytes="
264+
+ totalBytesWithoutOperation
265+
+ ", "
266+
+ "coordinating_operation_bytes="
267+
+ bytes
268+
+ ", "
269+
+ "max_coordinating_bytes="
270+
+ coordinatingLimit
271+
+ "]",
272+
false
273+
);
274+
}
275+
currentOperations += operations;
276+
currentOperationsSize += bytes;
277+
logger.trace(() -> Strings.format("adding [%d] coordinating operations and [%d] bytes", operations, bytes));
278+
currentCoordinatingBytes.getAndAdd(bytes);
279+
currentCoordinatingOps.getAndAdd(operations);
280+
totalCombinedCoordinatingAndPrimaryBytes.getAndAdd(bytes);
281+
totalCoordinatingBytes.getAndAdd(bytes);
282+
totalCoordinatingOps.getAndAdd(operations);
283+
totalCoordinatingRequests.getAndIncrement();
284+
}
285+
286+
@Override
287+
public void close() {
288+
if (closed.compareAndSet(false, true)) {
289+
logger.trace(
290+
() -> Strings.format("removing [%d] coordinating operations and [%d] bytes", currentOperations, currentOperationsSize)
291+
);
292+
currentCombinedCoordinatingAndPrimaryBytes.getAndAdd(-currentOperationsSize);
293+
currentCoordinatingBytes.getAndAdd(-currentOperationsSize);
294+
currentCoordinatingOps.getAndAdd(-currentOperations);
295+
currentOperationsSize = 0;
296+
currentOperations = 0;
297+
} else {
298+
logger.error("IndexingPressure memory is adjusted twice", new IllegalStateException("Releasable is called twice"));
299+
assert false : "IndexingPressure is adjusted twice";
300+
}
169301
}
170-
logger.trace(() -> Strings.format("adding [%d] coordinating operations and [%d] bytes", operations, bytes));
171-
currentCoordinatingBytes.getAndAdd(bytes);
172-
currentCoordinatingOps.getAndAdd(operations);
173-
totalCombinedCoordinatingAndPrimaryBytes.getAndAdd(bytes);
174-
totalCoordinatingBytes.getAndAdd(bytes);
175-
totalCoordinatingOps.getAndAdd(operations);
176-
totalCoordinatingRequests.getAndIncrement();
177-
return wrapReleasable(() -> {
178-
logger.trace(() -> Strings.format("removing [%d] coordinating operations and [%d] bytes", operations, bytes));
179-
this.currentCombinedCoordinatingAndPrimaryBytes.getAndAdd(-bytes);
180-
this.currentCoordinatingBytes.getAndAdd(-bytes);
181-
this.currentCoordinatingOps.getAndAdd(-operations);
182-
});
183302
}
184303

185304
public Releasable markPrimaryOperationLocalToCoordinatingNodeStarted(int operations, long bytes) {
@@ -266,21 +385,6 @@ public Releasable markReplicaOperationStarted(int operations, long bytes, boolea
266385
});
267386
}
268387

269-
public boolean shouldSplitBulk(long size) {
270-
long currentUsage = (currentCombinedCoordinatingAndPrimaryBytes.get() + currentReplicaBytes.get());
271-
if (currentUsage >= highWatermark && size >= highWatermarkSize) {
272-
highWaterMarkSplits.getAndIncrement();
273-
logger.trace(() -> Strings.format("Split bulk due to high watermark: current bytes [%d] and size [%d]", currentUsage, size));
274-
return (true);
275-
}
276-
if (currentUsage >= lowWatermark && size >= lowWatermarkSize) {
277-
lowWaterMarkSplits.getAndIncrement();
278-
logger.trace(() -> Strings.format("Split bulk due to low watermark: current bytes [%d] and size [%d]", currentUsage, size));
279-
return (true);
280-
}
281-
return (false);
282-
}
283-
284388
public IndexingPressureStats stats() {
285389
return new IndexingPressureStats(
286390
totalCombinedCoordinatingAndPrimaryBytes.get(),

0 commit comments

Comments
 (0)