Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Make RocksDB instance responsible for closing associated ColumnFamilyHandle instances #7428

Closed
wants to merge 3 commits into from

Conversation

alucarded
Copy link
Contributor

@alucarded alucarded commented Sep 23, 2020

  • Takes the burden off developer to close ColumnFamilyHandle instances before closing RocksDB instance
  • The change is backward-compatible

Previously the pattern for working with Column Families was:

try (final ColumnFamilyOptions cfOpts = new ColumnFamilyOptions().optimizeUniversalStyleCompaction()) {

  // list of column family descriptors, first entry must always be default column family
  final List<ColumnFamilyDescriptor> cfDescriptors = Arrays.asList(
      new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, cfOpts),
      new ColumnFamilyDescriptor("my-first-columnfamily".getBytes(), cfOpts)
  );

  // a list which will hold the handles for the column families once the db is opened
  final List<ColumnFamilyHandle> columnFamilyHandleList =
      new ArrayList<>();

  try (final DBOptions options = new DBOptions()
      .setCreateIfMissing(true)
      .setCreateMissingColumnFamilies(true);
       final RocksDB db = RocksDB.open(options,
           "path/to/do", cfDescriptors,
           columnFamilyHandleList)) {

    try {

      // do something

    } finally {

      // NOTE user must explicitly frees the column family handles before freeing the db
      for (final ColumnFamilyHandle columnFamilyHandle :
          columnFamilyHandleList) {
        columnFamilyHandle.close();
      }
    } // frees the column family options
  }
} // frees the db and the db options

With the changes in this PR, the Java user no longer has to worry about manually closing the Column Families, which allows them to write simpler symmetrical create/free oriented code like this:

try (final ColumnFamilyOptions cfOpts = new ColumnFamilyOptions().optimizeUniversalStyleCompaction()) {

  // list of column family descriptors, first entry must always be default column family
  final List<ColumnFamilyDescriptor> cfDescriptors = Arrays.asList(
      new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, cfOpts),
      new ColumnFamilyDescriptor("my-first-columnfamily".getBytes(), cfOpts)
  );

  // a list which will hold the handles for the column families once the db is opened
  final List<ColumnFamilyHandle> columnFamilyHandleList =
      new ArrayList<>();

  try (final DBOptions options = new DBOptions()
      .setCreateIfMissing(true)
      .setCreateMissingColumnFamilies(true);
       final RocksDB db = RocksDB.open(options,
           "path/to/do", cfDescriptors,
           columnFamilyHandleList)) {   

        // do something

    } // frees the column family options, then frees the db and the db options
  }
}

NOTE: The changes in this PR are backwards API compatible, which means existing code using the original approach will also continue to function correctly.

Copy link
Collaborator

@adamretter adamretter left a comment

Choose a reason for hiding this comment

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

I wonder if we should also implement RocksDB#destroyColumnFamily - which would also close and remove the cf from the list?

I also wonder if we should leave a test for the original approach where you have to manually close the column families and mark it as deprecated, to show that for the time being that is still supported. When this PR is merged we should also likely update the Wiki page about RocksJava object memory management.

Also, perhaps on RocksDB#close we should consider calling ownedColumnFamilies.clear() when we have finishing closing the cfs.

java/src/main/java/org/rocksdb/RocksDB.java Show resolved Hide resolved
java/src/main/java/org/rocksdb/RocksDB.java Show resolved Hide resolved
Copy link
Collaborator

@adamretter adamretter left a comment

Choose a reason for hiding this comment

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

LGTM just some small changes required

@alucarded alucarded force-pushed the cfhandle branch 2 times, most recently from 1d0cf36 to fcaa1b3 Compare September 24, 2020 12:22
Copy link
Contributor

@facebook-github-bot facebook-github-bot left a comment

Choose a reason for hiding this comment

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

@jay-zhuang has imported this pull request. If you are a Facebook employee, you can view this diff on Phabricator.

@facebook-github-bot
Copy link
Contributor

@alucarded has updated the pull request. You must reimport the pull request before landing.

Copy link
Contributor

@facebook-github-bot facebook-github-bot left a comment

Choose a reason for hiding this comment

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

@jay-zhuang has imported this pull request. If you are a Facebook employee, you can view this diff on Phabricator.

@facebook-github-bot
Copy link
Contributor

@jay-zhuang merged this pull request in 05fba96.

codingrhythm pushed a commit to SafetyCulture/rocksdb that referenced this pull request Mar 5, 2021
…Handle instances (facebook#7428)

Summary:
- Takes the burden off developer to close ColumnFamilyHandle instances before closing RocksDB instance
- The change is backward-compatible

----
Previously the pattern for working with Column Families was:

```java
try (final ColumnFamilyOptions cfOpts = new ColumnFamilyOptions().optimizeUniversalStyleCompaction()) {

  // list of column family descriptors, first entry must always be default column family
  final List<ColumnFamilyDescriptor> cfDescriptors = Arrays.asList(
      new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, cfOpts),
      new ColumnFamilyDescriptor("my-first-columnfamily".getBytes(), cfOpts)
  );

  // a list which will hold the handles for the column families once the db is opened
  final List<ColumnFamilyHandle> columnFamilyHandleList =
      new ArrayList<>();

  try (final DBOptions options = new DBOptions()
      .setCreateIfMissing(true)
      .setCreateMissingColumnFamilies(true);
       final RocksDB db = RocksDB.open(options,
           "path/to/do", cfDescriptors,
           columnFamilyHandleList)) {

    try {

      // do something

    } finally {

      // NOTE user must explicitly frees the column family handles before freeing the db
      for (final ColumnFamilyHandle columnFamilyHandle :
          columnFamilyHandleList) {
        columnFamilyHandle.close();
      }
    } // frees the column family options
  }
} // frees the db and the db options
```

With the changes in this PR, the Java user no longer has to worry about manually closing the Column Families, which allows them to write simpler symmetrical create/free oriented code like this:

```java
try (final ColumnFamilyOptions cfOpts = new ColumnFamilyOptions().optimizeUniversalStyleCompaction()) {

  // list of column family descriptors, first entry must always be default column family
  final List<ColumnFamilyDescriptor> cfDescriptors = Arrays.asList(
      new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, cfOpts),
      new ColumnFamilyDescriptor("my-first-columnfamily".getBytes(), cfOpts)
  );

  // a list which will hold the handles for the column families once the db is opened
  final List<ColumnFamilyHandle> columnFamilyHandleList =
      new ArrayList<>();

  try (final DBOptions options = new DBOptions()
      .setCreateIfMissing(true)
      .setCreateMissingColumnFamilies(true);
       final RocksDB db = RocksDB.open(options,
           "path/to/do", cfDescriptors,
           columnFamilyHandleList)) {

        // do something

    } // frees the column family options, then frees the db and the db options
  }
}
```

**NOTE**: The changes in this PR are backwards API compatible, which means existing code using the original approach will also continue to function correctly.

Pull Request resolved: facebook#7428

Reviewed By: cheng-chang

Differential Revision: D24063348

Pulled By: jay-zhuang

fbshipit-source-id: 648d7526669923128c863ead94516bf4d50ac658
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

4 participants