Skip to content
Merged
Show file tree
Hide file tree
Changes from 12 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 @@ -131,6 +131,7 @@
public abstract class AbfsClient implements Closeable {
public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
public static final String HUNDRED_CONTINUE_USER_AGENT = SINGLE_WHITE_SPACE + HUNDRED_CONTINUE + SEMICOLON;
public static final String ABFS_CLIENT_TIMER_THREAD_NAME = "abfs-timer-client";

private final URL baseUrl;
private final SharedKeyCredentials sharedKeyCredentials;
Expand All @@ -149,7 +150,7 @@ public abstract class AbfsClient implements Closeable {
private AccessTokenProvider tokenProvider;
private SASTokenProvider sasTokenProvider;
private final AbfsCounters abfsCounters;
private final Timer timer;
private Timer timer;
private final String abfsMetricUrl;
private boolean isMetricCollectionEnabled = false;
private final MetricFormat metricFormat;
Expand Down Expand Up @@ -258,9 +259,9 @@ private AbfsClient(final URL baseUrl,
throw new IOException("Exception while initializing metric credentials " + e);
}
}
this.timer = new Timer(
"abfs-timer-client", true);
if (isMetricCollectionEnabled) {
this.timer = new Timer(
ABFS_CLIENT_TIMER_THREAD_NAME, true);
timer.schedule(new TimerTaskImpl(),
metricIdlePeriod,
metricIdlePeriod);
Expand Down Expand Up @@ -292,9 +293,9 @@ public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredent

@Override
public void close() throws IOException {
if (runningTimerTask != null) {
if (runningTimerTask != null && isMetricCollectionEnabled) {
runningTimerTask.cancel();
timer.purge();
timer.cancel();
}
if (keepAliveCache != null) {
keepAliveCache.close();
Expand Down Expand Up @@ -1418,7 +1419,7 @@ private TracingContext getMetricTracingContext() {
boolean timerOrchestrator(TimerFunctionality timerFunctionality, TimerTask timerTask) {
switch (timerFunctionality) {
case RESUME:
if (isMetricCollectionStopped.get()) {
if (isMetricCollectionEnabled) {
synchronized (this) {
if (isMetricCollectionStopped.get()) {
resumeTimer();
Expand Down Expand Up @@ -1597,6 +1598,11 @@ KeepAliveCache getKeepAliveCache() {
return keepAliveCache;
}

@VisibleForTesting
protected Timer getTimer() {
return timer;
}

protected String getUserAgent() {
return userAgent;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,118 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.hadoop.fs.azurebfs.services;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
import org.apache.hadoop.fs.azurebfs.AbfsCountersImpl;
import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
import org.apache.hadoop.fs.azurebfs.utils.Base64;
import org.apache.hadoop.fs.azurebfs.utils.MetricFormat;
import org.assertj.core.api.Assertions;
import org.junit.Test;
import org.mockito.Mockito;

import java.net.URI;
import java.net.URL;
import java.util.Map;

import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_ACCOUNT_KEY;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_ACCOUNT_NAME;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_METRIC_FORMAT;
import static org.apache.hadoop.fs.azurebfs.services.AbfsClient.ABFS_CLIENT_TIMER_THREAD_NAME;

public class TestAbfsClient {
Copy link
Contributor

Choose a reason for hiding this comment

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

Add a little Java explaining what it is trying to test.

private static final String ACCOUNT_NAME = "bogusAccountName.dfs.core.windows.net";
private static final String ACCOUNT_KEY = "testKey";

@Test
public void testTimerNotInitialize() throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

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

  1. Add a comment stating that this verifies that's a timer is only created when metrics are being collected.
  2. Change the title of the method to something like testTimerNotInitializedWithoutMetricCollection

final Configuration configuration = new Configuration();
AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, ACCOUNT_NAME);

AbfsCounters abfsCounters = Mockito.spy(new AbfsCountersImpl(new URI("abcd")));
AbfsClientContext abfsClientContext = new AbfsClientContextBuilder().withAbfsCounters(abfsCounters).build();

// Get an instance of AbfsClient.
AbfsClient client = new AbfsDfsClient(new URL("https://azure.com"),
null,
abfsConfiguration,
(AccessTokenProvider) null,
null,
abfsClientContext);

Assertions.assertThat(client.getTimer())
.describedAs("Timer should not be initialized")
.isNull();

// Check if a thread with the name "abfs-timer-client" exists
Assertions.assertThat(isThreadRunning(ABFS_CLIENT_TIMER_THREAD_NAME))
.describedAs("Expected thread 'abfs-timer-client' not found")
.isEqualTo(false);
client.close();
}

@Test
public void testTimerInitialize() throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

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

  1. Change the title of the method to something like testTimerIsInitializedWithMetricCollection

final Configuration configuration = new Configuration();
configuration.set(FS_AZURE_METRIC_FORMAT, String.valueOf(MetricFormat.INTERNAL_BACKOFF_METRIC_FORMAT));
configuration.set(FS_AZURE_METRIC_ACCOUNT_NAME, ACCOUNT_NAME);
configuration.set(FS_AZURE_METRIC_ACCOUNT_KEY, Base64.encode(ACCOUNT_KEY.getBytes()));
AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, ACCOUNT_NAME);

AbfsCounters abfsCounters = Mockito.spy(new AbfsCountersImpl(new URI("abcd")));
AbfsClientContext abfsClientContext = new AbfsClientContextBuilder().withAbfsCounters(abfsCounters).build();

// Get an instance of AbfsClient.
AbfsClient client = new AbfsDfsClient(new URL("https://azure.com"),
null,
abfsConfiguration,
(AccessTokenProvider) null,
null,
abfsClientContext);

Assertions.assertThat(client.getTimer())
.describedAs("Timer should be initialized")
.isNotNull();

// Check if a thread with the name "abfs-timer-client" exists
Assertions.assertThat(isThreadRunning(ABFS_CLIENT_TIMER_THREAD_NAME))
.describedAs("Expected thread 'abfs-timer-client' not found")
.isEqualTo(true);
client.close();

// Check if the thread is removed after closing the client
Assertions.assertThat(isThreadRunning("abfs-timer-client"))
.describedAs("Unexpected thread 'abfs-timer-client' found")
.isEqualTo(false);
}

private boolean isThreadRunning(String threadName) {
// Get all threads and their stack traces
Map<Thread, StackTraceElement[]> allThreads = Thread.getAllStackTraces();

// Check if any thread has the specified name
for (Thread thread : allThreads.keySet()) {
if (thread.getName().equals(threadName)) {
return true;
}
}
return false;
}
}