Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -269,6 +269,17 @@ public void testUpdate()
.hasMessage("Hive update is only supported for ACID transactional tables");
}

@Override
public void testUpdateRowConcurrently()
throws Exception
{
// TODO (https://github.com/trinodb/trino/issues/10518) test this with a TestHiveConnectorTest version that creates ACID tables by default, or in some other way
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

assertThatThrownBy(super::testUpdateRowConcurrently)
.hasMessage("Unexpected concurrent update failure")
.getCause()
.hasMessage("Hive update is only supported for ACID transactional tables");
}

@Override
public void testExplainAnalyzeWithDeleteWithSubquery()
{
Expand Down
5 changes: 5 additions & 0 deletions testing/trino-testing/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,11 @@
<artifactId>tpch</artifactId>
</dependency>

<dependency>
<groupId>io.airlift</groupId>
<artifactId>concurrent</artifactId>
</dependency>

<dependency>
<groupId>io.airlift</groupId>
<artifactId>log</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why trinoException != e?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

because trinoException == e implies that attaching e as suppressed is kind of pointless.
verifyConcurrentUpdateFailurePermissible should be rethrowing trinoException

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()
{
Expand Down