Skip to content

Conversation

@smengcl
Copy link
Contributor

@smengcl smengcl commented Nov 19, 2022

What changes were proposed in this pull request?

This is a follow-up to #3824. See jira HDDS-7508 description.

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/HDDS-7508

How was this patch tested?

  • Integration test TestOMSnapshotDAG.

…Node SnapshotID field debugging functionality

Change-Id: If85e8a5d00b4f1a8575d44b48e2630d039133d7f
…ore `FileAlreadyExistsException` in listener.
Change-Id: Iab7915677387df77a80a1448464e03ec951b676f
Change-Id: I46aa460b19ed912cde90d48264ad471e636d31cc
Change-Id: I5b9877bc329ef97a9c6628f419b572121f8fe8b8
@kerneltime
Copy link
Contributor

@GeorgeJahad @neils-dev

@kerneltime kerneltime added the snapshot https://issues.apache.org/jira/browse/HDDS-6517 label Nov 21, 2022
@kerneltime kerneltime requested a review from neils-dev November 21, 2022 17:38
// The result is deterministic. Retrieved from a run.
// final List<String> expectedDiffList21 = Arrays.asList(
// "000066", "000080", "000087", "000073", "000095");
// Assertions.assertEquals(expectedDiffList21, actualDiffList21);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this test no longer valid?

Copy link
Contributor Author

@smengcl smengcl Nov 22, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm going to remove this. It appears that once I set the DB column family write buffer size to a small enough value like we have here (now 256 KB, was 128 MB), the flush and compaction start to behave slightly differently on different architectures/environments. It gives a different set of results in the CI (x86, iirc) than on my Mac (arm64).

This indicates some other factors might be impacting the list of SSTs generated here now that write buffer size might no longer be the dominant factor triggering compactions. Could be some default RocksDB tunable differences for each platform? (This might be possible as each platform does have its own native JNI lib in the rocksdbjni jar.) Or simply because the detected system memory and # of CPU cores difference are causing some RDB compaction-relevant defaults to be different.

At least it seems the result here is stable if the environment stays the same. Two runs from a previous commit:

https://github.com/smengcl/hadoop-ozone/actions/runs/3502672241/jobs/5867215890#step:5:3353

https://github.com/apache/ozone/actions/runs/3502681795/jobs/5867234494#step:5:3350

Error:    TestOMSnapshotDAG.testZeroSizeKey:218 expected: <[000066, 000080, 000087, 000073, 000095]> but was: <[000067, 000060]>

// works.
List<String> actualDiffList32 = differ.getSSTDiffList(snap3, snap2);
// final List<String> expectedDiffList32 = Arrays.asList("000105");
// Assertions.assertEquals(expectedDiffList32, actualDiffList32);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this test no longer valid?

@GeorgeJahad
Copy link
Contributor

These methods don't appear to be used. Can they be removed from TestRocksDBCheckpointDiffer.java?

updateRocksDBInstance
testDefaultColumnFamilyOriginal
testCertainColumnFamily

@GeorgeJahad
Copy link
Contributor

Are these methods still needed?:
traverseGraph
printMutableGraphFromAGivenNode
printMutableGraph

They seem like they might be useful tools, so it might be good to keep them, but they don't appear to be checked by any tests. So there is no incentive to keep maintaining them, or awareness of when they've stopped working.

@GeorgeJahad
Copy link
Contributor

It seems like internalGetSSTDiffList() is the heart of the compaction diff code. But it's not clear the tests fully excercise it. It contains 5 or 6 different if statements to handle all the different possible cases, and I can't tell whether the tests hit each of those cases.

@smengcl
Copy link
Contributor Author

smengcl commented Nov 22, 2022

These methods don't appear to be used. Can they be removed from TestRocksDBCheckpointDiffer.java?

updateRocksDBInstance testDefaultColumnFamilyOriginal testCertainColumnFamily

Thanks. Will remove those unused test helper methods.

Change-Id: Id995eea1270ec40ba9b05c2e761c748f5cdc5965
Change-Id: Idd994ea9d535453ab783adad640847b57524abce
Change-Id: I6a52aad6b6d4a941c6bc70f0468a85e3aa0083fc
@smengcl
Copy link
Contributor Author

smengcl commented Nov 22, 2022

Are these methods still needed?: traverseGraph printMutableGraphFromAGivenNode printMutableGraph

They seem like they might be useful tools, so it might be good to keep them, but they don't appear to be checked by any tests. So there is no incentive to keep maintaining them, or awareness of when they've stopped working.

Make sense. Will move them to the test class.

… test class.

Change-Id: I44f93ada5ce66aa0ecf4899fff4d9e2c1c616a3d
Change-Id: Id16f497f83977a3b75dc6aa296aa46eb0da19a14
Change-Id: I060f464692af4f18bff9261a521bb88f2f40c921
@prashantpogde
Copy link
Contributor

LGTM. But I suspect the result can be different if the RocksDB behavior changes. We should have a way to directly call the
rocksDB compaction lister and generate the deterministic DAG that we want to generate.Such a DAG traversal can give a constant output result.
I guess the current PR can go in as it is. But this is the idea for a next PR that will be very deterministic.

+ "C 000093,000090,000083:\n" // Deletion ?
+ "S 14980 e7ad72f8-52df-4430-93f6-0ee91d4a47fd\n" // Snapshot 2
+ "C 000098,000096,000085,000078,000071,000064,000060,000052:000099\n"
+ "C 000105,000095,000088:000107\n"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This line is right before the creation of snapshot 3, and compacts 3 sst files into "107". But in snapSST3 below I don't see 107 and do see the precompacted files. Am I misunderstanding something?

Copy link
Contributor Author

@smengcl smengcl Nov 29, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm. I took the compaction log from TestOMSnapshotDAG where I set a breakpoint right before OM is restarted (cluster.restartOzoneManager()).

It looks like this compaction is indeed not captured in snapshot 3. As I checked the active DB, it seems the compaction happens after snapshot 3 is taken.

So it seems when the compaction log is being appended (even in onCompactionCompleted), RocksDB hasn't really completed the flush (and updated the manifest). Thus, the immediately following DB checkpoint operation wouldn't pick it up.

Copy link
Contributor Author

@smengcl smengcl Nov 29, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Though this (unexpected ordering) alone shouldn't affect the correctness of the differ since we currently read the whole compaction log and reconstruct the DAG upon OM restarts, I'm curious what is actually happening.

Change-Id: I541719b0d4eeb0ec9a9a428c46624291a9702fc9
Change-Id: I0313625e930d68461a9fb718553b28f1c5011e00
Change-Id: I1a891f80b707d20607891f01391461f36f0bcd4b
@GeorgeJahad
Copy link
Contributor

lgtm (I made a few final comments above.)

Copy link
Contributor

@hemantk-12 hemantk-12 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

General comment:
I don't understand the purpose of having debug logs in tests. Test name, scenarios and comments (if needed) should be enough to tell what and why test is doing. You can add log or print statements when writing test but should be removed before sent for review.

That's my understanding.

// Name of the SST file
private final String fileName;
// The last snapshot created before this node came into existence
private final String snapshotId;
Copy link
Contributor

@hemantk-12 hemantk-12 Dec 1, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm skeptical about this. I don't understand its usage and it might not provide correct information. Let's say node A gets added because of snapshot-1 and it is used by snapshot-1, snapshot-2, ... snapshot-10. Now DAG pruning kicks in and deletes Snapshot-1 to Snapshot-5 but node A can't be deleted because it is used by Snapshot-6 to Snapshot-10 (That's my understanding). What would be the SnapshotID of the node in this case? Snapshot-1's ID or Snapshot-6's ID? I don't think Snapshot-1's ID is correct because Snapshot-1 doesn't exist in DAG.

Copy link
Contributor Author

@smengcl smengcl Dec 5, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What difference does it make from fileName if it can be any random string? Why can't we just use fileName?

Copy link
Contributor Author

@smengcl smengcl Dec 6, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah. Originally it was used by @prashantpogde to provide extra info when drawing the DAG for debugging. SST file name alone doesn't tell which checkpoint captures the SST. I broke it with my last PR and is just restoring that here.

TBH we don't even need to fill in snapshotId when debugging is disabled. Though it shouldn't take too much memory to store that.

private final long totalNumberOfKeys;
private long cumulativeKeysReverseTraversal;

CompactionNode(String file, String ssId, long numKeys, long seqNum) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is it package private?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As it was an internal class in RocksDBCheckpointDiffer. It is only used in the differ (for now). We can make it public later if used in other packages.

Is there a strong reason on your mind to make it public atm?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Don't have very strong reason to make it public. You can keep it as it is.

To me, it is a simple data class. Don't see any issue if it is made public.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

this.cumulativeKeysReverseTraversal = cumulativeKeysReverseTraversal;
}

public void addCumulativeKeysReverseTraversal(long diff) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Curious if it needs to be thread safe?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think not, at least for now. It is only used in the test for traverseGraph.

* @param writeBufferSize Specify column family write buffer size.
* @return ManagedColumnFamilyOptions
*/
private ManagedColumnFamilyOptions getDefaultCfOptions(long writeBufferSize) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. I think getDefaultCfOption() and getDefaultCfOptions(long writeBufferSize) are unnecessary one liner functions. Also CfOptions returned by getDefaultCfOptions(long writeBufferSize) is not default anymore.

  2. This function is getting invoked multiple unnecessarily at line # 287 and then inside loop # 297. CfOptions can be store in a variable and used at both the places.

I'll prefer:

  private Set<TableConfig> makeTableConfigs() {
    Set<TableConfig> tableConfigs = new HashSet<>();

    ManagedColumnFamilyOptions columnFamilyOptions =
        Optional.ofNullable(defaultCfOptions)
            .orElseGet(defaultCfProfile::getColumnFamilyOptions);
    columnFamilyOptions.setWriteBufferSize(rocksDbCfWriteBufferSize);

    // If default column family was not added, add it with the default options.
    cfOptions.putIfAbsent(DEFAULT_COLUMN_FAMILY_NAME, columnFamilyOptions);

    for (Map.Entry<String, ManagedColumnFamilyOptions> entry:
        cfOptions.entrySet()) {
      String name = entry.getKey();
      ManagedColumnFamilyOptions options = entry.getValue();

      if (options == null) {
        LOG.debug("using default column family options for table: {}", name);
        tableConfigs.add(new TableConfig(name, columnFamilyOptions));
      } else {
        tableConfigs.add(new TableConfig(name, options));
      }
    }

    return tableConfigs;
  }

If you really want a function, it could be one function only:

  private Set<TableConfig> makeTableConfigs() {
    Set<TableConfig> tableConfigs = new HashSet<>();

    ManagedColumnFamilyOptions columnFamilyOptions =
        getCfOptions(rocksDbCfWriteBufferSize);

    // If default column family was not added, add it with the default options.
    cfOptions.putIfAbsent(DEFAULT_COLUMN_FAMILY_NAME, columnFamilyOptions);

    for (Map.Entry<String, ManagedColumnFamilyOptions> entry:
        cfOptions.entrySet()) {
      String name = entry.getKey();
      ManagedColumnFamilyOptions options = entry.getValue();

      if (options == null) {
        LOG.debug("using default column family options for table: {}", name);
        tableConfigs.add(new TableConfig(name, columnFamilyOptions));
      } else {
        tableConfigs.add(new TableConfig(name, options));
      }
    }

    return tableConfigs;
  }

  /**
   * Get default column family options, but with column family write buffer
   * size limit overridden.
   */
  private ManagedColumnFamilyOptions getCfOptions(long writeBufferSize) {
    ManagedColumnFamilyOptions cfOptions = Optional.ofNullable(defaultCfOptions)
        .orElseGet(defaultCfProfile::getColumnFamilyOptions);
    cfOptions.setWriteBufferSize(writeBufferSize);
    return cfOptions;
  }

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'd prefer delaying any further refactoring of a shared class like DBStoreBuilder until we merge to master to prevent headaches around merge conflicts. Would you like to file a jira for this so we can fix this after the merge?

It is still the "default" in the sense that we are still getting all the other default CF options. Just with write buffer size explicitly overridden, as mentioned in the javadoc. Similar to getDefaultDBOptions(tableConfigs) right below.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I will disagree that it is still the "default" but I'll leave it upto you.

I think the real confusion here is if you are creating default config or using default config when config is absent. We are doing the later, using default config when config is not provided.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Renamed to getCfOptions

// RDB compaction. Take another snapshot and do the diff again.
// Then restart OM, do the same diff again to see if DAG reconstruction
// works.
List<String> sstDiffList32 = differ.getSSTDiffList(snap3, snap2);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. I believe these test cases can be changed to parameterized tests.
  2. Why test is package private not public? What is testZeroSizeKey test actually testing? testZeroSizeKey name doesn't give me any insight of what it is testing.

Copy link
Contributor Author

@smengcl smengcl Dec 6, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. I don't see how we would change the existing checks one dependent on another into parameterized test here?
  2. testZeroSizeKey name was originally taken from existing freon test:

Renamed. Key size would stay zero as the test doesn't need to involve DNs.

Change-Id: I37916276a53bb6fa36021c72da5df5121738d945
Change-Id: I5b899e34f402b85b24ddaf73f2e2dddf17c9a7cc
Change-Id: I34dacd837e36f497c64ae90b7f9dee3c0e99abad
Change-Id: I0b009a14a3a89d748ad7eae3dd684b004821f201
Copy link
Contributor

@hemantk-12 hemantk-12 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

Thanks @smengcl for the patch and addressing all the comments.

Conflicts:
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java

Change-Id: I92a09d4bad91bd91a59cbb6878d0417b417ff37c
@smengcl smengcl merged commit 2554ec2 into apache:HDDS-6517-Snapshot Dec 7, 2022
@smengcl
Copy link
Contributor Author

smengcl commented Dec 7, 2022

Thanks @GeorgeJahad @prashantpogde @hemantk-12 for reviewing this.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

snapshot https://issues.apache.org/jira/browse/HDDS-6517

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants