Browse Source

HADOOP-16922. ABFS: Change User-Agent header (#1938)

Contributed by Bilahari T H.
bilaharith 5 years ago
parent
commit
f53ded6185

+ 17 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java

@@ -168,9 +168,17 @@ public class AbfsConfiguration{
   private boolean enableAutoThrottling;
 
   @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_USER_AGENT_PREFIX_KEY,
-      DefaultValue = "")
+      DefaultValue = DEFAULT_FS_AZURE_USER_AGENT_PREFIX)
   private String userAgentId;
 
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_CLUSTER_NAME,
+      DefaultValue = DEFAULT_VALUE_UNKNOWN)
+  private String clusterName;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_CLUSTER_TYPE,
+      DefaultValue = DEFAULT_VALUE_UNKNOWN)
+  private String clusterType;
+
   @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_DELEGATION_TOKEN,
       DefaultValue = DEFAULT_ENABLE_DELEGATION_TOKEN)
   private boolean enableDelegationToken;
@@ -479,6 +487,14 @@ public class AbfsConfiguration{
     return this.userAgentId;
   }
 
+  public String getClusterName() {
+    return this.clusterName;
+  }
+
+  public String getClusterType() {
+    return this.clusterType;
+  }
+
   public DelegatingSSLSocketFactory.SSLChannelMode getPreferredSSLFactoryOption() {
     return getEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DEFAULT_FS_AZURE_SSL_CHANNEL_MODE);
   }

+ 3 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java

@@ -42,10 +42,13 @@ public final class AbfsHttpConstants {
   public static final String DEFAULT_TIMEOUT = "90";
   public static final String TOKEN_VERSION = "2";
 
+  public static final String JAVA_VENDOR = "java.vendor";
   public static final String JAVA_VERSION = "java.version";
   public static final String OS_NAME = "os.name";
   public static final String OS_VERSION = "os.version";
+  public static final String OS_ARCH = "os.arch";
 
+  public static final String APN_VERSION = "APN/1.0";
   public static final String CLIENT_VERSION = "Azure Blob FS/" + VersionInfo.getVersion();
 
   // Abfs Http Verb

+ 2 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java

@@ -62,6 +62,8 @@ public final class ConfigurationKeys {
    *  Default value of this config is true. **/
   public static final String FS_AZURE_DISABLE_OUTPUTSTREAM_FLUSH = "fs.azure.disable.outputstream.flush";
   public static final String FS_AZURE_USER_AGENT_PREFIX_KEY = "fs.azure.user.agent.prefix";
+  public static final String FS_AZURE_CLUSTER_NAME = "fs.azure.cluster.name";
+  public static final String FS_AZURE_CLUSTER_TYPE = "fs.azure.cluster.type";
   public static final String FS_AZURE_SSL_CHANNEL_MODE_KEY = "fs.azure.ssl.channel.mode";
   /** Provides a config to enable/disable the checkAccess API.
    *  By default this will be

+ 5 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java

@@ -22,6 +22,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
 
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
+
 /**
  * Responsible to keep all the Azure Blob File System related configurations.
  */
@@ -71,5 +73,8 @@ public final class FileSystemConfigurations {
   public static final boolean DEFAULT_ENABLE_CHECK_ACCESS = false;
   public static final boolean DEFAULT_ABFS_LATENCY_TRACK = false;
 
+  public static final String DEFAULT_FS_AZURE_USER_AGENT_PREFIX = EMPTY_STRING;
+  public static final String DEFAULT_VALUE_UNKNOWN = "UNKNOWN";
+
   private FileSystemConfigurations() {}
 }

+ 49 - 21
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java

@@ -679,32 +679,60 @@ public class AbfsClient implements Closeable {
 
   @VisibleForTesting
   String initializeUserAgent(final AbfsConfiguration abfsConfiguration,
-                             final String sslProviderName) {
+      final String sslProviderName) {
+
     StringBuilder sb = new StringBuilder();
-    sb.append("(JavaJRE ");
+
+    sb.append(APN_VERSION);
+    sb.append(SINGLE_WHITE_SPACE);
+    sb.append(CLIENT_VERSION);
+    sb.append(SINGLE_WHITE_SPACE);
+
+    sb.append("(");
+
+    sb.append(System.getProperty(JAVA_VENDOR)
+        .replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING));
+    sb.append(SINGLE_WHITE_SPACE);
+    sb.append("JavaJRE");
+    sb.append(SINGLE_WHITE_SPACE);
     sb.append(System.getProperty(JAVA_VERSION));
-    sb.append("; ");
-    sb.append(
-        System.getProperty(OS_NAME).replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING));
-    sb.append(" ");
+    sb.append(SEMICOLON);
+    sb.append(SINGLE_WHITE_SPACE);
+
+    sb.append(System.getProperty(OS_NAME)
+        .replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING));
+    sb.append(SINGLE_WHITE_SPACE);
     sb.append(System.getProperty(OS_VERSION));
-    if (sslProviderName != null && !sslProviderName.isEmpty()) {
-      sb.append("; ");
-      sb.append(sslProviderName);
-    }
-    String tokenProviderField =
-        ExtensionHelper.getUserAgentSuffix(tokenProvider, "");
-    if (!tokenProviderField.isEmpty()) {
-      sb.append("; ").append(tokenProviderField);
-    }
+    sb.append(FORWARD_SLASH);
+    sb.append(System.getProperty(OS_ARCH));
+    sb.append(SEMICOLON);
+
+    appendIfNotEmpty(sb, sslProviderName, true);
+    appendIfNotEmpty(sb,
+        ExtensionHelper.getUserAgentSuffix(tokenProvider, EMPTY_STRING), true);
+
+    sb.append(SINGLE_WHITE_SPACE);
+    sb.append(abfsConfiguration.getClusterName());
+    sb.append(FORWARD_SLASH);
+    sb.append(abfsConfiguration.getClusterType());
+
     sb.append(")");
-    final String userAgentComment = sb.toString();
-    String customUserAgentId = abfsConfiguration.getCustomUserAgentPrefix();
-    if (customUserAgentId != null && !customUserAgentId.isEmpty()) {
-      return String.format(Locale.ROOT, CLIENT_VERSION + " %s %s",
-          userAgentComment, customUserAgentId);
+
+    appendIfNotEmpty(sb, abfsConfiguration.getCustomUserAgentPrefix(), false);
+
+    return String.format(Locale.ROOT, sb.toString());
+  }
+
+  private void appendIfNotEmpty(StringBuilder sb, String regEx,
+      boolean shouldAppendSemiColon) {
+    if (regEx == null || regEx.trim().isEmpty()) {
+      return;
+    }
+    sb.append(SINGLE_WHITE_SPACE);
+    sb.append(regEx);
+    if (shouldAppendSemiColon) {
+      sb.append(SEMICOLON);
     }
-    return String.format(Locale.ROOT, CLIENT_VERSION + " %s", userAgentComment);
   }
 
   @VisibleForTesting

+ 185 - 39
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java

@@ -18,11 +18,11 @@
 
 package org.apache.hadoop.fs.azurebfs.services;
 
+import java.io.IOException;
+import java.net.MalformedURLException;
 import java.net.URL;
 import java.util.regex.Pattern;
 
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
-import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
@@ -30,7 +30,24 @@ import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
 import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
-import org.apache.hadoop.util.VersionInfo;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APN_VERSION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CLIENT_VERSION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.JAVA_VENDOR;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.JAVA_VERSION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.OS_ARCH;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.OS_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.OS_VERSION;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SEMICOLON;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SINGLE_WHITE_SPACE;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLUSTER_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLUSTER_TYPE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_VALUE_UNKNOWN;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME;
 
 /**
  * Test useragent of abfs client.
@@ -38,60 +55,189 @@ import org.apache.hadoop.util.VersionInfo;
  */
 public final class TestAbfsClient {
 
-  private final String accountName = "bogusAccountName.dfs.core.windows.net";
+  private static final String ACCOUNT_NAME = "bogusAccountName.dfs.core.windows.net";
+  private static final String FS_AZURE_USER_AGENT_PREFIX = "Partner Service";
+
+  private final Pattern userAgentStringPattern;
+
+  public TestAbfsClient(){
+    StringBuilder regEx = new StringBuilder();
+    regEx.append("^");
+    regEx.append(APN_VERSION);
+    regEx.append(SINGLE_WHITE_SPACE);
+    regEx.append(CLIENT_VERSION);
+    regEx.append(SINGLE_WHITE_SPACE);
+    regEx.append("\\(");
+    regEx.append(System.getProperty(JAVA_VENDOR)
+        .replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING));
+    regEx.append(SINGLE_WHITE_SPACE);
+    regEx.append("JavaJRE");
+    regEx.append(SINGLE_WHITE_SPACE);
+    regEx.append(System.getProperty(JAVA_VERSION));
+    regEx.append(SEMICOLON);
+    regEx.append(SINGLE_WHITE_SPACE);
+    regEx.append(System.getProperty(OS_NAME)
+        .replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING));
+    regEx.append(SINGLE_WHITE_SPACE);
+    regEx.append(System.getProperty(OS_VERSION));
+    regEx.append(FORWARD_SLASH);
+    regEx.append(System.getProperty(OS_ARCH));
+    regEx.append(SEMICOLON);
+    regEx.append("([a-zA-Z].*; )?");      // Regex for sslProviderName
+    regEx.append("([a-zA-Z].*; )?");      // Regex for tokenProvider
+    regEx.append(" ?");
+    regEx.append(".+");                   // cluster name
+    regEx.append(FORWARD_SLASH);
+    regEx.append(".+");            // cluster type
+    regEx.append("\\)");
+    regEx.append("( .*)?");        //  Regex for user agent prefix
+    regEx.append("$");
+    this.userAgentStringPattern = Pattern.compile(regEx.toString());
+  }
 
-  private void validateUserAgent(String expectedPattern,
-                                 URL baseUrl,
-                                 AbfsConfiguration config,
-                                 boolean includeSSLProvider)
-      throws AzureBlobFileSystemException {
-    AbfsClient client = new AbfsClient(baseUrl, null,
+  private String getUserAgentString(AbfsConfiguration config,
+      boolean includeSSLProvider) throws MalformedURLException {
+    AbfsClient client = new AbfsClient(new URL("https://azure.com"), null,
         config, null, (AccessTokenProvider) null, null);
     String sslProviderName = null;
     if (includeSSLProvider) {
-      sslProviderName = DelegatingSSLSocketFactory.getDefaultFactory().getProviderName();
+      sslProviderName = DelegatingSSLSocketFactory.getDefaultFactory()
+          .getProviderName();
     }
-    String userAgent = client.initializeUserAgent(config, sslProviderName);
-    Pattern pattern = Pattern.compile(expectedPattern);
-    Assert.assertTrue("Incorrect User Agent String",
-        pattern.matcher(userAgent).matches());
+    return client.initializeUserAgent(config, sslProviderName);
   }
 
   @Test
-  public void verifyUnknownUserAgent() throws Exception {
-    String clientVersion = "Azure Blob FS/" + VersionInfo.getVersion();
-    String expectedUserAgentPattern = String.format(clientVersion
-        + " %s", "\\(JavaJRE ([^\\)]+)\\)");
+  public void verifybBasicInfo() throws Exception {
     final Configuration configuration = new Configuration();
-    configuration.unset(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY);
-    AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, accountName);
-    validateUserAgent(expectedUserAgentPattern, new URL("http://azure.com"),
-        abfsConfiguration, false);
+    configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
+    AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration,
+        ACCOUNT_NAME);
+    verifybBasicInfo(getUserAgentString(abfsConfiguration, false));
+  }
+
+  private void verifybBasicInfo(String userAgentStr) {
+    assertThat(userAgentStr)
+        .describedAs("User-Agent string [" + userAgentStr
+            + "] should be of the pattern: " + this.userAgentStringPattern.pattern())
+        .matches(this.userAgentStringPattern)
+        .describedAs("User-Agent string should contain java vendor")
+        .contains(System.getProperty(JAVA_VENDOR)
+            .replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING))
+        .describedAs("User-Agent string should contain java version")
+        .contains(System.getProperty(JAVA_VERSION))
+        .describedAs("User-Agent string should contain  OS name")
+        .contains(System.getProperty(OS_NAME)
+            .replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING))
+        .describedAs("User-Agent string should contain OS version")
+        .contains(System.getProperty(OS_VERSION))
+        .describedAs("User-Agent string should contain OS arch")
+        .contains(System.getProperty(OS_ARCH));
   }
 
   @Test
-  public void verifyUserAgent() throws Exception {
-    String clientVersion = "Azure Blob FS/" + VersionInfo.getVersion();
-    String expectedUserAgentPattern = String.format(clientVersion
-        + " %s", "\\(JavaJRE ([^\\)]+)\\) Partner Service");
+  public void verifyUserAgentPrefix()
+      throws IOException, IllegalAccessException {
     final Configuration configuration = new Configuration();
-    configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, "Partner Service");
-    AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, accountName);
-    validateUserAgent(expectedUserAgentPattern, new URL("http://azure.com"),
-        abfsConfiguration, false);
+    configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
+    configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, FS_AZURE_USER_AGENT_PREFIX);
+    AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration,
+        ACCOUNT_NAME);
+    String userAgentStr = getUserAgentString(abfsConfiguration, false);
+
+    verifybBasicInfo(userAgentStr);
+    assertThat(userAgentStr)
+      .describedAs("User-Agent string should contain " + FS_AZURE_USER_AGENT_PREFIX)
+      .contains(FS_AZURE_USER_AGENT_PREFIX);
+
+    configuration.unset(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY);
+    abfsConfiguration = new AbfsConfiguration(configuration,
+        ACCOUNT_NAME);
+    userAgentStr = getUserAgentString(abfsConfiguration, false);
+
+    verifybBasicInfo(userAgentStr);
+    assertThat(userAgentStr)
+      .describedAs("User-Agent string should not contain " + FS_AZURE_USER_AGENT_PREFIX)
+      .doesNotContain(FS_AZURE_USER_AGENT_PREFIX);
   }
 
   @Test
-  public void verifyUserAgentWithSSLProvider() throws Exception {
-    String clientVersion = "Azure Blob FS/" + VersionInfo.getVersion();
-    String expectedUserAgentPattern = String.format(clientVersion
-        + " %s", "\\(JavaJRE ([^\\)]+)\\) Partner Service");
+  public void verifyUserAgentWithoutSSLProvider() throws Exception {
     final Configuration configuration = new Configuration();
-    configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, "Partner Service");
+    configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
     configuration.set(ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY,
         DelegatingSSLSocketFactory.SSLChannelMode.Default_JSSE.name());
-    AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, accountName);
-    validateUserAgent(expectedUserAgentPattern, new URL("https://azure.com"),
-        abfsConfiguration, true);
+    AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration,
+        ACCOUNT_NAME);
+    String userAgentStr = getUserAgentString(abfsConfiguration, true);
+
+    verifybBasicInfo(userAgentStr);
+    assertThat(userAgentStr)
+      .describedAs("User-Agent string should contain sslProvider")
+      .contains(DelegatingSSLSocketFactory.getDefaultFactory().getProviderName());
+
+    userAgentStr = getUserAgentString(abfsConfiguration, false);
+
+    verifybBasicInfo(userAgentStr);
+    assertThat(userAgentStr)
+      .describedAs("User-Agent string should not contain sslProvider")
+      .doesNotContain(DelegatingSSLSocketFactory.getDefaultFactory().getProviderName());
   }
+
+  @Test
+  public void verifyUserAgentClusterName() throws Exception {
+    final String clusterName = "testClusterName";
+    final Configuration configuration = new Configuration();
+    configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
+    configuration.set(FS_AZURE_CLUSTER_NAME, clusterName);
+    AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration,
+        ACCOUNT_NAME);
+    String userAgentStr = getUserAgentString(abfsConfiguration, false);
+
+    verifybBasicInfo(userAgentStr);
+    assertThat(userAgentStr)
+      .describedAs("User-Agent string should contain cluster name")
+      .contains(clusterName);
+
+    configuration.unset(FS_AZURE_CLUSTER_NAME);
+    abfsConfiguration = new AbfsConfiguration(configuration,
+        ACCOUNT_NAME);
+    userAgentStr = getUserAgentString(abfsConfiguration, false);
+
+    verifybBasicInfo(userAgentStr);
+    assertThat(userAgentStr)
+      .describedAs("User-Agent string should not contain cluster name")
+      .doesNotContain(clusterName)
+      .describedAs("User-Agent string should contain UNKNOWN as cluster name config is absent")
+      .contains(DEFAULT_VALUE_UNKNOWN);
+  }
+
+  @Test
+  public void verifyUserAgentClusterType() throws Exception {
+    final String clusterType = "testClusterType";
+    final Configuration configuration = new Configuration();
+    configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
+    configuration.set(FS_AZURE_CLUSTER_TYPE, clusterType);
+    AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration,
+        ACCOUNT_NAME);
+    String userAgentStr = getUserAgentString(abfsConfiguration, false);
+
+    verifybBasicInfo(userAgentStr);
+    assertThat(userAgentStr)
+      .describedAs("User-Agent string should contain cluster type")
+      .contains(clusterType);
+
+    configuration.unset(FS_AZURE_CLUSTER_TYPE);
+    abfsConfiguration = new AbfsConfiguration(configuration,
+        ACCOUNT_NAME);
+    userAgentStr = getUserAgentString(abfsConfiguration, false);
+
+    verifybBasicInfo(userAgentStr);
+    assertThat(userAgentStr)
+      .describedAs("User-Agent string should not contain cluster type")
+      .doesNotContain(clusterType)
+      .describedAs("User-Agent string should contain UNKNOWN as cluster type config is absent")
+      .contains(DEFAULT_VALUE_UNKNOWN);
+  }
+
 }