Skip to content
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

Avoid test cleanup errors in Raptor #12802

Merged
merged 2 commits into from
Sep 12, 2024
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 @@ -36,12 +36,14 @@
import java.util.concurrent.atomic.AtomicInteger;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.util.concurrent.MoreExecutors.shutdownAndAwaitTermination;
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static io.trino.plugin.raptor.legacy.RaptorErrorCode.RAPTOR_BACKUP_CORRUPTION;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.CompletableFuture.completedFuture;
import static java.util.concurrent.CompletableFuture.runAsync;
import static java.util.concurrent.Executors.newFixedThreadPool;
import static java.util.concurrent.TimeUnit.SECONDS;

public class BackupManager
{
Expand Down Expand Up @@ -72,7 +74,7 @@ public BackupManager(Optional<BackupStore> backupStore, StorageService storageSe
@PreDestroy
public void shutdown()
{
executorService.shutdownNow();
shutdownAndAwaitTermination(executorService, 10, SECONDS);
}

public CompletableFuture<Void> submit(UUID uuid, File source)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ public void configure(Binder binder)
@ForMetadata
public static ConnectionFactory createConnectionFactory(H2DatabaseConfig config)
{
String url = format("jdbc:h2:%s;DB_CLOSE_DELAY=-1", config.getFilename());
String url = format("jdbc:h2:retry:%s", config.getFilename());
return () -> DriverManager.getConnection(url);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@

import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
import static com.google.common.util.concurrent.MoreExecutors.shutdownAndAwaitTermination;
import static io.airlift.concurrent.MoreFutures.addExceptionCallback;
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static io.airlift.units.DataSize.succinctBytes;
Expand Down Expand Up @@ -136,8 +137,8 @@ public void start()
@PreDestroy
public void shutdown()
{
executorService.shutdownNow();
missingShardExecutor.shutdownNow();
shutdownAndAwaitTermination(missingShardExecutor, 10, SECONDS);
shutdownAndAwaitTermination(executorService, 10, SECONDS);
}

private void scheduleRecoverMissingShards()
Expand Down Expand Up @@ -171,7 +172,9 @@ private synchronized void enqueueMissingShards()
}
}
catch (Throwable t) {
log.error(t, "Error creating shard recovery tasks");
if (!executorService.isShutdown()) {
log.error(t, "Error creating shard recovery tasks");
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,14 +28,15 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.atomic.AtomicBoolean;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.collect.Sets.newConcurrentHashSet;
import static com.google.common.util.concurrent.MoreExecutors.shutdownAndAwaitTermination;
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.CompletableFuture.runAsync;
import static java.util.concurrent.Executors.newFixedThreadPool;
import static java.util.concurrent.TimeUnit.SECONDS;

public class ShardOrganizer
{
Expand All @@ -44,8 +45,6 @@ public class ShardOrganizer
private final ExecutorService executorService;
private final ThreadPoolExecutorMBean executorMBean;

private final AtomicBoolean shutdown = new AtomicBoolean();

// Tracks shards that are scheduled for compaction so that we do not schedule them more than once
private final Set<UUID> shardsInProgress = newConcurrentHashSet();
private final JobFactory jobFactory;
Expand All @@ -69,9 +68,7 @@ public ShardOrganizer(JobFactory jobFactory, int threads)
@PreDestroy
public void shutdown()
{
if (!shutdown.getAndSet(true)) {
executorService.shutdownNow();
}
shutdownAndAwaitTermination(executorService, 10, SECONDS);
}

public CompletableFuture<?> enqueue(OrganizationSet organizationSet)
Expand Down