-
Notifications
You must be signed in to change notification settings - Fork 25.2k
ESQL: Limit Replace function memory usage #127924
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: main
Are you sure you want to change the base?
Changes from all commits
a6f883a
821e627
992b11e
833e381
4aef2f9
162c91a
455b1fd
49f05ab
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,5 @@ | ||
pr: 127924 | ||
summary: Limit Replace function memory usage | ||
area: ES|QL | ||
type: enhancement | ||
issues: [] |
Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.
Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -28,6 +28,7 @@ | |
import java.io.IOException; | ||
import java.util.Arrays; | ||
import java.util.List; | ||
import java.util.regex.Matcher; | ||
import java.util.regex.Pattern; | ||
import java.util.regex.PatternSyntaxException; | ||
|
||
|
@@ -121,24 +122,63 @@ public boolean foldable() { | |
return str.foldable() && regex.foldable() && newStr.foldable(); | ||
} | ||
|
||
@Evaluator(extraName = "Constant", warnExceptions = PatternSyntaxException.class) | ||
@Evaluator(extraName = "Constant", warnExceptions = IllegalArgumentException.class) | ||
static BytesRef process(BytesRef str, @Fixed Pattern regex, BytesRef newStr) { | ||
if (str == null || regex == null || newStr == null) { | ||
return null; | ||
} | ||
return new BytesRef(regex.matcher(str.utf8ToString()).replaceAll(newStr.utf8ToString())); | ||
return safeReplace(str, regex, newStr); | ||
} | ||
|
||
@Evaluator(warnExceptions = PatternSyntaxException.class) | ||
@Evaluator(warnExceptions = IllegalArgumentException.class) | ||
static BytesRef process(BytesRef str, BytesRef regex, BytesRef newStr) { | ||
if (str == null) { | ||
return null; | ||
} | ||
|
||
if (regex == null || newStr == null) { | ||
return str; | ||
} | ||
return new BytesRef(str.utf8ToString().replaceAll(regex.utf8ToString(), newStr.utf8ToString())); | ||
return safeReplace(str, Pattern.compile(regex.utf8ToString()), newStr); | ||
} | ||
|
||
/** | ||
* Executes a Replace without surpassing the memory limit. | ||
*/ | ||
private static BytesRef safeReplace(BytesRef strBytesRef, Pattern regex, BytesRef newStrBytesRef) { | ||
String str = strBytesRef.utf8ToString(); | ||
Matcher m = regex.matcher(str); | ||
if (false == m.find()) { | ||
return strBytesRef; | ||
} | ||
String newStr = newStrBytesRef.utf8ToString(); | ||
|
||
// Count potential groups (E.g. "$1") used in the replacement | ||
int constantReplacementLength = newStr.length(); | ||
int groupsInReplacement = 0; | ||
for (int i = 0; i < newStr.length(); i++) { | ||
if (newStr.charAt(i) == '$') { | ||
groupsInReplacement++; | ||
constantReplacementLength -= 2; | ||
i++; | ||
} | ||
} | ||
|
||
// Initialize the buffer with an approximate size for the first replacement | ||
StringBuilder result = new StringBuilder(str.length() + newStr.length() + 8); | ||
do { | ||
int matchSize = m.end() - m.start(); | ||
int potentialReplacementSize = constantReplacementLength + groupsInReplacement * matchSize; | ||
int remainingStr = str.length() - m.end(); | ||
if (result.length() + potentialReplacementSize + remainingStr > MAX_BYTES_REF_RESULT_SIZE) { | ||
throw new IllegalArgumentException( | ||
"Creating strings with more than [" + MAX_BYTES_REF_RESULT_SIZE + "] bytes is not supported" | ||
); | ||
} | ||
|
||
m.appendReplacement(result, newStr); | ||
} while (m.find()); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Comparing this with the implementation of |
||
m.appendTail(result); | ||
return new BytesRef(result.toString()); | ||
} | ||
|
||
@Override | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,139 @@ | ||
/* | ||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one | ||
* or more contributor license agreements. Licensed under the Elastic License | ||
* 2.0; you may not use this file except in compliance with the Elastic License | ||
* 2.0. | ||
*/ | ||
|
||
package org.elasticsearch.xpack.esql.expression.function.scalar.string; | ||
|
||
import org.apache.lucene.util.BytesRef; | ||
import org.elasticsearch.common.breaker.CircuitBreaker; | ||
import org.elasticsearch.common.unit.ByteSizeValue; | ||
import org.elasticsearch.common.util.BigArrays; | ||
import org.elasticsearch.common.util.MockBigArrays; | ||
import org.elasticsearch.common.util.PageCacheRecycler; | ||
import org.elasticsearch.compute.data.Block; | ||
import org.elasticsearch.compute.data.BlockFactory; | ||
import org.elasticsearch.compute.data.BlockUtils; | ||
import org.elasticsearch.compute.data.Page; | ||
import org.elasticsearch.compute.operator.DriverContext; | ||
import org.elasticsearch.compute.test.TestBlockFactory; | ||
import org.elasticsearch.test.ESTestCase; | ||
import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; | ||
import org.elasticsearch.xpack.esql.core.expression.function.scalar.ScalarFunction; | ||
import org.elasticsearch.xpack.esql.core.tree.Source; | ||
import org.elasticsearch.xpack.esql.core.type.DataType; | ||
import org.elasticsearch.xpack.esql.core.type.EsField; | ||
import org.elasticsearch.xpack.esql.expression.function.AbstractScalarFunctionTestCase; | ||
import org.junit.After; | ||
|
||
import java.util.ArrayList; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.Map; | ||
|
||
import static org.hamcrest.Matchers.equalTo; | ||
|
||
/** | ||
* These tests create rows that are 1MB in size. Test classes | ||
* which extend AbstractScalarFunctionTestCase rerun test cases with | ||
* many randomized inputs. Unfortunately, tests are run with | ||
* limited memory, and instantiating many copies of these | ||
* tests with large rows causes out of memory. | ||
*/ | ||
public class ReplaceStaticTests extends ESTestCase { | ||
|
||
public void testLimit() { | ||
int textLength = (int) ScalarFunction.MAX_BYTES_REF_RESULT_SIZE / 10; | ||
String text = randomAlphaOfLength((int) ScalarFunction.MAX_BYTES_REF_RESULT_SIZE / 10); | ||
String regex = "^(.+)$"; | ||
|
||
// 10 times the original text + the remainder | ||
String extraString = "a".repeat((int) ScalarFunction.MAX_BYTES_REF_RESULT_SIZE % 10); | ||
assert textLength * 10 + extraString.length() == ScalarFunction.MAX_BYTES_REF_RESULT_SIZE; | ||
String newStr = "$0$0$0$0$0$0$0$0$0$0" + extraString; | ||
|
||
String result = process(text, regex, newStr); | ||
assertThat(result, equalTo(newStr.replaceAll("\\$\\d", text))); | ||
} | ||
|
||
public void testTooBig() { | ||
String textAndNewStr = randomAlphaOfLength((int) (ScalarFunction.MAX_BYTES_REF_RESULT_SIZE / 10)); | ||
String regex = "."; | ||
|
||
String result = process(textAndNewStr, regex, textAndNewStr); | ||
assertNull(result); | ||
assertWarnings( | ||
"Line -1:-1: evaluation of [] failed, treating result as null. Only first 20 failures recorded.", | ||
"Line -1:-1: java.lang.IllegalArgumentException: " | ||
+ "Creating strings with more than [" | ||
+ ScalarFunction.MAX_BYTES_REF_RESULT_SIZE | ||
+ "] bytes is not supported" | ||
); | ||
} | ||
|
||
public void testTooBigWithGroups() { | ||
int textLength = (int) ScalarFunction.MAX_BYTES_REF_RESULT_SIZE / 10; | ||
String text = randomAlphaOfLength(textLength); | ||
String regex = "(.+)"; | ||
|
||
// 10 times the original text + the remainder + 1 | ||
String extraString = "a".repeat(1 + (int) ScalarFunction.MAX_BYTES_REF_RESULT_SIZE % 10); | ||
assert textLength * 10 + extraString.length() == ScalarFunction.MAX_BYTES_REF_RESULT_SIZE + 1; | ||
String newStr = "$0$1$0$1$0$1$0$1$0$1" + extraString; | ||
|
||
String result = process(text, regex, newStr); | ||
assertNull(result); | ||
assertWarnings( | ||
"Line -1:-1: evaluation of [] failed, treating result as null. Only first 20 failures recorded.", | ||
"Line -1:-1: java.lang.IllegalArgumentException: " | ||
+ "Creating strings with more than [" | ||
+ ScalarFunction.MAX_BYTES_REF_RESULT_SIZE | ||
+ "] bytes is not supported" | ||
); | ||
} | ||
|
||
public String process(String text, String regex, String newStr) { | ||
try ( | ||
var eval = AbstractScalarFunctionTestCase.evaluator( | ||
new Replace( | ||
Source.EMPTY, | ||
field("text", DataType.KEYWORD), | ||
field("regex", DataType.KEYWORD), | ||
field("newStr", DataType.KEYWORD) | ||
) | ||
).get(driverContext()); | ||
Block block = eval.eval(row(List.of(new BytesRef(text), new BytesRef(regex), new BytesRef(newStr)))); | ||
) { | ||
return block.isNull(0) ? null : ((BytesRef) BlockUtils.toJavaObject(block, 0)).utf8ToString(); | ||
} | ||
} | ||
|
||
/** | ||
* The following fields and methods were borrowed from AbstractScalarFunctionTestCase | ||
*/ | ||
private final List<CircuitBreaker> breakers = Collections.synchronizedList(new ArrayList<>()); | ||
|
||
private static Page row(List<Object> values) { | ||
return new Page(1, BlockUtils.fromListRow(TestBlockFactory.getNonBreakingInstance(), values)); | ||
} | ||
|
||
private static FieldAttribute field(String name, DataType type) { | ||
return new FieldAttribute(Source.synthetic(name), name, new EsField(name, type, Map.of(), true)); | ||
} | ||
|
||
private DriverContext driverContext() { | ||
BigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, ByteSizeValue.ofMb(256)).withCircuitBreaking(); | ||
CircuitBreaker breaker = bigArrays.breakerService().getBreaker(CircuitBreaker.REQUEST); | ||
breakers.add(breaker); | ||
return new DriverContext(bigArrays, new BlockFactory(breaker, bigArrays)); | ||
} | ||
|
||
@After | ||
public void allMemoryReleased() { | ||
for (CircuitBreaker breaker : breakers) { | ||
assertThat(breaker.getUsed(), equalTo(0L)); | ||
} | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
PatternSyntaxException
is anIllegalArgumentException
. If we choose another exception for this PR, we would have to restorePatternSyntaxException