You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/10/31 18:06:31 UTC

[GitHub] [ozone] hemantk-12 commented on a diff in pull request #3885: HDDS-7410. Ozone snapshot diff skeleton code.

hemantk-12 commented on code in PR #3885:
URL: https://github.com/apache/ozone/pull/3885#discussion_r1008589835


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdb/util/ManagedSSTFileReader.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.ozone.rocksdb.util;
+
+import org.rocksdb.Options;
+import org.rocksdb.ReadOptions;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.SstFileReader;
+import org.rocksdb.SstFileReaderIterator;
+
+import java.io.Closeable;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Spliterator;
+import java.util.Spliterators;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+
+public class ManagedSSTFileReader {
+
+  private final Collection<String> sstFiles;
+
+  public ManagedSSTFileReader(final Collection<String> sstFiles) {
+    this.sstFiles = sstFiles;
+  }
+  public Stream<String> getKeyStream() throws RocksDBException {
+    final OMSSTFileIterator iterator = new OMSSTFileIterator(sstFiles);
+    final Spliterator<String> spliterator = Spliterators
+        .spliteratorUnknownSize(iterator, 0);
+    return StreamSupport.stream(spliterator, false).onClose(iterator::close);
+  }
+
+  public interface ClosableIterator<T> extends Iterator<T>, Closeable {}

Review Comment:
   What's the purpose of this interface? Why can't `OMSSTFileIterator` directly implement `Iterator` and `Closeable`?



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdb/util/ManagedSSTFileReader.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.ozone.rocksdb.util;
+
+import org.rocksdb.Options;
+import org.rocksdb.ReadOptions;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.SstFileReader;
+import org.rocksdb.SstFileReaderIterator;
+
+import java.io.Closeable;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Spliterator;
+import java.util.Spliterators;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+
+public class ManagedSSTFileReader {

Review Comment:
   nit/suggestion: I think `PascalCase` should be used for acronyms otherwise it becomes difficult to read when multiple acronyms put together. For example, `OMSSTFileIterator` should be `OmSstFileIterator` and `ManagedSSTFileReader` should be `ManagedSstFileReader`. Also in parity of existing stuff `OmSnapshot` and `OmSnapshotManager`. 



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.ozone.om.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.snapshot.SnapshotDiffReport.DiffType;
+import org.apache.hadoop.ozone.om.snapshot.SnapshotDiffReport.DiffReportEntry;
+
+import org.apache.ozone.rocksdb.util.ManagedSSTFileReader;
+import org.rocksdb.RocksDBException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Stream;
+
+public class SnapshotDiffManager {
+
+  public SnapshotDiffReport getSnapshotDiffReport(final String volume,
+                                                  final String bucket,
+                                                  final OmSnapshot fromSnapshot,
+                                                  final OmSnapshot toSnapshot)
+      throws IOException, RocksDBException {
+
+    // TODO: Once RocksDBCheckpointDiffer exposes method to get list
+    //  of delta SST files, plug it in here.
+
+    final Set<String> deltaFiles = Collections.emptySet();
+
+    // TODO: Filter out the files.
+
+    final Stream<String> keysToCheck = new ManagedSSTFileReader(deltaFiles)
+        .getKeyStream();
+
+    final BucketLayout bucketLayout = getBucketLayout(volume, bucket,
+        fromSnapshot.getMetadataManager());
+
+    final Table<String, OmKeyInfo> fsKeyTable = fromSnapshot
+        .getMetadataManager().getKeyTable(bucketLayout);
+    final Table<String, OmKeyInfo> tsKeyTable = toSnapshot
+        .getMetadataManager().getKeyTable(bucketLayout);
+
+    /*
+     * The reason for having ObjectID to KeyName mapping instead of OmKeyInfo
+     * is to reduce the memory footprint.
+     */
+    final Map<Long, String> oldObjIdToKeyMap = new HashMap<>();
+    // Long --> const. length
+    // String --> var. length "/dir1/dir2/dir3/dir4/dir5/key1"
+    final Map<Long, String> newObjIdToKeyMap = new HashMap<>();
+
+    final Set<Long> objectIDsToCheck = new HashSet<>();
+
+    keysToCheck.forEach(key -> {
+      try {
+        final OmKeyInfo oldKey = fsKeyTable.get(key);
+        final OmKeyInfo newKey = tsKeyTable.get(key);
+        if (areKeysEqual(oldKey, newKey)) {
+          // We don't have to do anything.
+          return;
+        }
+        if (oldKey != null) {
+          final long oldObjId = oldKey.getObjectID();
+          oldObjIdToKeyMap.put(oldObjId, oldKey.getKeyName());
+          objectIDsToCheck.add(oldObjId);
+        }
+        if (newKey != null) {
+          final long newObjId = newKey.getObjectID();
+          newObjIdToKeyMap.put(newObjId, newKey.getKeyName());
+          objectIDsToCheck.add(newObjId);
+        }
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    });
+    keysToCheck.close();
+
+    return new SnapshotDiffReport(volume, bucket, fromSnapshot.getName(),
+        toSnapshot.getName(), generateDiffReport(objectIDsToCheck,
+        oldObjIdToKeyMap, newObjIdToKeyMap));
+  }
+
+  private List<DiffReportEntry> generateDiffReport(
+      final Set<Long> objectIDsToCheck,
+      final Map<Long, String> oldObjIdToKeyMap,
+      final Map<Long, String> newObjIdToKeyMap) {
+
+    final List<DiffReportEntry> deleteDiffs = new ArrayList<>();
+    final List<DiffReportEntry> renameDiffs = new ArrayList<>();
+    final List<DiffReportEntry> createDiffs = new ArrayList<>();
+    final List<DiffReportEntry> modifyDiffs = new ArrayList<>();
+
+
+    for (Long id : objectIDsToCheck) {
+      /*
+       * This key can be
+       * -> Created after the old snapshot was taken, which means it will be
+       *    missing in oldKeyTable and present in newKeyTable.
+       * -> Deleted after the old snapshot was taken, which means it will be
+       *    present in oldKeyTable and missing in newKeyTable.
+       * -> Modified after the old snapshot was taken, which means it will be
+       *    present in oldKeyTable and present in newKeyTable with same
+       *    Object ID but with different metadata.
+       * -> Renamed after the old snapshot was taken, which means it will be
+       *    present in oldKeyTable and present in newKeyTable but with different
+       *    name and same Object ID.
+       */
+
+      final String oldKeyName = oldObjIdToKeyMap.get(id);
+      final String newKeyName = newObjIdToKeyMap.get(id);
+
+      if (oldKeyName == null && newKeyName == null) {
+        // This cannot happen.
+        continue;
+      }
+
+      // Key Created.
+      if (oldKeyName == null) {
+        createDiffs.add(DiffReportEntry.of(DiffType.CREATE, newKeyName));
+        continue;
+      }
+
+      // Key Deleted.
+      if(newKeyName == null) {
+        deleteDiffs.add(DiffReportEntry.of(DiffType.DELETE, oldKeyName));
+        continue;
+      }
+
+      // Key modified.
+      if(oldKeyName.equals(newKeyName)) {
+        modifyDiffs.add(DiffReportEntry.of(DiffType.MODIFY, newKeyName));
+        continue;
+      }
+
+      // Key Renamed.
+      renameDiffs.add(DiffReportEntry.of(DiffType.RENAME,
+          oldKeyName, newKeyName));
+    }
+    /*
+     * The order in which snap-diff should be applied
+     *
+     *     1. Delete diffs
+     *     2. Rename diffs
+     *     3. Create diffs
+     *     4. Modified diffs
+     *
+     * Consider the following scenario
+     *
+     *    1. File "A" is created.
+     *    2. File "B" is created.
+     *    3. File "C" is created.
+     *    Snapshot "1" is taken.
+     *
+     * Case 1:
+     *   1. File "A" is deleted.
+     *   2. File "B" is renamed to "A".
+     *   Snapshot "2" is taken.
+     *
+     *   Snapshot diff should be applied in the following order:
+     *    1. Delete "A"
+     *    2. Rename "B" to "A"
+     *
+     *
+     * Case 2:
+     *    1. File "B" is renamed to "C".
+     *    2. File "B" is created.
+     *    Snapshot "2" is taken.
+     *
+     *   Snapshot diff should be applied in the following order:
+     *    1. Rename "B" to "C"
+     *    2. Create "B"
+     *
+     */
+
+    final List<DiffReportEntry> snapshotDiffs = new ArrayList<>();
+    snapshotDiffs.addAll(deleteDiffs);
+    snapshotDiffs.addAll(renameDiffs);
+    snapshotDiffs.addAll(createDiffs);
+    snapshotDiffs.addAll(modifyDiffs);
+    return snapshotDiffs;
+  }
+
+  private BucketLayout getBucketLayout(final String volume,
+                                       final String bucket,
+                                       final OMMetadataManager mManager)
+      throws IOException {
+    final String bucketTableKey = mManager.getBucketKey(volume, bucket);
+    return mManager.getBucketTable().get(bucketTableKey).getBucketLayout();

Review Comment:
   Are `mManager.getBucketKey(volume, bucket)` and `mManager.getBucketTable().get(bucketTableKey)` nullable?



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdb/util/ManagedSSTFileReader.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.ozone.rocksdb.util;
+
+import org.rocksdb.Options;
+import org.rocksdb.ReadOptions;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.SstFileReader;
+import org.rocksdb.SstFileReaderIterator;
+
+import java.io.Closeable;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Spliterator;
+import java.util.Spliterators;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+
+public class ManagedSSTFileReader {
+
+  private final Collection<String> sstFiles;
+
+  public ManagedSSTFileReader(final Collection<String> sstFiles) {
+    this.sstFiles = sstFiles;
+  }
+  public Stream<String> getKeyStream() throws RocksDBException {
+    final OMSSTFileIterator iterator = new OMSSTFileIterator(sstFiles);
+    final Spliterator<String> spliterator = Spliterators
+        .spliteratorUnknownSize(iterator, 0);
+    return StreamSupport.stream(spliterator, false).onClose(iterator::close);
+  }
+
+  public interface ClosableIterator<T> extends Iterator<T>, Closeable {}
+
+  private static class OMSSTFileIterator implements ClosableIterator<String> {
+
+    private final Iterator<String> fileNameIterator;
+    private final Options options;
+    private final ReadOptions readOptions;
+    private String currentFile;
+    private SstFileReader currentFileReader;
+    private SstFileReaderIterator currentFileIterator;
+
+    private OMSSTFileIterator(Collection<String> files) throws RocksDBException {
+      // TODO: Check if default Options and ReadOptions is enough.
+      this.options = new Options();
+      this.readOptions = new ReadOptions();
+      this.fileNameIterator = files.iterator();
+      moveToNextFile();
+    }
+
+    @Override
+    public boolean hasNext() {
+      try {
+        do {
+          if (currentFileIterator.isValid()) {
+            return true;
+          }
+        } while (moveToNextFile());
+      } catch (RocksDBException e) {
+        return false;

Review Comment:
   Is it fine to swallow exception here? I won't feel safe if it is some kind of retryable exception.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java:
##########
@@ -196,4 +203,40 @@ public static boolean isSnapshotKey(String[] keyParts) {
     return (keyParts.length > 1) &&
         (keyParts[0].compareTo(OM_SNAPSHOT_INDICATOR) == 0);
   }
-}
+
+  public SnapshotDiffReport getSnapshotDiffReport(final String volume,
+                                                  final String bucket,
+                                                  final String fromSnapshot,
+                                                  final String toSnapshot)
+      throws IOException {
+    // Validate fromSnapshot and toSnapshot
+    final SnapshotInfo fsInfo = getSnapshotInfo(volume, bucket, fromSnapshot);
+    final SnapshotInfo tsInfo = getSnapshotInfo(volume, bucket, toSnapshot);
+    verifySnapshotInfoForSnapDiff(fsInfo, tsInfo);
+
+    final String fsKey = SnapshotInfo.getTableKey(volume, bucket, fromSnapshot);
+    final String tsKey = SnapshotInfo.getTableKey(volume, bucket, toSnapshot);
+    try {
+      final OmSnapshot fs = snapshotCache.get(fsKey);
+      final OmSnapshot ts = snapshotCache.get(tsKey);
+      return snapshotDiffManager.getSnapshotDiffReport(volume, bucket, fs, ts);
+    } catch (ExecutionException | RocksDBException e) {
+      throw new IOException(e.getCause());
+    }
+  }
+
+  private void verifySnapshotInfoForSnapDiff(final SnapshotInfo fromSnapshot,
+                                             final SnapshotInfo toSnapshot)
+      throws IOException {
+    if ((fromSnapshot.getSnapshotStatus() != SnapshotStatus.SNAPSHOT_ACTIVE) ||
+        (toSnapshot.getSnapshotStatus() != SnapshotStatus.SNAPSHOT_ACTIVE)) {
+      // TODO: throw custom snapshot exception.
+      throw new IOException("Cannot generate snapshot diff for non-active " +
+          "snapshots.");
+    }
+    if (fromSnapshot.getCreationTime() > toSnapshot.getCreationTime()) {
+      throw new IOException("fromSnapshot should be older than to toSnapshot");

Review Comment:
   Add actual snapshot names in exception message if available.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java:
##########
@@ -196,4 +203,40 @@ public static boolean isSnapshotKey(String[] keyParts) {
     return (keyParts.length > 1) &&
         (keyParts[0].compareTo(OM_SNAPSHOT_INDICATOR) == 0);
   }
-}
+
+  public SnapshotDiffReport getSnapshotDiffReport(final String volume,
+                                                  final String bucket,
+                                                  final String fromSnapshot,
+                                                  final String toSnapshot)
+      throws IOException {
+    // Validate fromSnapshot and toSnapshot
+    final SnapshotInfo fsInfo = getSnapshotInfo(volume, bucket, fromSnapshot);
+    final SnapshotInfo tsInfo = getSnapshotInfo(volume, bucket, toSnapshot);
+    verifySnapshotInfoForSnapDiff(fsInfo, tsInfo);
+
+    final String fsKey = SnapshotInfo.getTableKey(volume, bucket, fromSnapshot);
+    final String tsKey = SnapshotInfo.getTableKey(volume, bucket, toSnapshot);
+    try {
+      final OmSnapshot fs = snapshotCache.get(fsKey);
+      final OmSnapshot ts = snapshotCache.get(tsKey);
+      return snapshotDiffManager.getSnapshotDiffReport(volume, bucket, fs, ts);
+    } catch (ExecutionException | RocksDBException e) {
+      throw new IOException(e.getCause());
+    }
+  }
+
+  private void verifySnapshotInfoForSnapDiff(final SnapshotInfo fromSnapshot,
+                                             final SnapshotInfo toSnapshot)
+      throws IOException {
+    if ((fromSnapshot.getSnapshotStatus() != SnapshotStatus.SNAPSHOT_ACTIVE) ||
+        (toSnapshot.getSnapshotStatus() != SnapshotStatus.SNAPSHOT_ACTIVE)) {
+      // TODO: throw custom snapshot exception.
+      throw new IOException("Cannot generate snapshot diff for non-active " +
+          "snapshots.");
+    }
+    if (fromSnapshot.getCreationTime() > toSnapshot.getCreationTime()) {
+      throw new IOException("fromSnapshot should be older than to toSnapshot");
+    }
+  }
+
+  }

Review Comment:
   nit: alignment is bit off.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffReport.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.ozone.om.snapshot;
+
+import java.util.Collections;
+import java.util.List;
+
+public class SnapshotDiffReport {
+
+  private final static String LINE_SEPARATOR = System.getProperty(
+      "line.separator", "\n");
+
+  /**
+   * Types of the difference, which include CREATE, MODIFY, DELETE, and RENAME.
+   * Each type has a label for representation:
+   * +  CREATE
+   * M  MODIFY
+   * -  DELETE
+   * R  RENAME
+   */
+  public enum DiffType {
+    CREATE("+"),
+    MODIFY("M"),
+    DELETE("-"),
+    RENAME("R");
+
+    private final String label;
+
+    DiffType(String label) {
+      this.label = label;
+    }
+
+    public String getLabel() {
+      return label;
+    }
+  }
+
+  public static class DiffReportEntry {
+
+    /**
+     * The type of diff.
+     */
+    private final DiffType type;
+
+    /**
+     * Source File/Object path.
+     */
+    private final String sourcePath;
+
+    /**
+     * Destination File/Object path, if this is a re-name operation.
+     */
+    private final String targetPath;
+
+    private DiffReportEntry(final DiffType type, final String sourcePath,
+                           final String targetPath) {

Review Comment:
   nit: alignment is off.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffReport.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.ozone.om.snapshot;
+
+import java.util.Collections;
+import java.util.List;
+
+public class SnapshotDiffReport {
+
+  private final static String LINE_SEPARATOR = System.getProperty(
+      "line.separator", "\n");
+
+  /**
+   * Types of the difference, which include CREATE, MODIFY, DELETE, and RENAME.
+   * Each type has a label for representation:
+   * +  CREATE
+   * M  MODIFY
+   * -  DELETE
+   * R  RENAME
+   */
+  public enum DiffType {
+    CREATE("+"),
+    MODIFY("M"),
+    DELETE("-"),
+    RENAME("R");
+
+    private final String label;
+
+    DiffType(String label) {
+      this.label = label;
+    }
+
+    public String getLabel() {
+      return label;
+    }
+  }
+
+  public static class DiffReportEntry {
+
+    /**
+     * The type of diff.
+     */
+    private final DiffType type;
+
+    /**
+     * Source File/Object path.
+     */
+    private final String sourcePath;
+
+    /**
+     * Destination File/Object path, if this is a re-name operation.
+     */
+    private final String targetPath;
+
+    private DiffReportEntry(final DiffType type, final String sourcePath,
+                           final String targetPath) {
+      this.type = type;
+      this.sourcePath = sourcePath;
+      this.targetPath = targetPath;
+    }
+
+    public static DiffReportEntry of(final DiffType type,
+                                     final String sourcePath) {
+      return of(type, sourcePath, null);
+    }
+
+    public static DiffReportEntry of(final DiffType type,
+                                     final String sourcePath,
+                                     final String targetPath) {
+      return new DiffReportEntry(type, sourcePath, null);

Review Comment:
   `targetPath` instead `null`.
   `DiffReportEntry(type, sourcePath, targetPath);`



-- 
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: issues-unsubscribe@ozone.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org