Skip to content

Commit

Permalink
Fix flaky test
Browse files Browse the repository at this point in the history
  • Loading branch information
sopel39 committed Jun 11, 2021
1 parent 41ff64c commit 43fe897
Showing 1 changed file with 4 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,6 @@
import static io.trino.spi.type.BigintType.BIGINT;
import static io.trino.testing.TestingSession.testSessionBuilder;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;

@Test(singleThreaded = true)
Expand All @@ -55,7 +54,7 @@ public class TestRefreshMaterializedView
private ListeningExecutorService executorService;
private SettableFuture<?> startRefreshMaterializedView;
private SettableFuture<?> finishRefreshMaterializedView;
private boolean refreshInterrupted;
private SettableFuture<?> refreshInterrupted;

@BeforeClass
public void setUp()
Expand All @@ -74,7 +73,7 @@ public void resetState()
{
startRefreshMaterializedView = SettableFuture.create();
finishRefreshMaterializedView = SettableFuture.create();
refreshInterrupted = false;
refreshInterrupted = SettableFuture.create();
}

@Override
Expand Down Expand Up @@ -109,7 +108,7 @@ protected QueryRunner createQueryRunner()
startRefreshMaterializedView.set(null);
SettableFuture<?> refreshMaterializedView = SettableFuture.create();
finishRefreshMaterializedView.addListener(() -> refreshMaterializedView.set(null), directExecutor());
addExceptionCallback(refreshMaterializedView, () -> refreshInterrupted = true);
addExceptionCallback(refreshMaterializedView, () -> refreshInterrupted.set(null));
return toCompletableFuture(refreshMaterializedView);
}))
.build()));
Expand Down Expand Up @@ -144,7 +143,7 @@ public void testDelegateRefreshMaterializedViewToConnectorWithCancellation()

assertThatThrownBy(() -> getFutureValue(queryFuture))
.hasMessage("Query was canceled");
assertThat(refreshInterrupted).isTrue();
getFutureValue(refreshInterrupted);
}

private ListenableFuture<?> assertUpdateAsync(@Language("SQL") String sql)
Expand Down

0 comments on commit 43fe897

Please sign in to comment.