You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by fa...@apache.org on 2018/05/09 02:00:09 UTC

hadoop git commit: HADOOP-15420 s3guard ITestS3GuardToolLocal failures in diff tests (Gabor Bota)

Repository: hadoop
Updated Branches:
  refs/heads/trunk 69aac696d -> 8981674bb


HADOOP-15420 s3guard ITestS3GuardToolLocal failures in diff tests (Gabor Bota)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8981674b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8981674b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8981674b

Branch: refs/heads/trunk
Commit: 8981674bbcff0663af820f3e87a3eaea5789968a
Parents: 69aac69
Author: Aaron Fabbri <fa...@apache.org>
Authored: Tue May 8 18:58:42 2018 -0700
Committer: Aaron Fabbri <fa...@apache.org>
Committed: Tue May 8 18:58:42 2018 -0700

----------------------------------------------------------------------
 .../fs/s3a/s3guard/LocalMetadataStore.java      | 16 ++++-
 .../s3guard/AbstractS3GuardToolTestBase.java    | 68 ++++++++++++++++++++
 .../fs/s3a/s3guard/ITestS3GuardToolLocal.java   | 68 --------------------
 3 files changed, 83 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8981674b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
index 742c41a..95689e1 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
@@ -369,10 +369,24 @@ public class LocalMetadataStore implements MetadataStore {
   }
 
   private boolean expired(FileStatus status, long expiry, String keyPrefix) {
+    // remove the protocol from path string to be able to compare
+    String bucket = status.getPath().toUri().getHost();
+    String statusTranslatedPath = "";
+    if(bucket != null && !bucket.isEmpty()){
+      // if there's a bucket, (well defined host in Uri) the pathToParentKey
+      // can be used to get the path from the status
+      statusTranslatedPath =
+          PathMetadataDynamoDBTranslation.pathToParentKey(status.getPath());
+    } else {
+      // if there's no bucket in the path the pathToParentKey will fail, so
+      // this is the fallback to get the path from status
+      statusTranslatedPath = status.getPath().toUri().getPath();
+    }
+
     // Note: S3 doesn't track modification time on directories, so for
     // consistency with the DynamoDB implementation we ignore that here
     return status.getModificationTime() < expiry && !status.isDirectory()
-      && status.getPath().toString().startsWith(keyPrefix);
+      && statusTranslatedPath.startsWith(keyPrefix);
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8981674b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
index 397d8b6..4381749 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
@@ -18,11 +18,16 @@
 
 package org.apache.hadoop.fs.s3a.s3guard;
 
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.io.PrintStream;
 import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
@@ -316,4 +321,67 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
     assertEquals("Command " + cmd + " failed\n"+ buf, 0, r);
   }
 
+  @Test
+  public void testDiffCommand() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    ms = getMetadataStore();
+    Set<Path> filesOnS3 = new HashSet<>(); // files on S3.
+    Set<Path> filesOnMS = new HashSet<>(); // files on metadata store.
+
+    Path testPath = path("test-diff");
+    mkdirs(testPath, true, true);
+
+    Path msOnlyPath = new Path(testPath, "ms_only");
+    mkdirs(msOnlyPath, false, true);
+    filesOnMS.add(msOnlyPath);
+    for (int i = 0; i < 5; i++) {
+      Path file = new Path(msOnlyPath, String.format("file-%d", i));
+      createFile(file, false, true);
+      filesOnMS.add(file);
+    }
+
+    Path s3OnlyPath = new Path(testPath, "s3_only");
+    mkdirs(s3OnlyPath, true, false);
+    filesOnS3.add(s3OnlyPath);
+    for (int i = 0; i < 5; i++) {
+      Path file = new Path(s3OnlyPath, String.format("file-%d", i));
+      createFile(file, true, false);
+      filesOnS3.add(file);
+    }
+
+    ByteArrayOutputStream buf = new ByteArrayOutputStream();
+    S3GuardTool.Diff cmd = new S3GuardTool.Diff(fs.getConf());
+    cmd.setStore(ms);
+    exec(cmd, buf, "diff", "-meta", DYNAMODB_TABLE, testPath.toString());
+
+    Set<Path> actualOnS3 = new HashSet<>();
+    Set<Path> actualOnMS = new HashSet<>();
+    boolean duplicates = false;
+    try (BufferedReader reader =
+        new BufferedReader(new InputStreamReader(
+            new ByteArrayInputStream(buf.toByteArray())))) {
+      String line;
+      while ((line = reader.readLine()) != null) {
+        String[] fields = line.split("\\s");
+        assertEquals("[" + line + "] does not have enough fields",
+            4, fields.length);
+        String where = fields[0];
+        Path path = new Path(fields[3]);
+        if (S3GuardTool.Diff.S3_PREFIX.equals(where)) {
+          duplicates = duplicates || actualOnS3.contains(path);
+          actualOnS3.add(path);
+        } else if (S3GuardTool.Diff.MS_PREFIX.equals(where)) {
+          duplicates = duplicates || actualOnMS.contains(path);
+          actualOnMS.add(path);
+        } else {
+          fail("Unknown prefix: " + where);
+        }
+      }
+    }
+    String actualOut = buf.toString();
+    assertEquals("Mismatched metadata store outputs: " + actualOut,
+        filesOnMS, actualOnMS);
+    assertEquals("Mismatched s3 outputs: " + actualOut, filesOnS3, actualOnS3);
+    assertFalse("Diff contained duplicates", duplicates);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8981674b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
index 64a2b13..a40c7a5 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
@@ -26,9 +26,7 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 import java.util.concurrent.Callable;
 
 import org.apache.hadoop.test.LambdaTestUtils;
@@ -38,7 +36,6 @@ import org.junit.Test;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Diff;
 
 import static org.apache.hadoop.fs.s3a.MultipartTestUtils.*;
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.*;
@@ -91,71 +88,6 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
   }
 
   @Test
-  public void testDiffCommand() throws Exception {
-    S3AFileSystem fs = getFileSystem();
-    MetadataStore ms = getMetadataStore();
-    Set<Path> filesOnS3 = new HashSet<>(); // files on S3.
-    Set<Path> filesOnMS = new HashSet<>(); // files on metadata store.
-
-    Path testPath = path("test-diff");
-    mkdirs(testPath, true, true);
-
-    Path msOnlyPath = new Path(testPath, "ms_only");
-    mkdirs(msOnlyPath, false, true);
-    filesOnMS.add(msOnlyPath);
-    for (int i = 0; i < 5; i++) {
-      Path file = new Path(msOnlyPath, String.format("file-%d", i));
-      createFile(file, false, true);
-      filesOnMS.add(file);
-    }
-
-    Path s3OnlyPath = new Path(testPath, "s3_only");
-    mkdirs(s3OnlyPath, true, false);
-    filesOnS3.add(s3OnlyPath);
-    for (int i = 0; i < 5; i++) {
-      Path file = new Path(s3OnlyPath, String.format("file-%d", i));
-      createFile(file, true, false);
-      filesOnS3.add(file);
-    }
-
-    ByteArrayOutputStream buf = new ByteArrayOutputStream();
-    Diff cmd = new Diff(fs.getConf());
-    cmd.setStore(ms);
-    exec(cmd, buf, "diff", "-meta", LOCAL_METADATA,
-            testPath.toString());
-
-    Set<Path> actualOnS3 = new HashSet<>();
-    Set<Path> actualOnMS = new HashSet<>();
-    boolean duplicates = false;
-    try (BufferedReader reader =
-             new BufferedReader(new InputStreamReader(
-                 new ByteArrayInputStream(buf.toByteArray())))) {
-      String line;
-      while ((line = reader.readLine()) != null) {
-        String[] fields = line.split("\\s");
-        assertEquals("[" + line + "] does not have enough fields",
-            4, fields.length);
-        String where = fields[0];
-        Path path = new Path(fields[3]);
-        if (Diff.S3_PREFIX.equals(where)) {
-          duplicates = duplicates || actualOnS3.contains(path);
-          actualOnS3.add(path);
-        } else if (Diff.MS_PREFIX.equals(where)) {
-          duplicates = duplicates || actualOnMS.contains(path);
-          actualOnMS.add(path);
-        } else {
-          fail("Unknown prefix: " + where);
-        }
-      }
-    }
-    String actualOut = buf.toString();
-    assertEquals("Mismatched metadata store outputs: " + actualOut,
-        filesOnMS, actualOnMS);
-    assertEquals("Mismatched s3 outputs: " + actualOut, filesOnS3, actualOnS3);
-    assertFalse("Diff contained duplicates", duplicates);
-  }
-
-  @Test
   public void testDestroyBucketExistsButNoTable() throws Throwable {
     run(Destroy.NAME,
         "-meta", LOCAL_METADATA,


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org