-
Notifications
You must be signed in to change notification settings - Fork 0
Fix remaining issues with snapshot expiration for branching and tagging #1
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
Fix remaining issues with snapshot expiration for branching and tagging #1
Conversation
| refs = refsFromJson(node.get(REFS)); | ||
| } else if (currentSnapshotId != -1) { | ||
| // initialize the main branch if there are no refs | ||
| refs = ImmutableMap.of(SnapshotRef.MAIN_BRANCH, SnapshotRef.branchBuilder(currentSnapshotId).build()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I guess it's not directly related to the expiration logic, but Is initializing the main branch when parsing sufficient for setting the main ref on the next commit? I was thinking it would have to be set in the builder.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should follow up with a more thorough fix, but this is a good start.
| private Set<Long> unreferencedSnapshotsToRetain(Collection<SnapshotRef> refs) { | ||
| Set<Long> referencedSnapshots = Sets.newHashSet(); | ||
| for (SnapshotRef ref : refs) { | ||
| for (Snapshot snapshot : SnapshotUtil.ancestorsOf(ref.snapshotId(), base::snapshot)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah this works too, I built the complex map before hand so that the ancestors didn't need to be traversed again and could be maintained in memory for the later computation, but maybe that's a overkill (and also consumes even more memory)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This isn't that expensive. We could also pass a set into the branch logic to update, but that would require changes there and would use an input variable as output, so I'd prefer not to.
|
Thanks @rdblue for cleaning it up, yeah I think the complex map is not really needed. I had it because I thought it would be more efficient to avoid having traverse different branches multiple times. once a branch's lineage was traversed it could reside in memory. But this may be overkill and not the dimension to optimize for. Thanks for adding more tests as well! |
Core: Pass in parent snapshot to apply and validate.
…flake-managed Iceberg tables (apache#6428) * Initial read-only Snowflake Catalog implementation by @sfc-gh-mparmar (#1) Initial read-only Snowflake Catalog implementation built on top of the Snowflake JDBC driver, providing support for basic listing of namespaces, listing of tables, and loading/reads of tables. Auth options are passthrough to the JDBC driver. Co-authored-by: Maninder Parmar <[email protected]> Co-authored-by: Maninder Parmar <[email protected]> Co-authored-by: Dennis Huo <[email protected]> * Add JdbcSnowflakeClientTest using mocks (#2) Add JdbcSnowflakeClientTest using mocks; provides full coverage of JdbcSnowflakeClient and entities' ResultSetHandler logic. Also update target Spark runtime versions to be included. * Add test { useJUnitPlatform() } tuple to iceberg-snowflake for consistency and future interoperability with inheriting from abstact unittest base classes. * Extract versions into versions.props per PR review * Misc test-related refactors per review suggestions -Convert unittests to all use assertj/Assertions for "fluent assertions" -Refactor test injection into overloaded initialize() method -Add test cases for close() propagation -Use CloseableGroup. * Fix unsupported behaviors of loadNamedpaceMetadata and defaultWarehouseLocation * Move TableIdentifier checks out of newTableOps into the SnowflakTableOperations class itself, add test case. * Refactor out any Namespace-related business logic from the lower SnowflakeClient/JdbcSnowflakeClient layers and merge SnowflakeTable and SnowflakeSchema into a single SnowflakeIdentifier that also encompasses ROOT and DATABASE level identifiers. A SnowflakeIdentifier thus functions like a type-checked/constrained Iceberg TableIdentifier, and eliminates any tight coupling between a SnowflakeClient and Catalog business logic. Parsing of Namespace numerical levels into a SnowflakeIdentifier is now fully encapsulated in NamespaceHelpers so that callsites don't duplicate namespace-handling/validation logic. * Finish migrating JdbcSnowflakeClientTest off any usage of org.junit.Assert in favor of assertj's Assertions. * Style refactorings from review comments, expanded and moved InMemoryFileIO into core with its own unittest. * Fix behavior of getNamespaceMetadata to throw when the namespace doesn't exist. Refactor for naming conventions and consolidating identifier handling into NamespaceHandlers. Make FileIO instantiated fresh for each newTableOps call. * Move private constructor to top, add assertion to test case. * Define minimal ResultSetParser/QueryHarness classes to fully replace any use of commons-dbutils; refactor ResultSet handling fully into JdbcSnowflakeClient.java. * Update snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableOperations.java Co-authored-by: Eduard Tudenhöfner <[email protected]> * Refactor style suggestions; remove debug-level logging, arguments in exceptions, private members if not accessed outside, move precondition checks, add test for NamespaceHelpers. * Fix precondition messages, remove getConf() * Clean up varargs. * Make data members final, include rawJsonVal in toString for debuggability. * Combine some small test cases into roundtrip test cases, misc cleanup * Add comment for why a factory class is exposed for testing purposes. Co-authored-by: Dennis Huo <[email protected]> Co-authored-by: Maninder Parmar <[email protected]> Co-authored-by: Maninder Parmar <[email protected]> Co-authored-by: Eduard Tudenhöfner <[email protected]>
This updates apache#4578 to fix a few remaining issues.
@amogh-jahagirdar, I was going to comment on the last issue about how to compute the unreferenced snapshots to retain, but I thought it was easier to demonstrate with a PR instead.
I also went ahead and expanded a few tests and fixed problems in the tests, like creating branches from hard-coded snapshot IDs.