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

Fix race in AzureClient factory fetch #16525

Merged
merged 3 commits into from
Jun 1, 2024
Merged
Show file tree
Hide file tree
Changes from 2 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 @@ -32,8 +32,8 @@

import javax.annotation.Nullable;
import java.time.Duration;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;

/**
* Factory class for generating BlobServiceClient objects used for deep storage.
Expand All @@ -47,7 +47,7 @@ public class AzureClientFactory
public AzureClientFactory(AzureAccountConfig config)
{
this.config = config;
this.cachedBlobServiceClients = new HashMap<>();
this.cachedBlobServiceClients = new ConcurrentHashMap<>();
}

// It's okay to store clients in a map here because all the configs for specifying azure retries are static, and there are only 2 of them.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,11 +24,16 @@
import com.azure.storage.blob.BlobServiceClient;
import com.azure.storage.common.StorageSharedKeyCredential;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.junit.Assert;
import org.junit.Test;

import java.net.MalformedURLException;
import java.net.URL;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;

public class AzureClientFactoryTest
{
Expand Down Expand Up @@ -173,4 +178,51 @@ public void test_blobServiceClientBuilder_useAzureAccountConfigWithStorageAccoun
BlobServiceClient blobServiceClient = azureClientFactory.getBlobServiceClient(null, ACCOUNT);
Assert.assertEquals(expectedAccountUrl.toString(), blobServiceClient.getAccountUrl());
}

@Test
public void test_concurrent_azureClientFactory_gets() throws Exception
{
for (int i = 0; i < 10; i++) {
concurrentAzureClientFactoryGets();
}
}

private void concurrentAzureClientFactoryGets() throws Exception
{
final int threads = 100;
String endpointSuffix = "core.nonDefault.windows.net";
String storageAccountEndpointSuffix = "ABC123.blob.storage.azure.net";
AzureAccountConfig config = new AzureAccountConfig();
config.setKey("key");
config.setEndpointSuffix(endpointSuffix);
config.setStorageAccountEndpointSuffix(storageAccountEndpointSuffix);
final AzureClientFactory localAzureClientFactory = new AzureClientFactory(config);
final URL expectedAccountUrl = new URL(
AzureAccountConfig.DEFAULT_PROTOCOL,
ACCOUNT + "." + storageAccountEndpointSuffix,
""
);

final CountDownLatch latch = new CountDownLatch(threads);
ExecutorService executorService = Execs.multiThreaded(threads, "azure-client-fetcher-%d");
final AtomicReference<Exception> failureExecption = new AtomicReference<>();
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
final AtomicReference<Exception> failureExecption = new AtomicReference<>();
final AtomicReference<Exception> failureException = new AtomicReference<>();

for (int i = 0; i < threads; i++) {
final int retry = i % 2;
executorService.submit(() -> {
try {
latch.countDown();
latch.await();
BlobServiceClient blobServiceClient = localAzureClientFactory.getBlobServiceClient(retry, ACCOUNT);
Assert.assertEquals(expectedAccountUrl.toString(), blobServiceClient.getAccountUrl());
}
catch (Exception e) {
failureExecption.compareAndSet(null, e);
}
});
}
executorService.awaitTermination(1000, TimeUnit.MICROSECONDS);
if (failureExecption.get() != null) {
throw failureExecption.get();
}
}
}
Loading