Browse Source

HADOOP-17311. ABFS: Logs should redact SAS signature (#2422)

Contributed by bilaharith.

Change-Id: Iff0ed4303ac5ce41b62bfda8150ee983dafa40be
bilaharith 4 years ago
parent
commit
b8454a4b10

+ 2 - 2
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java

@@ -87,7 +87,7 @@ public class AbfsRestOperationException extends AzureBlobFileSystemException {
               "Operation failed: \"%1$s\", %2$s, HEAD, %3$s",
               abfsHttpOperation.getStatusDescription(),
               abfsHttpOperation.getStatusCode(),
-              abfsHttpOperation.getUrl().toString());
+              abfsHttpOperation.getSignatureMaskedUrl());
     }
 
     return String.format(
@@ -95,7 +95,7 @@ public class AbfsRestOperationException extends AzureBlobFileSystemException {
             abfsHttpOperation.getStatusDescription(),
             abfsHttpOperation.getStatusCode(),
             abfsHttpOperation.getMethod(),
-            abfsHttpOperation.getUrl().toString(),
+            abfsHttpOperation.getSignatureMaskedUrl(),
             abfsHttpOperation.getStorageErrorCode(),
             // Remove break line to ensure the request id and timestamp can be shown in console.
             abfsHttpOperation.getStorageErrorMessage().replaceAll("\\n", " "));

+ 46 - 12
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java

@@ -51,6 +51,8 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
 public class AbfsHttpOperation implements AbfsPerfLoggable {
   private static final Logger LOG = LoggerFactory.getLogger(AbfsHttpOperation.class);
 
+  public static final String SIGNATURE_QUERY_PARAM_KEY = "sig=";
+
   private static final int CONNECT_TIMEOUT = 30 * 1000;
   private static final int READ_TIMEOUT = 30 * 1000;
 
@@ -61,6 +63,8 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
 
   private final String method;
   private final URL url;
+  private String maskedUrl;
+  private String maskedEncodedUrl;
 
   private HttpURLConnection connection;
   private int statusCode;
@@ -103,8 +107,8 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
     return method;
   }
 
-  public URL getUrl() {
-    return url;
+  public String getHost() {
+    return url.getHost();
   }
 
   public int getStatusCode() {
@@ -154,7 +158,6 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
   // Returns a trace message for the request
   @Override
   public String toString() {
-    final String urlStr = url.toString();
     final StringBuilder sb = new StringBuilder();
     sb.append(statusCode);
     sb.append(",");
@@ -180,19 +183,12 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
     sb.append(",");
     sb.append(method);
     sb.append(",");
-    sb.append(urlStr);
+    sb.append(getSignatureMaskedUrl());
     return sb.toString();
   }
 
   // Returns a trace message for the ABFS API logging service to consume
   public String getLogString() {
-    String urlStr = null;
-
-    try {
-      urlStr = URLEncoder.encode(url.toString(), "UTF-8");
-    } catch(UnsupportedEncodingException e) {
-      urlStr = "https%3A%2F%2Ffailed%2Fto%2Fencode%2Furl";
-    }
 
     final StringBuilder sb = new StringBuilder();
     sb.append("s=")
@@ -220,7 +216,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
       .append(" m=")
       .append(method)
       .append(" u=")
-      .append(urlStr);
+      .append(getSignatureMaskedEncodedUrl());
 
     return sb.toString();
   }
@@ -513,4 +509,42 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
   private boolean isNullInputStream(InputStream stream) {
     return stream == null ? true : false;
   }
+
+  public static String getSignatureMaskedUrl(String url) {
+    int qpStrIdx = url.indexOf('?' + SIGNATURE_QUERY_PARAM_KEY);
+    if (qpStrIdx == -1) {
+      qpStrIdx = url.indexOf('&' + SIGNATURE_QUERY_PARAM_KEY);
+    }
+    if (qpStrIdx == -1) {
+      return url;
+    }
+    final int sigStartIdx = qpStrIdx + SIGNATURE_QUERY_PARAM_KEY.length() + 1;
+    final int ampIdx = url.indexOf("&", sigStartIdx);
+    final int sigEndIndex = (ampIdx != -1) ? ampIdx : url.length();
+    String signature = url.substring(sigStartIdx, sigEndIndex);
+    return url.replace(signature, "XXXX");
+  }
+
+  public static String encodedUrlStr(String url) {
+    try {
+      return URLEncoder.encode(url, "UTF-8");
+    } catch (UnsupportedEncodingException e) {
+      return "https%3A%2F%2Ffailed%2Fto%2Fencode%2Furl";
+    }
+  }
+
+  public String getSignatureMaskedUrl() {
+    if (this.maskedUrl == null) {
+      this.maskedUrl = getSignatureMaskedUrl(this.url.toString());
+    }
+    return this.maskedUrl;
+  }
+
+  public String getSignatureMaskedEncodedUrl() {
+    if (this.maskedEncodedUrl == null) {
+      this.maskedEncodedUrl = encodedUrlStr(getSignatureMaskedUrl());
+    }
+    return this.maskedEncodedUrl;
+  }
+
 }

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

@@ -58,6 +58,9 @@ public final class AbfsIoUtils {
         if (key.contains("Cookie")) {
           values = "*cookie info*";
         }
+        if (key.equals("sig")) {
+          values = "XXXX";
+        }
         LOG.debug("  {}={}",
             key,
             values);

+ 11 - 4
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java

@@ -254,11 +254,18 @@ public class AbfsRestOperation {
         incrementCounter(AbfsStatistic.BYTES_RECEIVED,
             httpOperation.getBytesReceived());
       }
-    } catch (IOException ex) {
-      if (ex instanceof UnknownHostException) {
-        LOG.warn(String.format("Unknown host name: %s. Retrying to resolve the host name...", httpOperation.getUrl().getHost()));
+    } catch (UnknownHostException ex) {
+      String hostname = null;
+      if (httpOperation != null) {
+        hostname = httpOperation.getHost();
       }
-
+      LOG.warn("Unknown host name: %s. Retrying to resolve the host name...",
+          hostname);
+      if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
+        throw new InvalidAbfsRestOperationException(ex);
+      }
+      return false;
+    } catch (IOException ex) {
       if (LOG.isDebugEnabled()) {
         if (httpOperation != null) {
           LOG.debug("HttpRequestFailure: " + httpOperation.toString(), ex);

+ 30 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java

@@ -25,7 +25,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.UUID;
 
-import org.junit.Assert;
+import org.assertj.core.api.Assertions;
 import org.junit.Assume;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -39,6 +39,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.extensions.MockDelegationSASTokenProvider;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.apache.hadoop.fs.azurebfs.services.AuthType;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntryScope;
@@ -53,6 +55,7 @@ import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
 import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT;
 import static org.apache.hadoop.fs.permission.AclEntryType.GROUP;
 import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test Perform Authorization Check operation
@@ -381,4 +384,30 @@ public class ITestAzureBlobFileSystemDelegationSAS extends AbstractAbfsIntegrati
 
     assertArrayEquals(propertyValue, fs.getXAttr(reqPath, propertyName));
   }
+
+  @Test
+  public void testSignatureMask() throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    String src = "/testABC/test.xt";
+    fs.create(new Path(src));
+    AbfsRestOperation abfsHttpRestOperation = fs.getAbfsClient()
+        .renamePath(src, "/testABC" + "/abc.txt", null);
+    AbfsHttpOperation result = abfsHttpRestOperation.getResult();
+    String url = result.getSignatureMaskedUrl();
+    String encodedUrl = result.getSignatureMaskedEncodedUrl();
+    Assertions.assertThat(url.substring(url.indexOf("sig=")))
+        .describedAs("Signature query param should be masked")
+        .startsWith("sig=XXXX");
+    Assertions.assertThat(encodedUrl.substring(encodedUrl.indexOf("sig%3D")))
+        .describedAs("Signature query param should be masked")
+        .startsWith("sig%3DXXXX");
+  }
+
+  @Test
+  public void testSignatureMaskOnExceptionMessage() throws Exception {
+    intercept(IOException.class, "sig=XXXX",
+        () -> getFileSystem().getAbfsClient()
+            .renamePath("testABC/test.xt", "testABC/abc.txt", null));
+  }
+
 }

+ 95 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java

@@ -0,0 +1,95 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 java.io.UnsupportedEncodingException;
+import java.net.MalformedURLException;
+import java.net.URLEncoder;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+public class TestAbfsHttpOperation {
+
+  @Test
+  public void testMaskingAndEncoding()
+      throws MalformedURLException, UnsupportedEncodingException {
+    testIfMaskAndEncodeSuccessful("Where sig is the only query param",
+        "http://www.testurl.net?sig=abcd", "http://www.testurl.net?sig=XXXX");
+
+    testIfMaskAndEncodeSuccessful("Where sig is the first query param",
+        "http://www.testurl.net?sig=abcd&abc=xyz",
+        "http://www.testurl.net?sig=XXXX&abc=xyz");
+
+    testIfMaskAndEncodeSuccessful(
+        "Where sig is neither first nor last query param",
+        "http://www.testurl.net?lmn=abc&sig=abcd&abc=xyz",
+        "http://www.testurl.net?lmn=abc&sig=XXXX&abc=xyz");
+
+    testIfMaskAndEncodeSuccessful("Where sig is the last query param",
+        "http://www.testurl.net?abc=xyz&sig=abcd",
+        "http://www.testurl.net?abc=xyz&sig=XXXX");
+
+    testIfMaskAndEncodeSuccessful("Where sig query param is not present",
+        "http://www.testurl.net?abc=xyz", "http://www.testurl.net?abc=xyz");
+
+    testIfMaskAndEncodeSuccessful(
+        "Where sig query param is not present but mysig",
+        "http://www.testurl.net?abc=xyz&mysig=qwerty",
+        "http://www.testurl.net?abc=xyz&mysig=qwerty");
+
+    testIfMaskAndEncodeSuccessful(
+        "Where sig query param is not present but sigmy",
+        "http://www.testurl.net?abc=xyz&sigmy=qwerty",
+        "http://www.testurl.net?abc=xyz&sigmy=qwerty");
+
+    testIfMaskAndEncodeSuccessful(
+        "Where sig query param is not present but a " + "value sig",
+        "http://www.testurl.net?abc=xyz&mnop=sig",
+        "http://www.testurl.net?abc=xyz&mnop=sig");
+
+    testIfMaskAndEncodeSuccessful(
+        "Where sig query param is not present but a " + "value ends with sig",
+        "http://www.testurl.net?abc=xyz&mnop=abcsig",
+        "http://www.testurl.net?abc=xyz&mnop=abcsig");
+
+    testIfMaskAndEncodeSuccessful(
+        "Where sig query param is not present but a " + "value starts with sig",
+        "http://www.testurl.net?abc=xyz&mnop=sigabc",
+        "http://www.testurl.net?abc=xyz&mnop=sigabc");
+  }
+
+  private void testIfMaskAndEncodeSuccessful(final String scenario,
+      final String url, final String expectedMaskedUrl)
+      throws UnsupportedEncodingException {
+
+    Assertions.assertThat(AbfsHttpOperation.getSignatureMaskedUrl(url))
+        .describedAs(url + " (" + scenario + ") after masking should be: "
+            + expectedMaskedUrl).isEqualTo(expectedMaskedUrl);
+
+    final String expectedMaskedEncodedUrl = URLEncoder
+        .encode(expectedMaskedUrl, "UTF-8");
+    Assertions.assertThat(AbfsHttpOperation.encodedUrlStr(expectedMaskedUrl))
+        .describedAs(
+            url + " (" + scenario + ") after masking and encoding should "
+                + "be: " + expectedMaskedEncodedUrl)
+        .isEqualTo(expectedMaskedEncodedUrl);
+  }
+
+}