You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@gobblin.apache.org by GitBox <gi...@apache.org> on 2022/11/02 20:31:11 UTC

[GitHub] [gobblin] phet commented on a diff in pull request #3577: [GOBBLIN-1720]Add ancestors owner permissions preservations for iceberg distcp

phet commented on code in PR #3577:
URL: https://github.com/apache/gobblin/pull/3577#discussion_r1012232921


##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/IcebergDataset.java:
##########
@@ -144,13 +146,17 @@ Collection<CopyEntity> generateCopyEntities(FileSystem targetFs, CopyConfigurati
       FileStatus srcFileStatus = entry.getValue();
       // TODO: should be the same FS each time; try creating once, reusing thereafter, to not recreate wastefully
       FileSystem actualSourceFs = getSourceFileSystemFromFileStatus(srcFileStatus, defaultHadoopConfiguration);
+      Path greatestAncestorPath = PathUtils.getRootPathChild(srcPath);
 
-      // TODO: Add preservation of ancestor ownership and permissions!
-
+      // preserving ancestor permissions till root path's child between src and dest
+      List<OwnerAndPermission> ancestorOwnerAndPermission =

Review Comment:
   nit: using a plural name or appending `List` would better indicate it's not just for one ancestor



##########
gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/iceberg/IcebergDatasetTest.java:
##########
@@ -307,105 +329,137 @@ protected IcebergTable validateGetFilePathsGivenDestState(
   /** @return `paths` after adding to it all paths of every one of `snapshotDefs` */
   protected static Set<Path> withAllSnapshotPaths(Set<Path> paths, MockIcebergTable.SnapshotPaths... snapshotDefs) {
     Arrays.stream(snapshotDefs).flatMap(snapshotDef ->
-        snapshotDef.asSnapshotInfo().getAllPaths().stream()
-    ).forEach(p ->
-        paths.add(new Path(p))
-    );
+            snapshotDef.asSnapshotInfo().getAllPaths().stream())
+        .forEach(p ->
+            paths.add(new Path(p))
+        );
     return paths;
   }
 
   private CopyConfiguration createEmptyCopyConfiguration(FileSystem fs) {
-    return CopyConfiguration.builder(fs, copyConfigProperties)
-        .copyContext(new CopyContext())
-        .build();
+    return CopyConfiguration.builder(fs, copyConfigProperties).copyContext(new CopyContext()).build();
   }
 
   private static void verifyCopyEntities(Collection<CopyEntity> copyEntities, List<String> expected) {
-    List<String> actual = new ArrayList<>();
+    Set<String> actual = new HashSet<>();
     for (CopyEntity copyEntity : copyEntities) {
       String json = copyEntity.toString();
-      String filepath = new Gson().fromJson(json, JsonObject.class)
-          .getAsJsonObject("object-data").getAsJsonObject("origin")
-          .getAsJsonObject("object-data").getAsJsonObject("path")
-          .getAsJsonObject("object-data").getAsJsonObject("uri")
-          .getAsJsonPrimitive("object-data").getAsString();
+      String filepath = CopyEntityDeserializer.getFilePathAsStringFromJson(json);
       actual.add(filepath);
     }
-    Assert.assertEquals(actual.size(), expected.size(),
-        "Set" + actual.toString() + " vs Set" + expected.toString());
+    Assert.assertEquals(actual.size(), expected.size(), "Set" + actual.toString() + " vs Set" + expected.toString());
     Assert.assertEqualsNoOrder(actual.toArray(), expected.toArray());
   }
 
+  private static void verifyFsOwnershipAndPermissionPreservation(Collection<CopyEntity> copyEntities,
+      Map<Path, FileStatus> expectedPathsAndFileStatuses) {
+
+    for (CopyEntity copyEntity : copyEntities) {
+      String copyEntityJson = copyEntity.toString();
+
+      JsonArray ancestorOwnerAndPermissions = CopyEntityDeserializer.getAncestorOwnerAndPermissions(copyEntityJson);
+      String filePath = CopyEntityDeserializer.getFilePathAsStringFromJson(copyEntityJson);
+      CopyEntityDeserializer.DestinationOwnerAndPermissions destinationOwnerAndPermissions = CopyEntityDeserializer.getDestinationOwnerAndPermissions(copyEntityJson);
+
+      Assert.assertEquals(ancestorOwnerAndPermissions.size(), new Path(filePath).getParent().depth() - 1);
+      Assert.assertEquals(expectedPathsAndFileStatuses.get(new Path(filePath)).getOwner(),
+          destinationOwnerAndPermissions.owner);
+      Assert.assertEquals(expectedPathsAndFileStatuses.get(new Path(filePath)).getGroup(),
+          destinationOwnerAndPermissions.group);
+      Assert.assertEquals(expectedPathsAndFileStatuses.get(new Path(filePath)).getPermission().getUserAction().toString(),
+          destinationOwnerAndPermissions.userActionPermission);
+      Assert.assertEquals(expectedPathsAndFileStatuses.get(new Path(filePath)).getPermission().getGroupAction().toString(),
+          destinationOwnerAndPermissions.groupActionPermission);
+      Assert.assertEquals(expectedPathsAndFileStatuses.get(new Path(filePath)).getPermission().getOtherAction().toString(),
+          destinationOwnerAndPermissions.otherActionPermission);

Review Comment:
   at the least, let's tidy up w/ an intermediate:
   ```
   FileStatus expectedFileStatus = expectedPathsAndFileStatuses.get(new Path(filePath));
   ```
   but even better might be to abstract:
   ```
   void verifyDestFileStatus(CopyEntityDeserializer.DestinationOwnerAndPermissions observed, FileStatus expected) ...
   ```



##########
gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/iceberg/IcebergDatasetTest.java:
##########
@@ -464,5 +521,54 @@ public static <T, R> Stream<R> transformWithIndex(Stream<T> inputs, BiFunction<T
           inputs, IntStream.iterate(0, i -> i + 1).boxed(), f);
     }
   }
+
+  private static class CopyEntityDeserializer {
+
+    @Data
+    public static class DestinationOwnerAndPermissions {

Review Comment:
   what's destination specific here?  maybe `FileOwnershipAndPerms`?



##########
gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/iceberg/IcebergDatasetTest.java:
##########
@@ -307,105 +329,137 @@ protected IcebergTable validateGetFilePathsGivenDestState(
   /** @return `paths` after adding to it all paths of every one of `snapshotDefs` */
   protected static Set<Path> withAllSnapshotPaths(Set<Path> paths, MockIcebergTable.SnapshotPaths... snapshotDefs) {
     Arrays.stream(snapshotDefs).flatMap(snapshotDef ->
-        snapshotDef.asSnapshotInfo().getAllPaths().stream()
-    ).forEach(p ->
-        paths.add(new Path(p))
-    );
+            snapshotDef.asSnapshotInfo().getAllPaths().stream())
+        .forEach(p ->
+            paths.add(new Path(p))
+        );
     return paths;
   }
 
   private CopyConfiguration createEmptyCopyConfiguration(FileSystem fs) {
-    return CopyConfiguration.builder(fs, copyConfigProperties)
-        .copyContext(new CopyContext())
-        .build();
+    return CopyConfiguration.builder(fs, copyConfigProperties).copyContext(new CopyContext()).build();
   }
 
   private static void verifyCopyEntities(Collection<CopyEntity> copyEntities, List<String> expected) {
-    List<String> actual = new ArrayList<>();
+    Set<String> actual = new HashSet<>();
     for (CopyEntity copyEntity : copyEntities) {
       String json = copyEntity.toString();
-      String filepath = new Gson().fromJson(json, JsonObject.class)
-          .getAsJsonObject("object-data").getAsJsonObject("origin")
-          .getAsJsonObject("object-data").getAsJsonObject("path")
-          .getAsJsonObject("object-data").getAsJsonObject("uri")
-          .getAsJsonPrimitive("object-data").getAsString();
+      String filepath = CopyEntityDeserializer.getFilePathAsStringFromJson(json);
       actual.add(filepath);
     }
-    Assert.assertEquals(actual.size(), expected.size(),
-        "Set" + actual.toString() + " vs Set" + expected.toString());
+    Assert.assertEquals(actual.size(), expected.size(), "Set" + actual.toString() + " vs Set" + expected.toString());
     Assert.assertEqualsNoOrder(actual.toArray(), expected.toArray());
   }
 
+  private static void verifyFsOwnershipAndPermissionPreservation(Collection<CopyEntity> copyEntities,
+      Map<Path, FileStatus> expectedPathsAndFileStatuses) {
+
+    for (CopyEntity copyEntity : copyEntities) {
+      String copyEntityJson = copyEntity.toString();
+
+      JsonArray ancestorOwnerAndPermissions = CopyEntityDeserializer.getAncestorOwnerAndPermissions(copyEntityJson);
+      String filePath = CopyEntityDeserializer.getFilePathAsStringFromJson(copyEntityJson);
+      CopyEntityDeserializer.DestinationOwnerAndPermissions destinationOwnerAndPermissions = CopyEntityDeserializer.getDestinationOwnerAndPermissions(copyEntityJson);
+
+      Assert.assertEquals(ancestorOwnerAndPermissions.size(), new Path(filePath).getParent().depth() - 1);
+      Assert.assertEquals(expectedPathsAndFileStatuses.get(new Path(filePath)).getOwner(),
+          destinationOwnerAndPermissions.owner);
+      Assert.assertEquals(expectedPathsAndFileStatuses.get(new Path(filePath)).getGroup(),
+          destinationOwnerAndPermissions.group);
+      Assert.assertEquals(expectedPathsAndFileStatuses.get(new Path(filePath)).getPermission().getUserAction().toString(),
+          destinationOwnerAndPermissions.userActionPermission);
+      Assert.assertEquals(expectedPathsAndFileStatuses.get(new Path(filePath)).getPermission().getGroupAction().toString(),
+          destinationOwnerAndPermissions.groupActionPermission);
+      Assert.assertEquals(expectedPathsAndFileStatuses.get(new Path(filePath)).getPermission().getOtherAction().toString(),
+          destinationOwnerAndPermissions.otherActionPermission);
+    }
+  }
 
   /**
    *  Sadly, this is needed to avoid losing `FileSystem` mock to replacement from the `FileSystem.get` `static`
    *  Without this, so to lose the mock, we'd be unable to set up any source paths as existing.
    */
   protected static class TrickIcebergDataset extends IcebergDataset {
-    public TrickIcebergDataset(String db, String table, IcebergTable icebergTbl, Properties properties, FileSystem sourceFs) {
+    public TrickIcebergDataset(String db, String table, IcebergTable icebergTbl, Properties properties,
+        FileSystem sourceFs) {
       super(db, table, icebergTbl, properties, sourceFs);
     }
 
     @Override // as the `static` is not mock-able
-    protected FileSystem getSourceFileSystemFromFileStatus(FileStatus fileStatus, Configuration hadoopConfig) throws IOException {
+    protected FileSystem getSourceFileSystemFromFileStatus(FileStatus fileStatus, Configuration hadoopConfig)
+        throws IOException {
       return this.sourceFs;
     }
-  };
+  }
 
+  ;
 
   /** Builds a {@link FileSystem} mock */
   protected static class MockFileSystemBuilder {
     private final URI fsURI;
     /** when not `.isPresent()`, all paths exist; when `.get().isEmpty()`, none exist; else only those indicated do */
-    private final Optional<Set<Path>> optPaths;
+    private final Optional<Map<Path, FileStatus>> optPathsWithFileStatuses;
 
     public MockFileSystemBuilder(URI fsURI) {
       this(fsURI, false);
     }
 
     public MockFileSystemBuilder(URI fsURI, boolean shouldRepresentEveryPath) {
       this.fsURI = fsURI;
-      this.optPaths = shouldRepresentEveryPath ? Optional.empty() : Optional.of(Sets.newHashSet());
+      this.optPathsWithFileStatuses = shouldRepresentEveryPath ? Optional.empty() : Optional.of(Maps.newHashMap());
     }
 
-    public Optional<Set<Path>> getPaths() {
-      return this.optPaths.map(Sets::newHashSet); // copy before returning
+    public void addPaths(List<String> pathStrings) {
+      addPathsAndFileStatuses(pathStrings, false);
     }
 
-    public void addPaths(List<String> pathStrings) {
+    public void addPathsAndFileStatuses(List<String> pathStrings, boolean shouldCreateFileStatusWithPermissions) {
       for (String pathString : pathStrings) {
-        addPath(pathString);
+        addPathAndFileStatus(pathString, shouldCreateFileStatusWithPermissions);
       }
     }
 
-    public void addPath(String pathString) {
-      addPath(new Path(pathString));
+    public void addPathAndFileStatus(String pathString, boolean shouldCreateFileStatusWithPermissions) {
+      addPathAndFileStatus(new Path(pathString), shouldCreateFileStatusWithPermissions);
     }
 
-    public void addPath(Path path) {
-      if (!this.optPaths.isPresent()) {
+    public void addPathAndFileStatus(Path path, boolean shouldCreateFileStatusWithPermissions) {
+      if (!this.optPathsWithFileStatuses.isPresent()) {
         throw new IllegalStateException("unable to add paths when constructed with `shouldRepresentEveryPath == true`");
       }
-      if (this.optPaths.get().add(path) && !path.isRoot()) { // recursively add ancestors of a previously unknown path
-        addPath(path.getParent());
+      FileStatus fileStatus = shouldCreateFileStatusWithPermissions ? createFileStatus(path, "test_owner", "test_group", new FsPermission(FsAction.WRITE_EXECUTE, FsAction.READ_EXECUTE, FsAction.NONE)) : null;

Review Comment:
   hard-coding behind-the-scenes makes the mock brittle--and code harder to follow.  let's keep things simple w/ the code using the mock choosing the filestatus to supply for which path.



##########
gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/iceberg/IcebergDatasetTest.java:
##########
@@ -219,21 +220,20 @@ public void testPathErrorConsolidator() {
   public void testGenerateCopyEntitiesWhenDestEmpty() throws IOException {
     List<String> expectedPaths = Arrays.asList(METADATA_PATH, MANIFEST_LIST_PATH_0,
         MANIFEST_PATH_0, MANIFEST_DATA_PATH_0A, MANIFEST_DATA_PATH_0B);
-
     MockFileSystemBuilder sourceBuilder = new MockFileSystemBuilder(SRC_FS_URI);
-    sourceBuilder.addPaths(expectedPaths);
+    sourceBuilder.addPathsAndFileStatuses(expectedPaths, false);

Review Comment:
   `addPathsAndFileStatuses()` doesn't make sense when called w/ a collection of paths and a boolean.  let's think from the perspective of a new maintaner.  anyway, seems `.addPaths(ps)` has equivalent semantics and inserts `false` behind the scenes, no?



##########
gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/iceberg/IcebergDatasetTest.java:
##########
@@ -216,210 +228,295 @@ public void testPathErrorConsolidator() {
    * without calculating any difference between the source and destination
    */
   @Test
-  public void testGenerateCopyEntitiesWhenDestEmpty() throws IOException {
-    List<String> expectedPaths = Arrays.asList(METADATA_PATH, MANIFEST_LIST_PATH_0,
-        MANIFEST_PATH_0, MANIFEST_DATA_PATH_0A, MANIFEST_DATA_PATH_0B);
+  public void testGenerateCopyEntitiesWhenDestEmpty()
+      throws IOException {
+    Map<String, FileStatus> expectedPathsAndFileStatuses = Maps.newHashMap();
+    expectedPathsAndFileStatuses.put(METADATA_PATH, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_LIST_PATH_0, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_PATH_0, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_0A, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_0B, null);
+    Set<String> expectedPaths = expectedPathsAndFileStatuses.keySet();
 
     MockFileSystemBuilder sourceBuilder = new MockFileSystemBuilder(SRC_FS_URI);
-    sourceBuilder.addPaths(expectedPaths);
+    sourceBuilder.addPathsAndFileStatuses(expectedPathsAndFileStatuses);
     FileSystem sourceFs = sourceBuilder.build();
 
     IcebergTable icebergTable = MockIcebergTable.withSnapshots(Arrays.asList(SNAPSHOT_PATHS_0));
-    IcebergDataset icebergDataset = new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
+    IcebergDataset icebergDataset =
+        new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
 
     MockFileSystemBuilder destBuilder = new MockFileSystemBuilder(DEST_FS_URI);
     FileSystem destFs = destBuilder.build();
 
-    CopyConfiguration copyConfiguration = CopyConfiguration.builder(destFs, copyConfigProperties)
-        .preserve(PreserveAttributes.fromMnemonicString(""))
-        .copyContext(new CopyContext())
-        .build();
+    CopyConfiguration copyConfiguration =
+        CopyConfiguration.builder(destFs, copyConfigProperties).preserve(PreserveAttributes.fromMnemonicString(""))
+            .copyContext(new CopyContext()).build();
     Collection<CopyEntity> copyEntities = icebergDataset.generateCopyEntities(destFs, copyConfiguration);
     verifyCopyEntities(copyEntities, expectedPaths);
   }
 
   /** Test generating copy entities for a multi-snapshot iceberg; given empty dest, src-dest delta will be entirety */
   @Test
-  public void testGenerateCopyEntitiesMultiSnapshotWhenDestEmpty() throws IOException {
-    List<String> expectedPaths = Arrays.asList(METADATA_PATH,
-        MANIFEST_LIST_PATH_0, MANIFEST_PATH_0, MANIFEST_DATA_PATH_0A, MANIFEST_DATA_PATH_0B,
-        MANIFEST_LIST_PATH_1, MANIFEST_PATH_1, MANIFEST_DATA_PATH_1A, MANIFEST_DATA_PATH_1B);
+  public void testGenerateCopyEntitiesMultiSnapshotWhenDestEmpty()
+      throws IOException {
+    Map<String, FileStatus> expectedPathsAndFileStatuses = Maps.newHashMap();
+    expectedPathsAndFileStatuses.put(METADATA_PATH, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_LIST_PATH_0, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_PATH_0, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_0A, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_0B, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_LIST_PATH_1, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_PATH_1, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_1A, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_1B, null);
+    Set<String> expectedPaths = (expectedPathsAndFileStatuses.keySet());
 
     MockFileSystemBuilder sourceBuilder = new MockFileSystemBuilder(SRC_FS_URI);
-    sourceBuilder.addPaths(expectedPaths);
+    sourceBuilder.addPathsAndFileStatuses(expectedPathsAndFileStatuses);
     FileSystem sourceFs = sourceBuilder.build();
 
     IcebergTable icebergTable = MockIcebergTable.withSnapshots(Arrays.asList(SNAPSHOT_PATHS_1, SNAPSHOT_PATHS_0));
-    IcebergDataset icebergDataset = new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
+    IcebergDataset icebergDataset =
+        new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
 
     MockFileSystemBuilder destBuilder = new MockFileSystemBuilder(DEST_FS_URI);
     FileSystem destFs = destBuilder.build();
 
-    CopyConfiguration copyConfiguration = CopyConfiguration.builder(destFs, copyConfigProperties)
-        .preserve(PreserveAttributes.fromMnemonicString(""))
-        .copyContext(new CopyContext())
-        .build();
+    CopyConfiguration copyConfiguration =
+        CopyConfiguration.builder(destFs, copyConfigProperties).preserve(PreserveAttributes.fromMnemonicString(""))
+            .copyContext(new CopyContext()).build();
     Collection<CopyEntity> copyEntities = icebergDataset.generateCopyEntities(destFs, copyConfiguration);
     verifyCopyEntities(copyEntities, expectedPaths);
   }
 
+  @Test
+  public void testFsOwnershipAndPermissionPreservationWhenDestEmpty()
+      throws IOException {
+    Map<String, FileStatus> expectedPathsAndFileStatuses = Maps.newHashMap();
+    expectedPathsAndFileStatuses.put(METADATA_PATH, createFileStatus(METADATA_PATH, null, null,
+        new FsPermission(FsAction.WRITE_EXECUTE, FsAction.READ_EXECUTE, FsAction.NONE)));
+    expectedPathsAndFileStatuses.put(MANIFEST_LIST_PATH_0, createFileStatus(MANIFEST_LIST_PATH_0, null, null,
+        new FsPermission(FsAction.WRITE_EXECUTE, FsAction.READ_EXECUTE, FsAction.NONE)));
+    expectedPathsAndFileStatuses.put(MANIFEST_PATH_0, createFileStatus(MANIFEST_PATH_0, null, null,
+        new FsPermission(FsAction.WRITE_EXECUTE, FsAction.READ_EXECUTE, FsAction.NONE)));
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_0A, createFileStatus(MANIFEST_DATA_PATH_0A, null, null,
+        new FsPermission(FsAction.WRITE_EXECUTE, FsAction.READ_EXECUTE, FsAction.NONE)));
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_0B, createFileStatus(MANIFEST_DATA_PATH_0B, null, null,
+        new FsPermission(FsAction.WRITE_EXECUTE, FsAction.READ_EXECUTE, FsAction.NONE)));
+
+    MockFileSystemBuilder sourceBuilder = new MockFileSystemBuilder(SRC_FS_URI);
+    sourceBuilder.addPathsAndFileStatuses(expectedPathsAndFileStatuses);
+    FileSystem sourceFs = sourceBuilder.build();
+
+    IcebergTable icebergTable = MockIcebergTable.withSnapshots(Arrays.asList(SNAPSHOT_PATHS_0));
+    IcebergDataset icebergDataset =
+        new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
+
+    MockFileSystemBuilder destBuilder = new MockFileSystemBuilder(DEST_FS_URI);
+    FileSystem destFs = destBuilder.build();
+
+    CopyConfiguration copyConfiguration =
+        CopyConfiguration.builder(destFs, copyConfigProperties).preserve(PreserveAttributes.fromMnemonicString("ugp"))
+            .copyContext(new CopyContext()).build();
+
+    Collection<CopyEntity> copyEntities = icebergDataset.generateCopyEntities(destFs, copyConfiguration);
+    verifyFsOwnershipAndPermissionPreservation(copyEntities, expectedPathsAndFileStatuses);
+  }
+
   /**
    *  exercise {@link IcebergDataset::getFilePaths} and validate the result
    *  @return {@link IcebergTable} (mock!), for behavioral verification
    */
-  protected IcebergTable validateGetFilePathsGivenDestState(
-      List<MockIcebergTable.SnapshotPaths> sourceSnapshotPathSets,
-      List<String> existingDestPaths,
-      Set<Path> expectedResultPaths) throws IOException {
-    return validateGetFilePathsGivenDestState(sourceSnapshotPathSets, Optional.empty(),existingDestPaths, expectedResultPaths);
+  protected IcebergTable validateGetFilePathsGivenDestState(List<MockIcebergTable.SnapshotPaths> sourceSnapshotPathSets,
+      List<String> existingDestPaths, Set<Path> expectedResultPaths)
+      throws IOException {
+    return validateGetFilePathsGivenDestState(sourceSnapshotPathSets, Optional.empty(), existingDestPaths,
+        expectedResultPaths);
   }
 
   /**
    *  exercise {@link IcebergDataset::getFilePaths} and validate the result
    *  @return {@link IcebergTable} (mock!), for behavioral verification
    */
-  protected IcebergTable validateGetFilePathsGivenDestState(
-      List<MockIcebergTable.SnapshotPaths> sourceSnapshotPathSets,
-      Optional<List<String>> optExistingSourcePaths,
-      List<String> existingDestPaths,
-      Set<Path> expectedResultPaths) throws IOException {
+  protected IcebergTable validateGetFilePathsGivenDestState(List<MockIcebergTable.SnapshotPaths> sourceSnapshotPathSets,
+      Optional<List<String>> optExistingSourcePaths, List<String> existingDestPaths, Set<Path> expectedResultPaths)
+      throws IOException {
     IcebergTable icebergTable = MockIcebergTable.withSnapshots(sourceSnapshotPathSets);
+    Map<String, FileStatus> sourcePathAndFileStatusMap = Maps.newHashMap();
 
     MockFileSystemBuilder sourceFsBuilder = new MockFileSystemBuilder(SRC_FS_URI, !optExistingSourcePaths.isPresent());
-    optExistingSourcePaths.ifPresent(sourceFsBuilder::addPaths);
+    optExistingSourcePaths.ifPresent((pathList) -> {
+      for (String path : pathList) {
+        sourcePathAndFileStatusMap.putIfAbsent(path, null);
+      }
+      sourceFsBuilder.addPathsAndFileStatuses(sourcePathAndFileStatusMap);
+    });
     FileSystem sourceFs = sourceFsBuilder.build();
-    IcebergDataset icebergDataset = new IcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
+    IcebergDataset icebergDataset =
+        new IcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
 
     MockFileSystemBuilder destFsBuilder = new MockFileSystemBuilder(DEST_FS_URI);
-    destFsBuilder.addPaths(existingDestPaths);
+    Map<String, FileStatus> destPathAndFileStatusMap = Maps.newHashMap();
+    for (String destPath : existingDestPaths) {
+      destPathAndFileStatusMap.putIfAbsent(destPath, null);
+    }
+    destFsBuilder.addPathsAndFileStatuses(destPathAndFileStatusMap);
     FileSystem destFs = destFsBuilder.build();
     CopyConfiguration copyConfiguration = createEmptyCopyConfiguration(destFs);
 
     Map<Path, FileStatus> filePathsToFileStatus = icebergDataset.getFilePathsToFileStatus(destFs, copyConfiguration);
     Assert.assertEquals(filePathsToFileStatus.keySet(), expectedResultPaths);
     // verify solely the path portion of the `FileStatus`, since that's all mock sets up
-    Assert.assertEquals(
-        filePathsToFileStatus.values().stream().map(FileStatus::getPath).collect(Collectors.toSet()),
+    Assert.assertEquals(filePathsToFileStatus.values().stream().map(FileStatus::getPath).collect(Collectors.toSet()),
         expectedResultPaths);
     return icebergTable;
   }
 
   /** @return `paths` after adding to it all paths of every one of `snapshotDefs` */
   protected static Set<Path> withAllSnapshotPaths(Set<Path> paths, MockIcebergTable.SnapshotPaths... snapshotDefs) {
-    Arrays.stream(snapshotDefs).flatMap(snapshotDef ->
-        snapshotDef.asSnapshotInfo().getAllPaths().stream()
-    ).forEach(p ->
-        paths.add(new Path(p))
-    );
+    Arrays.stream(snapshotDefs).flatMap(snapshotDef -> snapshotDef.asSnapshotInfo().getAllPaths().stream())
+        .forEach(p -> paths.add(new Path(p)));
     return paths;
   }
 
   private CopyConfiguration createEmptyCopyConfiguration(FileSystem fs) {
-    return CopyConfiguration.builder(fs, copyConfigProperties)
-        .copyContext(new CopyContext())
-        .build();
+    return CopyConfiguration.builder(fs, copyConfigProperties).copyContext(new CopyContext()).build();
   }
 
-  private static void verifyCopyEntities(Collection<CopyEntity> copyEntities, List<String> expected) {
-    List<String> actual = new ArrayList<>();
+  private static void verifyCopyEntities(Collection<CopyEntity> copyEntities, Set<String> expected) {
+    Set<String> actual = new HashSet<>();
+    CopyEntityDeserializer copyEntityDeserializer = new CopyEntityDeserializer();
     for (CopyEntity copyEntity : copyEntities) {
       String json = copyEntity.toString();
-      String filepath = new Gson().fromJson(json, JsonObject.class)
-          .getAsJsonObject("object-data").getAsJsonObject("origin")
-          .getAsJsonObject("object-data").getAsJsonObject("path")
-          .getAsJsonObject("object-data").getAsJsonObject("uri")
-          .getAsJsonPrimitive("object-data").getAsString();
+      String filepath = copyEntityDeserializer.getFilePathAsStringFromJson(json);
       actual.add(filepath);
     }
-    Assert.assertEquals(actual.size(), expected.size(),
-        "Set" + actual.toString() + " vs Set" + expected.toString());
+    Assert.assertEquals(actual.size(), expected.size(), "Set" + actual.toString() + " vs Set" + expected.toString());
     Assert.assertEqualsNoOrder(actual.toArray(), expected.toArray());
   }
 
+  private void verifyFsOwnershipAndPermissionPreservation(Collection<CopyEntity> copyEntities,
+      Map<String, FileStatus> expectedMap) {
+    CopyEntityDeserializer copyEntityDeserializer = new CopyEntityDeserializer();
+
+    for (CopyEntity copyEntity : copyEntities) {
+      String copyEntityJson = copyEntity.toString();
+
+      JsonArray ancestorOwnerAndPermissions = copyEntityDeserializer.getAncestorOwnerAndPermissions(copyEntityJson);
+      String filePath = copyEntityDeserializer.getFilePathAsStringFromJson(copyEntityJson);
+      JsonObject destinationOwnerAndPermissions =
+          copyEntityDeserializer.getDestinationOwnerAndPermissions(copyEntityJson);
+
+      FileStatus expected = expectedMap.get(filePath);
+      Map<String, String> actual =
+          copyEntityDeserializer.getDestinationOwnerAndPermissionsFsPermissionsMap(destinationOwnerAndPermissions);
+
+      Assert.assertEquals(ancestorOwnerAndPermissions.size(), new Path(filePath).getParent().depth() - 1);

Review Comment:
   seems we still don't verify the specific values along the ancestor chain...



##########
gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/iceberg/IcebergDatasetTest.java:
##########
@@ -216,210 +228,295 @@ public void testPathErrorConsolidator() {
    * without calculating any difference between the source and destination
    */
   @Test
-  public void testGenerateCopyEntitiesWhenDestEmpty() throws IOException {
-    List<String> expectedPaths = Arrays.asList(METADATA_PATH, MANIFEST_LIST_PATH_0,
-        MANIFEST_PATH_0, MANIFEST_DATA_PATH_0A, MANIFEST_DATA_PATH_0B);
+  public void testGenerateCopyEntitiesWhenDestEmpty()
+      throws IOException {
+    Map<String, FileStatus> expectedPathsAndFileStatuses = Maps.newHashMap();
+    expectedPathsAndFileStatuses.put(METADATA_PATH, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_LIST_PATH_0, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_PATH_0, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_0A, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_0B, null);
+    Set<String> expectedPaths = expectedPathsAndFileStatuses.keySet();
 
     MockFileSystemBuilder sourceBuilder = new MockFileSystemBuilder(SRC_FS_URI);
-    sourceBuilder.addPaths(expectedPaths);
+    sourceBuilder.addPathsAndFileStatuses(expectedPathsAndFileStatuses);
     FileSystem sourceFs = sourceBuilder.build();
 
     IcebergTable icebergTable = MockIcebergTable.withSnapshots(Arrays.asList(SNAPSHOT_PATHS_0));
-    IcebergDataset icebergDataset = new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
+    IcebergDataset icebergDataset =
+        new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
 
     MockFileSystemBuilder destBuilder = new MockFileSystemBuilder(DEST_FS_URI);
     FileSystem destFs = destBuilder.build();
 
-    CopyConfiguration copyConfiguration = CopyConfiguration.builder(destFs, copyConfigProperties)
-        .preserve(PreserveAttributes.fromMnemonicString(""))
-        .copyContext(new CopyContext())
-        .build();
+    CopyConfiguration copyConfiguration =
+        CopyConfiguration.builder(destFs, copyConfigProperties).preserve(PreserveAttributes.fromMnemonicString(""))
+            .copyContext(new CopyContext()).build();
     Collection<CopyEntity> copyEntities = icebergDataset.generateCopyEntities(destFs, copyConfiguration);
     verifyCopyEntities(copyEntities, expectedPaths);
   }
 
   /** Test generating copy entities for a multi-snapshot iceberg; given empty dest, src-dest delta will be entirety */
   @Test
-  public void testGenerateCopyEntitiesMultiSnapshotWhenDestEmpty() throws IOException {
-    List<String> expectedPaths = Arrays.asList(METADATA_PATH,
-        MANIFEST_LIST_PATH_0, MANIFEST_PATH_0, MANIFEST_DATA_PATH_0A, MANIFEST_DATA_PATH_0B,
-        MANIFEST_LIST_PATH_1, MANIFEST_PATH_1, MANIFEST_DATA_PATH_1A, MANIFEST_DATA_PATH_1B);
+  public void testGenerateCopyEntitiesMultiSnapshotWhenDestEmpty()
+      throws IOException {
+    Map<String, FileStatus> expectedPathsAndFileStatuses = Maps.newHashMap();
+    expectedPathsAndFileStatuses.put(METADATA_PATH, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_LIST_PATH_0, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_PATH_0, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_0A, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_0B, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_LIST_PATH_1, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_PATH_1, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_1A, null);
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_1B, null);
+    Set<String> expectedPaths = (expectedPathsAndFileStatuses.keySet());
 
     MockFileSystemBuilder sourceBuilder = new MockFileSystemBuilder(SRC_FS_URI);
-    sourceBuilder.addPaths(expectedPaths);
+    sourceBuilder.addPathsAndFileStatuses(expectedPathsAndFileStatuses);
     FileSystem sourceFs = sourceBuilder.build();
 
     IcebergTable icebergTable = MockIcebergTable.withSnapshots(Arrays.asList(SNAPSHOT_PATHS_1, SNAPSHOT_PATHS_0));
-    IcebergDataset icebergDataset = new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
+    IcebergDataset icebergDataset =
+        new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
 
     MockFileSystemBuilder destBuilder = new MockFileSystemBuilder(DEST_FS_URI);
     FileSystem destFs = destBuilder.build();
 
-    CopyConfiguration copyConfiguration = CopyConfiguration.builder(destFs, copyConfigProperties)
-        .preserve(PreserveAttributes.fromMnemonicString(""))
-        .copyContext(new CopyContext())
-        .build();
+    CopyConfiguration copyConfiguration =
+        CopyConfiguration.builder(destFs, copyConfigProperties).preserve(PreserveAttributes.fromMnemonicString(""))
+            .copyContext(new CopyContext()).build();
     Collection<CopyEntity> copyEntities = icebergDataset.generateCopyEntities(destFs, copyConfiguration);
     verifyCopyEntities(copyEntities, expectedPaths);
   }
 
+  @Test
+  public void testFsOwnershipAndPermissionPreservationWhenDestEmpty()
+      throws IOException {
+    Map<String, FileStatus> expectedPathsAndFileStatuses = Maps.newHashMap();
+    expectedPathsAndFileStatuses.put(METADATA_PATH, createFileStatus(METADATA_PATH, null, null,
+        new FsPermission(FsAction.WRITE_EXECUTE, FsAction.READ_EXECUTE, FsAction.NONE)));
+    expectedPathsAndFileStatuses.put(MANIFEST_LIST_PATH_0, createFileStatus(MANIFEST_LIST_PATH_0, null, null,
+        new FsPermission(FsAction.WRITE_EXECUTE, FsAction.READ_EXECUTE, FsAction.NONE)));
+    expectedPathsAndFileStatuses.put(MANIFEST_PATH_0, createFileStatus(MANIFEST_PATH_0, null, null,
+        new FsPermission(FsAction.WRITE_EXECUTE, FsAction.READ_EXECUTE, FsAction.NONE)));
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_0A, createFileStatus(MANIFEST_DATA_PATH_0A, null, null,
+        new FsPermission(FsAction.WRITE_EXECUTE, FsAction.READ_EXECUTE, FsAction.NONE)));
+    expectedPathsAndFileStatuses.put(MANIFEST_DATA_PATH_0B, createFileStatus(MANIFEST_DATA_PATH_0B, null, null,
+        new FsPermission(FsAction.WRITE_EXECUTE, FsAction.READ_EXECUTE, FsAction.NONE)));
+
+    MockFileSystemBuilder sourceBuilder = new MockFileSystemBuilder(SRC_FS_URI);
+    sourceBuilder.addPathsAndFileStatuses(expectedPathsAndFileStatuses);
+    FileSystem sourceFs = sourceBuilder.build();
+
+    IcebergTable icebergTable = MockIcebergTable.withSnapshots(Arrays.asList(SNAPSHOT_PATHS_0));
+    IcebergDataset icebergDataset =
+        new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
+
+    MockFileSystemBuilder destBuilder = new MockFileSystemBuilder(DEST_FS_URI);
+    FileSystem destFs = destBuilder.build();
+
+    CopyConfiguration copyConfiguration =
+        CopyConfiguration.builder(destFs, copyConfigProperties).preserve(PreserveAttributes.fromMnemonicString("ugp"))
+            .copyContext(new CopyContext()).build();
+
+    Collection<CopyEntity> copyEntities = icebergDataset.generateCopyEntities(destFs, copyConfiguration);
+    verifyFsOwnershipAndPermissionPreservation(copyEntities, expectedPathsAndFileStatuses);
+  }
+
   /**
    *  exercise {@link IcebergDataset::getFilePaths} and validate the result
    *  @return {@link IcebergTable} (mock!), for behavioral verification
    */
-  protected IcebergTable validateGetFilePathsGivenDestState(
-      List<MockIcebergTable.SnapshotPaths> sourceSnapshotPathSets,
-      List<String> existingDestPaths,
-      Set<Path> expectedResultPaths) throws IOException {
-    return validateGetFilePathsGivenDestState(sourceSnapshotPathSets, Optional.empty(),existingDestPaths, expectedResultPaths);
+  protected IcebergTable validateGetFilePathsGivenDestState(List<MockIcebergTable.SnapshotPaths> sourceSnapshotPathSets,
+      List<String> existingDestPaths, Set<Path> expectedResultPaths)
+      throws IOException {
+    return validateGetFilePathsGivenDestState(sourceSnapshotPathSets, Optional.empty(), existingDestPaths,
+        expectedResultPaths);
   }
 
   /**
    *  exercise {@link IcebergDataset::getFilePaths} and validate the result
    *  @return {@link IcebergTable} (mock!), for behavioral verification
    */
-  protected IcebergTable validateGetFilePathsGivenDestState(
-      List<MockIcebergTable.SnapshotPaths> sourceSnapshotPathSets,
-      Optional<List<String>> optExistingSourcePaths,
-      List<String> existingDestPaths,
-      Set<Path> expectedResultPaths) throws IOException {
+  protected IcebergTable validateGetFilePathsGivenDestState(List<MockIcebergTable.SnapshotPaths> sourceSnapshotPathSets,
+      Optional<List<String>> optExistingSourcePaths, List<String> existingDestPaths, Set<Path> expectedResultPaths)
+      throws IOException {
     IcebergTable icebergTable = MockIcebergTable.withSnapshots(sourceSnapshotPathSets);
+    Map<String, FileStatus> sourcePathAndFileStatusMap = Maps.newHashMap();
 
     MockFileSystemBuilder sourceFsBuilder = new MockFileSystemBuilder(SRC_FS_URI, !optExistingSourcePaths.isPresent());
-    optExistingSourcePaths.ifPresent(sourceFsBuilder::addPaths);
+    optExistingSourcePaths.ifPresent((pathList) -> {
+      for (String path : pathList) {
+        sourcePathAndFileStatusMap.putIfAbsent(path, null);
+      }
+      sourceFsBuilder.addPathsAndFileStatuses(sourcePathAndFileStatusMap);
+    });
     FileSystem sourceFs = sourceFsBuilder.build();
-    IcebergDataset icebergDataset = new IcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
+    IcebergDataset icebergDataset =
+        new IcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
 
     MockFileSystemBuilder destFsBuilder = new MockFileSystemBuilder(DEST_FS_URI);
-    destFsBuilder.addPaths(existingDestPaths);
+    Map<String, FileStatus> destPathAndFileStatusMap = Maps.newHashMap();
+    for (String destPath : existingDestPaths) {
+      destPathAndFileStatusMap.putIfAbsent(destPath, null);
+    }
+    destFsBuilder.addPathsAndFileStatuses(destPathAndFileStatusMap);
     FileSystem destFs = destFsBuilder.build();
     CopyConfiguration copyConfiguration = createEmptyCopyConfiguration(destFs);
 
     Map<Path, FileStatus> filePathsToFileStatus = icebergDataset.getFilePathsToFileStatus(destFs, copyConfiguration);
     Assert.assertEquals(filePathsToFileStatus.keySet(), expectedResultPaths);
     // verify solely the path portion of the `FileStatus`, since that's all mock sets up
-    Assert.assertEquals(
-        filePathsToFileStatus.values().stream().map(FileStatus::getPath).collect(Collectors.toSet()),
+    Assert.assertEquals(filePathsToFileStatus.values().stream().map(FileStatus::getPath).collect(Collectors.toSet()),
         expectedResultPaths);
     return icebergTable;
   }
 
   /** @return `paths` after adding to it all paths of every one of `snapshotDefs` */
   protected static Set<Path> withAllSnapshotPaths(Set<Path> paths, MockIcebergTable.SnapshotPaths... snapshotDefs) {
-    Arrays.stream(snapshotDefs).flatMap(snapshotDef ->
-        snapshotDef.asSnapshotInfo().getAllPaths().stream()
-    ).forEach(p ->
-        paths.add(new Path(p))
-    );
+    Arrays.stream(snapshotDefs).flatMap(snapshotDef -> snapshotDef.asSnapshotInfo().getAllPaths().stream())
+        .forEach(p -> paths.add(new Path(p)));
     return paths;
   }
 
   private CopyConfiguration createEmptyCopyConfiguration(FileSystem fs) {
-    return CopyConfiguration.builder(fs, copyConfigProperties)
-        .copyContext(new CopyContext())
-        .build();
+    return CopyConfiguration.builder(fs, copyConfigProperties).copyContext(new CopyContext()).build();
   }
 
-  private static void verifyCopyEntities(Collection<CopyEntity> copyEntities, List<String> expected) {
-    List<String> actual = new ArrayList<>();
+  private static void verifyCopyEntities(Collection<CopyEntity> copyEntities, Set<String> expected) {
+    Set<String> actual = new HashSet<>();
+    CopyEntityDeserializer copyEntityDeserializer = new CopyEntityDeserializer();
     for (CopyEntity copyEntity : copyEntities) {
       String json = copyEntity.toString();
-      String filepath = new Gson().fromJson(json, JsonObject.class)
-          .getAsJsonObject("object-data").getAsJsonObject("origin")
-          .getAsJsonObject("object-data").getAsJsonObject("path")
-          .getAsJsonObject("object-data").getAsJsonObject("uri")
-          .getAsJsonPrimitive("object-data").getAsString();
+      String filepath = copyEntityDeserializer.getFilePathAsStringFromJson(json);
       actual.add(filepath);
     }
-    Assert.assertEquals(actual.size(), expected.size(),
-        "Set" + actual.toString() + " vs Set" + expected.toString());
+    Assert.assertEquals(actual.size(), expected.size(), "Set" + actual.toString() + " vs Set" + expected.toString());
     Assert.assertEqualsNoOrder(actual.toArray(), expected.toArray());
   }
 
+  private void verifyFsOwnershipAndPermissionPreservation(Collection<CopyEntity> copyEntities,
+      Map<String, FileStatus> expectedMap) {
+    CopyEntityDeserializer copyEntityDeserializer = new CopyEntityDeserializer();
+
+    for (CopyEntity copyEntity : copyEntities) {
+      String copyEntityJson = copyEntity.toString();
+
+      JsonArray ancestorOwnerAndPermissions = copyEntityDeserializer.getAncestorOwnerAndPermissions(copyEntityJson);
+      String filePath = copyEntityDeserializer.getFilePathAsStringFromJson(copyEntityJson);
+      JsonObject destinationOwnerAndPermissions =
+          copyEntityDeserializer.getDestinationOwnerAndPermissions(copyEntityJson);
+
+      FileStatus expected = expectedMap.get(filePath);
+      Map<String, String> actual =
+          copyEntityDeserializer.getDestinationOwnerAndPermissionsFsPermissionsMap(destinationOwnerAndPermissions);
+
+      Assert.assertEquals(ancestorOwnerAndPermissions.size(), new Path(filePath).getParent().depth() - 1);
+      Assert.assertEquals(expected.getPermission().getUserAction().toString(), actual.get("useraction"));
+      Assert.assertEquals(expected.getPermission().getGroupAction().toString(), actual.get("groupaction"));
+      Assert.assertEquals(expected.getPermission().getOtherAction().toString(), actual.get("otheraction"));
+    }
+  }
+
+  private FileStatus createFileStatus(String pathString, String owner, String group, FsPermission fsPermission) {
+    Path path = new Path(pathString);
+    return new FileStatus(0, false, 0, 0, 0, 0, fsPermission, owner, group, path);
+  }
 
   /**
    *  Sadly, this is needed to avoid losing `FileSystem` mock to replacement from the `FileSystem.get` `static`
    *  Without this, so to lose the mock, we'd be unable to set up any source paths as existing.
    */
   protected static class TrickIcebergDataset extends IcebergDataset {
-    public TrickIcebergDataset(String db, String table, IcebergTable icebergTbl, Properties properties, FileSystem sourceFs) {
+    public TrickIcebergDataset(String db, String table, IcebergTable icebergTbl, Properties properties,
+        FileSystem sourceFs) {
       super(db, table, icebergTbl, properties, sourceFs);
     }
 
     @Override // as the `static` is not mock-able
-    protected FileSystem getSourceFileSystemFromFileStatus(FileStatus fileStatus, Configuration hadoopConfig) throws IOException {
+    protected FileSystem getSourceFileSystemFromFileStatus(FileStatus fileStatus, Configuration hadoopConfig)
+        throws IOException {
       return this.sourceFs;
     }
-  };
+  }
 
+  ;
 
   /** Builds a {@link FileSystem} mock */
   protected static class MockFileSystemBuilder {
     private final URI fsURI;
     /** when not `.isPresent()`, all paths exist; when `.get().isEmpty()`, none exist; else only those indicated do */
-    private final Optional<Set<Path>> optPaths;
+    //private final Optional<Set<Path>> optPaths; // convert it to a map<path, filestatus>
+    private final Optional<Map<Path, FileStatus>> optPathsWithFileStatuses;
 
     public MockFileSystemBuilder(URI fsURI) {
       this(fsURI, false);
     }
 
     public MockFileSystemBuilder(URI fsURI, boolean shouldRepresentEveryPath) {
       this.fsURI = fsURI;
-      this.optPaths = shouldRepresentEveryPath ? Optional.empty() : Optional.of(Sets.newHashSet());
-    }
-
-    public Optional<Set<Path>> getPaths() {
-      return this.optPaths.map(Sets::newHashSet); // copy before returning
+      //this.optPaths = shouldRepresentEveryPath ? Optional.empty() : Optional.of(Sets.newHashSet());
+      this.optPathsWithFileStatuses = shouldRepresentEveryPath ? Optional.empty() : Optional.of(Maps.newHashMap());
     }
 
-    public void addPaths(List<String> pathStrings) {
-      for (String pathString : pathStrings) {
-        addPath(pathString);
+    public void addPathsAndFileStatuses(Map<String, FileStatus> pathStringToFsPermissionsMap) {
+      for (Map.Entry<String, FileStatus> entry : pathStringToFsPermissionsMap.entrySet()) {
+        String path = entry.getKey();
+        FileStatus fileStatus = entry.getValue();
+        addPathAndFileStatus(path, fileStatus);
       }
     }
 
-    public void addPath(String pathString) {
-      addPath(new Path(pathString));
+    public void addPathAndFileStatus(String pathString, FileStatus fileStatus) {
+      addPathAndFileStatus(new Path(pathString), fileStatus);
     }
 
-    public void addPath(Path path) {
-      if (!this.optPaths.isPresent()) {
+    public void addPathAndFileStatus(Path path, FileStatus fileStatus) {
+      if (!this.optPathsWithFileStatuses.isPresent()) {
         throw new IllegalStateException("unable to add paths when constructed with `shouldRepresentEveryPath == true`");
       }
-      if (this.optPaths.get().add(path) && !path.isRoot()) { // recursively add ancestors of a previously unknown path
-        addPath(path.getParent());
+      optPathsWithFileStatuses.get().putIfAbsent(path, fileStatus);
+      if (!path.isRoot()) { // recursively add ancestors of a previously unknown path
+        addPathAndFileStatus(path.getParent(), fileStatus);
       }
     }
 
-    public FileSystem build() throws IOException {
+    public FileSystem build()
+        throws IOException {
       FileSystem fs = Mockito.mock(FileSystem.class);
       Mockito.when(fs.getUri()).thenReturn(fsURI);
       Mockito.when(fs.makeQualified(any(Path.class)))
           .thenAnswer(invocation -> invocation.getArgumentAt(0, Path.class).makeQualified(fsURI, new Path("/")));
 
-      if (!this.optPaths.isPresent()) {
-        Mockito.when(fs.getFileStatus(any(Path.class))).thenAnswer(invocation ->
-            createEmptyFileStatus(invocation.getArgumentAt(0, Path.class).toString()));
+      if (!this.optPathsWithFileStatuses.isPresent()) {
+        Mockito.when(fs.getFileStatus(any(Path.class)))
+            .thenAnswer(invocation -> createEmptyFileStatus(invocation.getArgumentAt(0, Path.class).toString()));
       } else {
         // WARNING: order is critical--specific paths *after* `any(Path)`; in addition, since mocking further
         // an already-mocked instance, `.doReturn/.when` is needed (vs. `.when/.thenReturn`)
         Mockito.when(fs.getFileStatus(any(Path.class))).thenThrow(new FileNotFoundException());
-        for (Path p : this.optPaths.get()) {
-          Mockito.doReturn(createEmptyFileStatus(p.toString())).when(fs).getFileStatus(p);
+        for (Path p : this.optPathsWithFileStatuses.get().keySet()) {
+          FileStatus fileStatus = this.optPathsWithFileStatuses.get().get(p);
+          if (fileStatus != null) {
+            Mockito.doReturn(fileStatus).when(fs).getFileStatus(p);
+          } else {
+            Mockito.doReturn(createEmptyFileStatus(p.toString())).when(fs).getFileStatus(p);

Review Comment:
   NBD, but FYI what I meant was:
   ```
   Mockito.doReturn(fileStatus != null ? fileStatus : createEmptyFileStatus(p.toString())).when(fs).getFileStatus(p);
   ```



##########
gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/iceberg/IcebergDatasetTest.java:
##########
@@ -307,105 +329,137 @@ protected IcebergTable validateGetFilePathsGivenDestState(
   /** @return `paths` after adding to it all paths of every one of `snapshotDefs` */
   protected static Set<Path> withAllSnapshotPaths(Set<Path> paths, MockIcebergTable.SnapshotPaths... snapshotDefs) {
     Arrays.stream(snapshotDefs).flatMap(snapshotDef ->
-        snapshotDef.asSnapshotInfo().getAllPaths().stream()
-    ).forEach(p ->
-        paths.add(new Path(p))
-    );
+            snapshotDef.asSnapshotInfo().getAllPaths().stream())
+        .forEach(p ->
+            paths.add(new Path(p))
+        );
     return paths;
   }
 
   private CopyConfiguration createEmptyCopyConfiguration(FileSystem fs) {
-    return CopyConfiguration.builder(fs, copyConfigProperties)
-        .copyContext(new CopyContext())
-        .build();
+    return CopyConfiguration.builder(fs, copyConfigProperties).copyContext(new CopyContext()).build();
   }
 
   private static void verifyCopyEntities(Collection<CopyEntity> copyEntities, List<String> expected) {
-    List<String> actual = new ArrayList<>();
+    Set<String> actual = new HashSet<>();

Review Comment:
   I see this changed... did you find there might be duplicates--and more importantly that our tests ought to ignore rather than call out as a regression?



##########
gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/iceberg/IcebergDatasetTest.java:
##########
@@ -246,52 +246,74 @@ public void testGenerateCopyEntitiesMultiSnapshotWhenDestEmpty() throws IOExcept
         MANIFEST_LIST_PATH_1, MANIFEST_PATH_1, MANIFEST_DATA_PATH_1A, MANIFEST_DATA_PATH_1B);
 
     MockFileSystemBuilder sourceBuilder = new MockFileSystemBuilder(SRC_FS_URI);
-    sourceBuilder.addPaths(expectedPaths);
+    sourceBuilder.addPathsAndFileStatuses(expectedPaths, false);
     FileSystem sourceFs = sourceBuilder.build();
 
     IcebergTable icebergTable = MockIcebergTable.withSnapshots(Arrays.asList(SNAPSHOT_PATHS_1, SNAPSHOT_PATHS_0));
-    IcebergDataset icebergDataset = new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
+    IcebergDataset icebergDataset =
+        new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
 
     MockFileSystemBuilder destBuilder = new MockFileSystemBuilder(DEST_FS_URI);
     FileSystem destFs = destBuilder.build();
 
-    CopyConfiguration copyConfiguration = CopyConfiguration.builder(destFs, copyConfigProperties)
-        .preserve(PreserveAttributes.fromMnemonicString(""))
-        .copyContext(new CopyContext())
-        .build();
+    CopyConfiguration copyConfiguration =
+        CopyConfiguration.builder(destFs, copyConfigProperties).preserve(PreserveAttributes.fromMnemonicString(""))
+            .copyContext(new CopyContext()).build();
     Collection<CopyEntity> copyEntities = icebergDataset.generateCopyEntities(destFs, copyConfiguration);
     verifyCopyEntities(copyEntities, expectedPaths);
   }
 
+  @Test
+  public void testFsOwnershipAndPermissionPreservationWhenDestEmpty() throws IOException {
+    List<String> expectedPaths = Arrays.asList(METADATA_PATH, MANIFEST_LIST_PATH_0,
+        MANIFEST_PATH_0, MANIFEST_DATA_PATH_0A, MANIFEST_DATA_PATH_0B);
+    MockFileSystemBuilder sourceBuilder = new MockFileSystemBuilder(SRC_FS_URI);
+    sourceBuilder.addPathsAndFileStatuses(expectedPaths, true);
+    FileSystem sourceFs = sourceBuilder.build();
+
+    IcebergTable icebergTable = MockIcebergTable.withSnapshots(Arrays.asList(SNAPSHOT_PATHS_0));
+    IcebergDataset icebergDataset = new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
+
+    MockFileSystemBuilder destBuilder = new MockFileSystemBuilder(DEST_FS_URI);
+    FileSystem destFs = destBuilder.build();
+
+    CopyConfiguration copyConfiguration =
+        CopyConfiguration.builder(destFs, copyConfigProperties)
+            // preserving attributes for owner, group and permissions respectively
+            .preserve(PreserveAttributes.fromMnemonicString("ugp"))
+            .copyContext(new CopyContext()).build();
+
+    Collection<CopyEntity> copyEntities = icebergDataset.generateCopyEntities(destFs, copyConfiguration);
+    Map<Path, FileStatus> expectedPathsAndFileStatuses = sourceBuilder.getPathsAndFileStatuses();
+    verifyFsOwnershipAndPermissionPreservation(copyEntities, expectedPathsAndFileStatuses);
+  }
+
   /**
    *  exercise {@link IcebergDataset::getFilePaths} and validate the result
    *  @return {@link IcebergTable} (mock!), for behavioral verification
    */
-  protected IcebergTable validateGetFilePathsGivenDestState(
-      List<MockIcebergTable.SnapshotPaths> sourceSnapshotPathSets,
-      List<String> existingDestPaths,
-      Set<Path> expectedResultPaths) throws IOException {
-    return validateGetFilePathsGivenDestState(sourceSnapshotPathSets, Optional.empty(),existingDestPaths, expectedResultPaths);
+  protected IcebergTable validateGetFilePathsGivenDestState(List<MockIcebergTable.SnapshotPaths> sourceSnapshotPathSets,
+      List<String> existingDestPaths, Set<Path> expectedResultPaths) throws IOException {
+    return validateGetFilePathsGivenDestState(sourceSnapshotPathSets, Optional.empty(), existingDestPaths,
+        expectedResultPaths);
   }
 
   /**
    *  exercise {@link IcebergDataset::getFilePaths} and validate the result
    *  @return {@link IcebergTable} (mock!), for behavioral verification
    */
-  protected IcebergTable validateGetFilePathsGivenDestState(
-      List<MockIcebergTable.SnapshotPaths> sourceSnapshotPathSets,
-      Optional<List<String>> optExistingSourcePaths,
-      List<String> existingDestPaths,
-      Set<Path> expectedResultPaths) throws IOException {
+  protected IcebergTable validateGetFilePathsGivenDestState(List<MockIcebergTable.SnapshotPaths> sourceSnapshotPathSets,
+      Optional<List<String>> optExistingSourcePaths, List<String> existingDestPaths, Set<Path> expectedResultPaths) throws IOException {
     IcebergTable icebergTable = MockIcebergTable.withSnapshots(sourceSnapshotPathSets);
 
     MockFileSystemBuilder sourceFsBuilder = new MockFileSystemBuilder(SRC_FS_URI, !optExistingSourcePaths.isPresent());
-    optExistingSourcePaths.ifPresent(sourceFsBuilder::addPaths);
+    optExistingSourcePaths.ifPresent(sourceFsBuilder :: addPaths);

Review Comment:
   seems incorrect... why add whitespace around `::`... we wouldn't do that w/ `.`, would we?



##########
gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/iceberg/IcebergDatasetTest.java:
##########
@@ -246,52 +246,74 @@ public void testGenerateCopyEntitiesMultiSnapshotWhenDestEmpty() throws IOExcept
         MANIFEST_LIST_PATH_1, MANIFEST_PATH_1, MANIFEST_DATA_PATH_1A, MANIFEST_DATA_PATH_1B);
 
     MockFileSystemBuilder sourceBuilder = new MockFileSystemBuilder(SRC_FS_URI);
-    sourceBuilder.addPaths(expectedPaths);
+    sourceBuilder.addPathsAndFileStatuses(expectedPaths, false);
     FileSystem sourceFs = sourceBuilder.build();
 
     IcebergTable icebergTable = MockIcebergTable.withSnapshots(Arrays.asList(SNAPSHOT_PATHS_1, SNAPSHOT_PATHS_0));
-    IcebergDataset icebergDataset = new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
+    IcebergDataset icebergDataset =
+        new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
 
     MockFileSystemBuilder destBuilder = new MockFileSystemBuilder(DEST_FS_URI);
     FileSystem destFs = destBuilder.build();
 
-    CopyConfiguration copyConfiguration = CopyConfiguration.builder(destFs, copyConfigProperties)
-        .preserve(PreserveAttributes.fromMnemonicString(""))
-        .copyContext(new CopyContext())
-        .build();
+    CopyConfiguration copyConfiguration =
+        CopyConfiguration.builder(destFs, copyConfigProperties).preserve(PreserveAttributes.fromMnemonicString(""))
+            .copyContext(new CopyContext()).build();
     Collection<CopyEntity> copyEntities = icebergDataset.generateCopyEntities(destFs, copyConfiguration);
     verifyCopyEntities(copyEntities, expectedPaths);
   }
 
+  @Test
+  public void testFsOwnershipAndPermissionPreservationWhenDestEmpty() throws IOException {
+    List<String> expectedPaths = Arrays.asList(METADATA_PATH, MANIFEST_LIST_PATH_0,
+        MANIFEST_PATH_0, MANIFEST_DATA_PATH_0A, MANIFEST_DATA_PATH_0B);
+    MockFileSystemBuilder sourceBuilder = new MockFileSystemBuilder(SRC_FS_URI);
+    sourceBuilder.addPathsAndFileStatuses(expectedPaths, true);
+    FileSystem sourceFs = sourceBuilder.build();
+
+    IcebergTable icebergTable = MockIcebergTable.withSnapshots(Arrays.asList(SNAPSHOT_PATHS_0));
+    IcebergDataset icebergDataset = new TrickIcebergDataset(testDbName, testTblName, icebergTable, new Properties(), sourceFs);
+
+    MockFileSystemBuilder destBuilder = new MockFileSystemBuilder(DEST_FS_URI);
+    FileSystem destFs = destBuilder.build();
+
+    CopyConfiguration copyConfiguration =
+        CopyConfiguration.builder(destFs, copyConfigProperties)
+            // preserving attributes for owner, group and permissions respectively
+            .preserve(PreserveAttributes.fromMnemonicString("ugp"))
+            .copyContext(new CopyContext()).build();

Review Comment:
   missing is a counter-test: that we avoid preserving attributes we're not asked to preserve



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org