Skip to content

Commit 02247cc

Browse files
author
Hendrik Muhs
committed
[ML-DataFrame] adapt page size on circuit breaker responses (#41149)
handle circuit breaker response and adapt page size to reduce memory pressure, reduce preview buckets to 100, initial page size to 500
1 parent 043c1f5 commit 02247cc

File tree

7 files changed

+409
-35
lines changed

7 files changed

+409
-35
lines changed

x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/DataFrameMessages.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -56,6 +56,11 @@ public class DataFrameMessages {
5656
"Failed to parse group_by for data frame pivot transform";
5757
public static final String LOG_DATA_FRAME_TRANSFORM_CONFIGURATION_BAD_AGGREGATION =
5858
"Failed to parse aggregation for data frame pivot transform";
59+
public static final String LOG_DATA_FRAME_TRANSFORM_PIVOT_REDUCE_PAGE_SIZE =
60+
"Search returned with out of memory error, reducing number of buckets per search from [{0}] to [{1}]";
61+
public static final String LOG_DATA_FRAME_TRANSFORM_PIVOT_LOW_PAGE_SIZE_FAILURE =
62+
"Search returned with out of memory error after repeated page size reductions to [{0}], unable to continue pivot, "
63+
+ "please simplify job or increase heap size on data nodes.";
5964

6065
public static final String FAILED_TO_PARSE_TRANSFORM_CHECKPOINTS =
6166
"Failed to parse transform checkpoints for [{0}]";

x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -260,7 +260,8 @@ public void testPreviewTransform() throws Exception {
260260
createPreviewRequest.setJsonEntity(config);
261261
Map<String, Object> previewDataframeResponse = entityAsMap(client().performRequest(createPreviewRequest));
262262
List<Map<String, Object>> preview = (List<Map<String, Object>>)previewDataframeResponse.get("preview");
263-
assertThat(preview.size(), equalTo(393));
263+
// preview is limited to 100
264+
assertThat(preview.size(), equalTo(100));
264265
Set<String> expectedFields = new HashSet<>(Arrays.asList("reviewer", "by_day", "avg_rating"));
265266
preview.forEach(p -> {
266267
Set<String> keys = p.keySet();

x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPreviewDataFrameTransformAction.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@
3535
public class TransportPreviewDataFrameTransformAction extends
3636
HandledTransportAction<PreviewDataFrameTransformAction.Request, PreviewDataFrameTransformAction.Response> {
3737

38+
private static final int NUMBER_OF_PREVIEW_BUCKETS = 100;
3839
private final XPackLicenseState licenseState;
3940
private final Client client;
4041
private final ThreadPool threadPool;
@@ -77,7 +78,7 @@ private void getPreview(Pivot pivot, ActionListener<List<Map<String, Object>>> l
7778
ClientHelper.DATA_FRAME_ORIGIN,
7879
client,
7980
SearchAction.INSTANCE,
80-
pivot.buildSearchRequest(null),
81+
pivot.buildSearchRequest(null, NUMBER_OF_PREVIEW_BUCKETS),
8182
ActionListener.wrap(
8283
r -> {
8384
final CompositeAggregation agg = r.getAggregations().get(COMPOSITE_AGGREGATION_NAME);

x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexer.java

Lines changed: 96 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -8,14 +8,21 @@
88

99
import org.apache.logging.log4j.LogManager;
1010
import org.apache.logging.log4j.Logger;
11+
import org.elasticsearch.ExceptionsHelper;
1112
import org.elasticsearch.action.ActionListener;
1213
import org.elasticsearch.action.index.IndexRequest;
14+
import org.elasticsearch.action.search.SearchPhaseExecutionException;
1315
import org.elasticsearch.action.search.SearchRequest;
1416
import org.elasticsearch.action.search.SearchResponse;
17+
import org.elasticsearch.action.search.ShardSearchFailure;
18+
import org.elasticsearch.common.breaker.CircuitBreakingException;
1519
import org.elasticsearch.common.xcontent.XContentBuilder;
1620
import org.elasticsearch.index.query.QueryBuilder;
1721
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation;
22+
import org.elasticsearch.xpack.core.common.notifications.Auditor;
1823
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
24+
import org.elasticsearch.xpack.core.dataframe.DataFrameMessages;
25+
import org.elasticsearch.xpack.core.dataframe.notifications.DataFrameAuditMessage;
1926
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats;
2027
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig;
2128
import org.elasticsearch.xpack.core.indexing.AsyncTwoPhaseIndexer;
@@ -26,6 +33,7 @@
2633
import java.io.IOException;
2734
import java.io.UncheckedIOException;
2835
import java.util.Map;
36+
import java.util.Objects;
2937
import java.util.concurrent.Executor;
3038
import java.util.concurrent.atomic.AtomicReference;
3139
import java.util.stream.Collectors;
@@ -35,22 +43,34 @@
3543

3644
public abstract class DataFrameIndexer extends AsyncTwoPhaseIndexer<Map<String, Object>, DataFrameIndexerTransformStats> {
3745

46+
public static final int MINIMUM_PAGE_SIZE = 10;
3847
public static final String COMPOSITE_AGGREGATION_NAME = "_data_frame";
3948
private static final Logger logger = LogManager.getLogger(DataFrameIndexer.class);
4049

50+
protected final Auditor<DataFrameAuditMessage> auditor;
51+
4152
private Pivot pivot;
53+
private int pageSize = 0;
4254

4355
public DataFrameIndexer(Executor executor,
56+
Auditor<DataFrameAuditMessage> auditor,
4457
AtomicReference<IndexerState> initialState,
4558
Map<String, Object> initialPosition,
4659
DataFrameIndexerTransformStats jobStats) {
4760
super(executor, initialState, initialPosition, jobStats);
61+
this.auditor = Objects.requireNonNull(auditor);
4862
}
4963

5064
protected abstract DataFrameTransformConfig getConfig();
5165

5266
protected abstract Map<String, String> getFieldMappings();
5367

68+
protected abstract void failIndexer(String message);
69+
70+
public int getPageSize() {
71+
return pageSize;
72+
}
73+
5474
/**
5575
* Request a checkpoint
5676
*/
@@ -62,6 +82,11 @@ protected void onStart(long now, ActionListener<Void> listener) {
6282
QueryBuilder queryBuilder = getConfig().getSource().getQueryConfig().getQuery();
6383
pivot = new Pivot(getConfig().getSource().getIndex(), queryBuilder, getConfig().getPivotConfig());
6484

85+
// if we haven't set the page size yet, if it is set we might have reduced it after running into an out of memory
86+
if (pageSize == 0) {
87+
pageSize = pivot.getInitialPageSize();
88+
}
89+
6590
// if run for the 1st time, create checkpoint
6691
if (getPosition() == null) {
6792
createCheckpoint(listener);
@@ -73,6 +98,12 @@ protected void onStart(long now, ActionListener<Void> listener) {
7398
}
7499
}
75100

101+
@Override
102+
protected void onFinish(ActionListener<Void> listener) {
103+
// reset the page size, so we do not memorize a low page size forever, the pagesize will be re-calculated on start
104+
pageSize = 0;
105+
}
106+
76107
@Override
77108
protected IterationResult<Map<String, Object>> doProcess(SearchResponse searchResponse) {
78109
final CompositeAggregation agg = searchResponse.getAggregations().get(COMPOSITE_AGGREGATION_NAME);
@@ -121,6 +152,70 @@ private Stream<IndexRequest> processBucketsToIndexRequests(CompositeAggregation
121152

122153
@Override
123154
protected SearchRequest buildSearchRequest() {
124-
return pivot.buildSearchRequest(getPosition());
155+
return pivot.buildSearchRequest(getPosition(), pageSize);
156+
}
157+
158+
/**
159+
* Handle the circuit breaking case: A search consumed to much memory and got aborted.
160+
*
161+
* Going out of memory we smoothly reduce the page size which reduces memory consumption.
162+
*
163+
* Implementation details: We take the values from the circuit breaker as a hint, but
164+
* note that it breaks early, that's why we also reduce using
165+
*
166+
* @param e Exception thrown, only {@link CircuitBreakingException} are handled
167+
* @return true if exception was handled, false if not
168+
*/
169+
protected boolean handleCircuitBreakingException(Exception e) {
170+
CircuitBreakingException circuitBreakingException = getCircuitBreakingException(e);
171+
172+
if (circuitBreakingException == null) {
173+
return false;
174+
}
175+
176+
double reducingFactor = Math.min((double) circuitBreakingException.getByteLimit() / circuitBreakingException.getBytesWanted(),
177+
1 - (Math.log10(pageSize) * 0.1));
178+
179+
int newPageSize = (int) Math.round(reducingFactor * pageSize);
180+
181+
if (newPageSize < MINIMUM_PAGE_SIZE) {
182+
String message = DataFrameMessages.getMessage(DataFrameMessages.LOG_DATA_FRAME_TRANSFORM_PIVOT_LOW_PAGE_SIZE_FAILURE, pageSize);
183+
failIndexer(message);
184+
return true;
185+
}
186+
187+
String message = DataFrameMessages.getMessage(DataFrameMessages.LOG_DATA_FRAME_TRANSFORM_PIVOT_REDUCE_PAGE_SIZE, pageSize,
188+
newPageSize);
189+
auditor.info(getJobId(), message);
190+
logger.info("Data frame transform [" + getJobId() + "]:" + message);
191+
192+
pageSize = newPageSize;
193+
return true;
194+
}
195+
196+
/**
197+
* Inspect exception for circuit breaking exception and return the first one it can find.
198+
*
199+
* @param e Exception
200+
* @return CircuitBreakingException instance if found, null otherwise
201+
*/
202+
private static CircuitBreakingException getCircuitBreakingException(Exception e) {
203+
// circuit breaking exceptions are at the bottom
204+
Throwable unwrappedThrowable = ExceptionsHelper.unwrapCause(e);
205+
206+
if (unwrappedThrowable instanceof CircuitBreakingException) {
207+
return (CircuitBreakingException) unwrappedThrowable;
208+
} else if (unwrappedThrowable instanceof SearchPhaseExecutionException) {
209+
SearchPhaseExecutionException searchPhaseException = (SearchPhaseExecutionException) e;
210+
for (ShardSearchFailure shardFailure : searchPhaseException.shardFailures()) {
211+
Throwable unwrappedShardFailure = ExceptionsHelper.unwrapCause(shardFailure.getCause());
212+
213+
if (unwrappedShardFailure instanceof CircuitBreakingException) {
214+
return (CircuitBreakingException) unwrappedShardFailure;
215+
}
216+
}
217+
}
218+
219+
return null;
125220
}
126221
}

x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java

Lines changed: 44 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -277,25 +277,6 @@ void persistStateToClusterState(DataFrameTransformState state,
277277
));
278278
}
279279

280-
private boolean isIrrecoverableFailure(Exception e) {
281-
return e instanceof IndexNotFoundException || e instanceof DataFrameConfigurationException;
282-
}
283-
284-
synchronized void handleFailure(Exception e) {
285-
if (isIrrecoverableFailure(e) || failureCount.incrementAndGet() > MAX_CONTINUOUS_FAILURES) {
286-
String failureMessage = isIrrecoverableFailure(e) ?
287-
"task encountered irrecoverable failure: " + e.getMessage() :
288-
"task encountered more than " + MAX_CONTINUOUS_FAILURES + " failures; latest failure: " + e.getMessage();
289-
auditor.error(transform.getId(), failureMessage);
290-
stateReason.set(failureMessage);
291-
taskState.set(DataFrameTransformTaskState.FAILED);
292-
persistStateToClusterState(getState(), ActionListener.wrap(
293-
r -> failureCount.set(0), // Successfully marked as failed, reset counter so that task can be restarted
294-
exception -> {} // Noop, internal method logs the failure to update the state
295-
));
296-
}
297-
}
298-
299280
/**
300281
* This is called when the persistent task signals that the allocated task should be terminated.
301282
* Termination in the task framework is essentially voluntary, as the allocated task can only be
@@ -313,13 +294,11 @@ public synchronized void onCancelled() {
313294

314295
protected class ClientDataFrameIndexer extends DataFrameIndexer {
315296
private static final int LOAD_TRANSFORM_TIMEOUT_IN_SECONDS = 30;
316-
private static final int CREATE_CHECKPOINT_TIMEOUT_IN_SECONDS = 30;
317297

318298
private final Client client;
319299
private final DataFrameTransformsConfigManager transformsConfigManager;
320300
private final DataFrameTransformsCheckpointService transformsCheckpointService;
321301
private final String transformId;
322-
private final Auditor<DataFrameAuditMessage> auditor;
323302
private volatile DataFrameIndexerTransformStats previouslyPersistedStats = null;
324303
// Keeps track of the last exception that was written to our audit, keeps us from spamming the audit index
325304
private volatile String lastAuditedExceptionMessage = null;
@@ -331,13 +310,12 @@ public ClientDataFrameIndexer(String transformId, DataFrameTransformsConfigManag
331310
DataFrameTransformsCheckpointService transformsCheckpointService,
332311
AtomicReference<IndexerState> initialState, Map<String, Object> initialPosition, Client client,
333312
Auditor<DataFrameAuditMessage> auditor) {
334-
super(threadPool.executor(ThreadPool.Names.GENERIC), initialState, initialPosition,
313+
super(threadPool.executor(ThreadPool.Names.GENERIC), auditor, initialState, initialPosition,
335314
new DataFrameIndexerTransformStats(transformId));
336315
this.transformId = transformId;
337316
this.transformsConfigManager = transformsConfigManager;
338317
this.transformsCheckpointService = transformsCheckpointService;
339318
this.client = client;
340-
this.auditor = auditor;
341319
}
342320

343321
@Override
@@ -474,19 +452,26 @@ protected void doSaveState(IndexerState indexerState, Map<String, Object> positi
474452

475453
@Override
476454
protected void onFailure(Exception exc) {
477-
// Since our schedule fires again very quickly after failures it is possible to run into the same failure numerous
478-
// times in a row, very quickly. We do not want to spam the audit log with repeated failures, so only record the first one
479-
if (exc.getMessage().equals(lastAuditedExceptionMessage) == false) {
480-
auditor.warning(transform.getId(), "Data frame transform encountered an exception: " + exc.getMessage());
481-
lastAuditedExceptionMessage = exc.getMessage();
455+
// the failure handler must not throw an exception due to internal problems
456+
try {
457+
logger.warn("Data frame transform [" + transform.getId() + "] encountered an exception: ", exc);
458+
459+
// Since our schedule fires again very quickly after failures it is possible to run into the same failure numerous
460+
// times in a row, very quickly. We do not want to spam the audit log with repeated failures, so only record the first one
461+
if (exc.getMessage().equals(lastAuditedExceptionMessage) == false) {
462+
auditor.warning(transform.getId(), "Data frame transform encountered an exception: " + exc.getMessage());
463+
lastAuditedExceptionMessage = exc.getMessage();
464+
}
465+
handleFailure(exc);
466+
} catch (Exception e) {
467+
logger.error("Data frame transform encountered an unexpected internal exception: " ,e);
482468
}
483-
logger.warn("Data frame transform [" + transform.getId() + "] encountered an exception: ", exc);
484-
handleFailure(exc);
485469
}
486470

487471
@Override
488472
protected void onFinish(ActionListener<Void> listener) {
489473
try {
474+
super.onFinish(listener);
490475
long checkpoint = currentCheckpoint.incrementAndGet();
491476
auditor.info(transform.getId(), "Finished indexing for data frame transform checkpoint [" + checkpoint + "]");
492477
logger.info("Finished indexing for data frame transform [" + transform.getId() + "] checkpoint [" + checkpoint + "]");
@@ -515,6 +500,35 @@ protected void createCheckpoint(ActionListener<Void> listener) {
515500
listener.onFailure(new RuntimeException("Failed to retrieve checkpoint", getCheckPointException));
516501
}));
517502
}
503+
504+
private boolean isIrrecoverableFailure(Exception e) {
505+
return e instanceof IndexNotFoundException || e instanceof DataFrameConfigurationException;
506+
}
507+
508+
synchronized void handleFailure(Exception e) {
509+
if (handleCircuitBreakingException(e)) {
510+
return;
511+
}
512+
513+
if (isIrrecoverableFailure(e) || failureCount.incrementAndGet() > MAX_CONTINUOUS_FAILURES) {
514+
String failureMessage = isIrrecoverableFailure(e) ?
515+
"task encountered irrecoverable failure: " + e.getMessage() :
516+
"task encountered more than " + MAX_CONTINUOUS_FAILURES + " failures; latest failure: " + e.getMessage();
517+
failIndexer(failureMessage);
518+
}
519+
}
520+
521+
@Override
522+
protected void failIndexer(String failureMessage) {
523+
logger.error("Data frame transform [" + getJobId() + "]:" + failureMessage);
524+
auditor.error(transform.getId(), failureMessage);
525+
stateReason.set(failureMessage);
526+
taskState.set(DataFrameTransformTaskState.FAILED);
527+
persistStateToClusterState(DataFrameTransformTask.this.getState(), ActionListener.wrap(
528+
r -> failureCount.set(0), // Successfully marked as failed, reset counter so that task can be restarted
529+
exception -> {} // Noop, internal method logs the failure to update the state
530+
));
531+
}
518532
}
519533

520534
class DataFrameConfigurationException extends RuntimeException {

x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/Pivot.java

Lines changed: 21 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,8 @@
3535
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
3636

3737
public class Pivot {
38+
public static final int DEFAULT_INITIAL_PAGE_SIZE = 500;
39+
3840
private static final String COMPOSITE_AGGREGATION_NAME = "_data_frame";
3941

4042
private final PivotConfig config;
@@ -68,11 +70,29 @@ public void deduceMappings(Client client, final ActionListener<Map<String, Strin
6870
SchemaUtil.deduceMappings(client, config, source, listener);
6971
}
7072

71-
public SearchRequest buildSearchRequest(Map<String, Object> position) {
73+
/**
74+
* Get the initial page size for this pivot.
75+
*
76+
* The page size is the main parameter for adjusting memory consumption. Memory consumption mainly depends on
77+
* the page size, the type of aggregations and the data. As the page size is the number of buckets we return
78+
* per page the page size is a multiplier for the costs of aggregating bucket.
79+
*
80+
* Initially this returns a default, in future it might inspect the configuration and base the initial size
81+
* on the aggregations used.
82+
*
83+
* @return the page size
84+
*/
85+
public int getInitialPageSize() {
86+
return DEFAULT_INITIAL_PAGE_SIZE;
87+
}
88+
89+
public SearchRequest buildSearchRequest(Map<String, Object> position, int pageSize) {
7290
if (position != null) {
7391
cachedCompositeAggregation.aggregateAfter(position);
7492
}
7593

94+
cachedCompositeAggregation.size(pageSize);
95+
7696
return cachedSearchRequest;
7797
}
7898

@@ -127,7 +147,6 @@ private static CompositeAggregationBuilder createCompositeAggregation(PivotConfi
127147
XContentParser parser = builder.generator().contentType().xContent().createParser(NamedXContentRegistry.EMPTY,
128148
LoggingDeprecationHandler.INSTANCE, BytesReference.bytes(builder).streamInput());
129149
compositeAggregation = CompositeAggregationBuilder.parse(COMPOSITE_AGGREGATION_NAME, parser);
130-
compositeAggregation.size(1000);
131150
config.getAggregationConfig().getAggregatorFactories().forEach(agg -> compositeAggregation.subAggregation(agg));
132151
} catch (IOException e) {
133152
throw new RuntimeException(DataFrameMessages.DATA_FRAME_TRANSFORM_PIVOT_FAILED_TO_CREATE_COMPOSITE_AGGREGATION, e);

0 commit comments

Comments
 (0)