-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Add reusable smoke test coverage for concurrent UPDATE #10368
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
Changes from all commits
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 |
|---|---|---|
|
|
@@ -26,16 +26,24 @@ | |
| import org.testng.annotations.DataProvider; | ||
| import org.testng.annotations.Test; | ||
|
|
||
| import java.util.List; | ||
| import java.util.Optional; | ||
| import java.util.concurrent.CyclicBarrier; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Future; | ||
| import java.util.function.Consumer; | ||
| import java.util.stream.IntStream; | ||
| import java.util.stream.Stream; | ||
|
|
||
| import static com.google.common.collect.ImmutableList.toImmutableList; | ||
| import static io.airlift.concurrent.MoreFutures.tryGetFutureValue; | ||
| import static io.trino.SystemSessionProperties.IGNORE_STATS_CALCULATOR_FAILURES; | ||
| import static io.trino.spi.type.VarcharType.VARCHAR; | ||
| import static io.trino.sql.planner.optimizations.PlanNodeSearcher.searchFrom; | ||
| import static io.trino.sql.planner.planprinter.PlanPrinter.textLogicalPlan; | ||
| import static io.trino.testing.DataProviders.toDataProvider; | ||
| import static io.trino.testing.QueryAssertions.assertContains; | ||
| import static io.trino.testing.QueryAssertions.getTrinoExceptionCause; | ||
| import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ADD_COLUMN; | ||
| import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ARRAY; | ||
| import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_COMMENT_ON_COLUMN; | ||
|
|
@@ -66,6 +74,9 @@ | |
| import static java.lang.String.join; | ||
| import static java.util.Collections.nCopies; | ||
| import static java.util.Locale.ENGLISH; | ||
| import static java.util.concurrent.Executors.newFixedThreadPool; | ||
| import static java.util.concurrent.TimeUnit.SECONDS; | ||
| import static java.util.stream.Collectors.joining; | ||
| import static org.assertj.core.api.Assertions.assertThat; | ||
| import static org.assertj.core.api.Assertions.assertThatThrownBy; | ||
| import static org.testng.Assert.assertFalse; | ||
|
|
@@ -1280,6 +1291,72 @@ public void testUpdate() | |
| } | ||
| } | ||
|
|
||
| // Repeat test with invocationCount for better test coverage, since the tested aspect is inherently non-deterministic. | ||
| @Test(timeOut = 60_000, invocationCount = 4) | ||
| public void testUpdateRowConcurrently() | ||
| throws Exception | ||
| { | ||
| if (!hasBehavior(SUPPORTS_UPDATE)) { | ||
| // Covered by testUpdate | ||
| return; | ||
| } | ||
|
|
||
| int threads = 4; | ||
| CyclicBarrier barrier = new CyclicBarrier(threads); | ||
| ExecutorService executor = newFixedThreadPool(threads); | ||
| try (TestTable table = new TestTable( | ||
| getQueryRunner()::execute, | ||
| "test_concurrent_update", | ||
| IntStream.range(0, threads) | ||
| .mapToObj(i -> format("col%s integer", i)) | ||
| .collect(joining(", ", "(", ")")))) { | ||
| String tableName = table.getName(); | ||
| assertUpdate(format("INSERT INTO %s VALUES (%s)", tableName, join(",", nCopies(threads, "0"))), 1); | ||
|
|
||
| List<Future<Boolean>> futures = IntStream.range(0, threads) | ||
| .mapToObj(threadNumber -> executor.submit(() -> { | ||
| barrier.await(10, SECONDS); | ||
| try { | ||
| String columnName = "col" + threadNumber; | ||
| getQueryRunner().execute(format("UPDATE %s SET %s = %s + 1", tableName, columnName, columnName)); | ||
| return true; | ||
| } | ||
| catch (Exception e) { | ||
| RuntimeException trinoException = getTrinoExceptionCause(e); | ||
| try { | ||
| verifyConcurrentUpdateFailurePermissible(trinoException); | ||
| } | ||
| catch (Throwable verifyFailure) { | ||
| if (trinoException != e && verifyFailure != e) { | ||
|
Member
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. why
Member
Author
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. because |
||
| verifyFailure.addSuppressed(e); | ||
| } | ||
| throw verifyFailure; | ||
| } | ||
| return false; | ||
| } | ||
| })) | ||
| .collect(toImmutableList()); | ||
|
|
||
| String expected = futures.stream() | ||
| .map(future -> tryGetFutureValue(future, 10, SECONDS).orElseThrow(() -> new RuntimeException("Wait timed out"))) | ||
| .map(success -> success ? "1" : "0") | ||
| .collect(joining(",", "VALUES (", ")")); | ||
|
|
||
| assertThat(query("TABLE " + tableName)) | ||
| .matches(expected); | ||
| } | ||
| finally { | ||
| executor.shutdownNow(); | ||
| executor.awaitTermination(10, SECONDS); | ||
| } | ||
| } | ||
|
|
||
| protected void verifyConcurrentUpdateFailurePermissible(Exception e) | ||
| { | ||
| // By default, do not expect UPDATE to fail in case of concurrent updates | ||
| throw new AssertionError("Unexpected concurrent update failure", e); | ||
| } | ||
|
|
||
| @Test | ||
| public void testTruncateTable() | ||
| { | ||
|
|
||
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.
#10518