Skip to content

Add support to be able to query new Delta protocols#22596

Merged
tdcmeehan merged 2 commits intoprestodb:masterfrom
mblanco-denodo:issue_22543
Jul 31, 2024
Merged

Add support to be able to query new Delta protocols#22596
tdcmeehan merged 2 commits intoprestodb:masterfrom
mblanco-denodo:issue_22543

Conversation

@mblanco-denodo
Copy link
Contributor

@mblanco-denodo mblanco-denodo commented Apr 24, 2024

Description

Use of delta-kernel-api instead of delta-standalone-api in order to be able to query delta tables in newer protocols (delta reader 3, delta writer 7)
Added new configurable parameter (delta.case-sensitive-partitions-enabled) to allow case sensitive partition matching, allowing to query tables with partition names in uppercase. By default it is set to true.

Motivation and Context

Solves #22543 and #21828.

Impact

Based on some query executions over some data in S3 (times in seconds):

Table master branch
nation (tpch100) 63.06 18.92
region (tpch100) 50.25 16.05
deltatbl_partition_prune 33.63 4.98

Also experienced huge improvement on some queries during testing. Here are the HTML reports of the tests suites in master and in the branch:
master_vs_branch_test_execution_performance.zip

Test Plan

Test refactor so that the current tests are all executed on older and newer versions of the same test data.

Contributor checklist

  • Please make sure your submission complies with our development, formatting, commit message, and attribution guidelines.
  • PR description addresses the issue accurately and concisely. If the change is non-trivial, a GitHub Issue is referenced.
  • Documented new properties (with its default value), SQL syntax, functions, or other functionality.
  • If release notes are required, they follow the release notes guidelines.
  • Adequate tests were added if applicable.
  • CI passed.

Release Notes

Please follow release notes guidelines and fill in the release notes below.

== RELEASE NOTES ==

Delta Connector Changes

  • Added support to read tables up to Delta Protocol Version 3,7 through the use of the new Delta-Kernel api. :pr:22596
  • Added new boolean configuration parameter delta.case-sensitive-partitions-enabled to be able to query data with partitioned columns with column names in uppercase. This property is set to true by default. :pr:22596

@mblanco-denodo mblanco-denodo force-pushed the issue_22543 branch 2 times, most recently from ee8caf0 to da3c3ae Compare April 24, 2024 12:01
@mblanco-denodo
Copy link
Contributor Author

PR set to draft while waiting for the delta kernel api to release v3.2.0. https://github.com/delta-io/delta/milestone/26
Current limitations that will be solved on 3.2.0:

  • Move to an snapshot different from the latest.
  • Move to an snapshot by timestamp
  • Reading partition columns of TimestampType is unsupported

@tdcmeehan tdcmeehan self-assigned this May 14, 2024
@steveburnett
Copy link
Contributor

Suggest adding the PR # to the items in the release note entry, for example:

== RELEASE NOTES ==

Delta Connector Changes
* Added support to read tables up to Delta Protocol Version 3,7 through the use of the new Delta-Kernel API. :pr:`22596`
* Added new boolean configuration parameter ``delta.case-sensitive-partitions-enabled`` to be able to query data with partitioned columns with column names in uppercase. This property is set to ``true`` by default. :pr:`22596`

@mblanco-denodo mblanco-denodo marked this pull request as ready for review June 28, 2024 10:13
@mblanco-denodo mblanco-denodo requested review from a team and steveburnett as code owners June 28, 2024 10:13
@mblanco-denodo
Copy link
Contributor Author

After the release of Delta Kernel 3.2.0, all limitations are resolved. Also, it seems that there is an improvement in performance reading both delta v1 and delta v3 (perfomance local tests attached in the PR description). PR is ready for review.

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

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

Just some initial minor comments, great work!

<scope>test</scope>
</dependency>
<dependency>
<groupId>org.jetbrains</groupId>
Copy link
Contributor

Choose a reason for hiding this comment

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

What is this dependency used for?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It was used to annotate sql queries in testing. I removed the annotations and the dependency.

Comment on lines +88 to +89
Optional<Engine> deltaEngine = loadDeltaTableClient(session, location,
schemaTableName);
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
Optional<Engine> deltaEngine = loadDeltaTableClient(session, location,
schemaTableName);
Optional<Engine> deltaEngine = loadDeltaTableClient(session, location, schemaTableName);

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

Comment on lines +235 to +248
try {
CloseableIterator<FilteredColumnarBatch> columnBatches = snapshot.getScanBuilder(deltaEngine)
.build().getScanFiles(deltaEngine);
Row row = null;
while (columnBatches.hasNext()) {
CloseableIterator<Row> rows = columnBatches.next().getRows();
if (rows.hasNext()) {
row = rows.next();
break;
}
}
Map<String, String> partitionValues = row != null ?
InternalScanFileUtils.getPartitionValues(row) : new HashMap<>(0);
columnBatches.close();
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
try {
CloseableIterator<FilteredColumnarBatch> columnBatches = snapshot.getScanBuilder(deltaEngine)
.build().getScanFiles(deltaEngine);
Row row = null;
while (columnBatches.hasNext()) {
CloseableIterator<Row> rows = columnBatches.next().getRows();
if (rows.hasNext()) {
row = rows.next();
break;
}
}
Map<String, String> partitionValues = row != null ?
InternalScanFileUtils.getPartitionValues(row) : new HashMap<>(0);
columnBatches.close();
try (CloseableIterator<FilteredColumnarBatch> columnBatches = snapshot.getScanBuilder(deltaEngine).build().getScanFiles(deltaEngine)) {
Row row = null;
while (columnBatches.hasNext()) {
CloseableIterator<Row> rows = columnBatches.next().getRows();
if (rows.hasNext()) {
row = rows.next();
break;
}
}
Map<String, String> partitionValues = row != null ?
InternalScanFileUtils.getPartitionValues(row) : new HashMap<>(0);
columnBatches.close();

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

List<DeltaColumnHandle> partitionColumns = columnDomains.map(domains -> domains.stream()
.filter(entry -> entry.getColumn().getColumnType() == PARTITION)
.map(TupleDomain.ColumnDomain::getColumn)
.collect(Collectors.toList())).orElse(Collections.emptyList());
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
.collect(Collectors.toList())).orElse(Collections.emptyList());
.collect(toImmutableList())).orElse(ImmutableList.of());

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

row = nextFile.getRows();
}
else {
logger.debug("There are still rows remaining int the iterator, not advancing the iterator");
Copy link
Contributor

Choose a reason for hiding this comment

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

Perhaps remove this

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

steveburnett
steveburnett previously approved these changes Jun 28, 2024
Copy link
Contributor

@steveburnett steveburnett left a comment

Choose a reason for hiding this comment

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

A single nit of punctuation, otherwise LGTM (docs). Thanks!

@@ -41,6 +41,8 @@ Property Name Description
In order for this option to work, also set
``experimental.pushdown-dereference-enabled`` to
``true``.
``delta.case-sensitive-partitions-enabled`` Allows matching the names of partitioned columns in a ``true``
case-sensitive manner
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
case-sensitive manner
case-sensitive manner.

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

Copy link
Member

@agrawalreetika agrawalreetika left a comment

Choose a reason for hiding this comment

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

Thanks for the PR. Had few tables created with new Protocol earlier but wasn't accessible before this new Protocol changes support. Now able to access those tables.
Dropped in few minor comments, please check!

You mentioned about performance improvement, Just curious which part of the new changes are contributing to performance improvement while reading the table?

format("Could not move to latest snapshot on table '%s.%s'", schemaTableName.getSchemaName(),
schemaTableName.getTableName()));
}
}
Copy link
Member

Choose a reason for hiding this comment

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

For better readability, should we move this code block into a method something like getSnapshot instead?

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

*/
public CloseableIterator<AddFile> listFiles(ConnectorSession session, DeltaTable deltaTable)
public CloseableIterator<FilteredColumnarBatch> listFiles(ConnectorSession session, DeltaTable deltaTable)
{
Copy link
Member

Choose a reason for hiding this comment

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

We can do null check for deltaTable here -
requireNonNull(deltaTable, "deltaTable is null");

Also I think, we can shift deltaTable.getSnapshotId().isPresent() check in here in the start itself from line 148.

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

row.close();
}
catch (IOException e) {
throw new GenericInternalException("Cloud not close row batch", e);
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
throw new GenericInternalException("Cloud not close row batch", e);
throw new GenericInternalException("Could not close row batch", e);

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

* @param hiveTableName Name of the Hive table that the Delta table is to be registered as in HMS
*/
private static void registerDeltaTableInHMS(QueryRunner queryRunner, String deltaTableName, String hiveTableName)
protected static void registerDeltaTableInHMS(QueryRunner queryRunner, String deltaTableName, String hiveTableName)
Copy link
Member

Choose a reason for hiding this comment

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

Don't see usage of this outside this class. Any reason for changing this from 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.

I intended in use it in other test class but at the end I didn't and I forgot to change the method visibility back. Reverted to private.

@mblanco-denodo
Copy link
Contributor Author

@agrawalreetika performance improvement is due to the new Kernel API.

steveburnett
steveburnett previously approved these changes Jul 1, 2024
Copy link
Contributor

@steveburnett steveburnett left a comment

Choose a reason for hiding this comment

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

LGTM! (docs)

Pull updated branch, new local docs build, looks good. Thanks!

@agrawalreetika
Copy link
Member

@mblanco-denodo Could you please squash the commits?
I think here 2 commits would be good. One for changes for delta.case-sensitive-partitions-enabled and other for the changes in Delta protocols

@mblanco-denodo
Copy link
Contributor Author

@agrawalreetika commits squashed into two as requested:

  • changes in delta protocol from standalone to kernel
  • changes with the new property for uppercase column processing

agrawalreetika
agrawalreetika previously approved these changes Jul 4, 2024
Optional<Engine> deltaEngine = loadDeltaTableClient(session,
new Path(deltaTable.getTableLocation()),
new SchemaTableName(deltaTable.getSchemaName(), deltaTable.getTableName()));
if (!deltaEngine.isPresent()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Do you know under what circumstances this error could happen?

Copy link
Contributor Author

@mblanco-denodo mblanco-denodo Jul 8, 2024

Choose a reason for hiding this comment

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

Right now it should not happen, as the method loadDeltaTableClient will return a non null Engine or throw an exception. The usage of Optional in that method is inherited from the previous code, and it is a good practice to check if deltaEngine.isPresent() as in future iterations the logic of loadDeltaTableClient could change (note that all the Delta Kernel API is annotated with the @Evolving annotation, and they are making big API changes even on minor versions).

return this.caseSensitivePartitionsEnabled;
}

@Config("delta.case-sensitive-partitions-enabled")
Copy link
Member

Choose a reason for hiding this comment

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

To get a little clarity, was the issue of not being able to query mixed-case/upper-case columns only happening if the column is a partitioned column? Can we also have regular columns as mixed case or upper case, and if yes, does it work as expected?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We've experienced the problem only on partitioned columns, as the non partitioned columns are in parquet data and reading parquet the case issue does not happen. However the fix should work on both cases.

Copy link
Member

Choose a reason for hiding this comment

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

The reason I wanted to get an idea is we have removed the toLowerCase conversion for all columns not just the partitioned columns. That's why wanted to make sure it does not have any side effects for regular columns and if it's required even for regular columns, should we rename the config to delta.case-sensitive-columns-enabled?

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 can add a test on regular columns with and without the property to be sure

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 confirm that with delta.case-sensitive-partitions-enabled=false uppercase non partitioned columns are properly returning the data, so I think we should leave the property name as it is because it only changes the behavior to partitioned columns. I added the new test that compares the output between a table with a non partitioned column in lowercase and other table with the same column in uppercase to assure that there is no side effect and both tables return the same data.

Copy link
Member

Choose a reason for hiding this comment

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

Thanks for adding the tests and confirming.

steveburnett
steveburnett previously approved these changes Jul 10, 2024
Copy link
Contributor

@steveburnett steveburnett left a comment

Choose a reason for hiding this comment

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

LGTM! (docs)
Pull updated branch, local doc build, looks good. Thanks!

imjalpreet
imjalpreet previously approved these changes Jul 19, 2024
Copy link
Member

@imjalpreet imjalpreet left a comment

Choose a reason for hiding this comment

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

LGTM 👍🏼

@mblanco-denodo Can you please rebase on master to fix the failing test suite?

snapshot = deltaTable.getLatestSnapshot(deltaEngine); // get the latest snapshot
}
catch (TableNotFoundException e) {
throw new PrestoException(GENERIC_INTERNAL_ERROR,
Copy link
Contributor

Choose a reason for hiding this comment

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

embed the original exception as the cause here

Copy link
Contributor

Choose a reason for hiding this comment

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

Also this should be a user error. Can you check in StandardErrorCodes and DeltaErrorCode to check to see if there's an appropriate user exception for this? If not, please add one.

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. Added new DeltaErrorCodes to candle these cases

String format = ((SnapshotImpl) snapshot).getMetadata().getFormat().getProvider();
if (!PARQUET.name().equalsIgnoreCase(format)) {
throw new PrestoException(DELTA_UNSUPPORTED_DATA_FORMAT,
format("Delta table %s has unsupported data format: %s. Currently only Parquet data format is supported", schemaTableName, format));
Copy link
Contributor

Choose a reason for hiding this comment

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

Currently only --> Only the

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

catch (TableNotFoundException e) {
throw new PrestoException(NOT_FOUND,
format("Delta table (%s.%s) no longer exists.", deltaTable.getSchemaName(), deltaTable.getTableName()));
format("Delta table not found in '%s'", deltaTable.getTableLocation()));
Copy link
Contributor

Choose a reason for hiding this comment

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

nest exception

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

Map<String, Domain> expectedConstraint,
Map<String, Domain> expectedEnforcedConstraint)
{
System.out.println(tableScanWithConstraints(tableName, expectedConstraint, expectedEnforcedConstraint).toString());
Copy link
Contributor

Choose a reason for hiding this comment

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

remove this line

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

new Path(deltaTable.getTableLocation()),
new SchemaTableName(deltaTable.getSchemaName(), deltaTable.getTableName()));
if (!deltaEngine.isPresent()) {
throw new PrestoException(GENERIC_INTERNAL_ERROR,
Copy link
Contributor

Choose a reason for hiding this comment

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

Please throw a system error here. GENERIC_INTERNAL_ERROR is an error for anything that is not categorized, so we shouldn't be intentionally throwing it.

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

format(TABLE_NOT_FOUND_ERROR_TEMPLATE, tableName.getSchemaName(), tableName.getTableName()));
}
catch (IOException e) {
throw new GenericInternalException("Could not close columnar batch row", e);
Copy link
Contributor

Choose a reason for hiding this comment

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

Please consider rethrowing an UncheckedIOException

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

@tdcmeehan tdcmeehan merged commit 2384402 into prestodb:master Jul 31, 2024
@abhiseksaikia
Copy link
Contributor

@mblanco-denodo The directory structure for the tests are pretty long and its breaking our build due to internal limit on path length (<490).
image

Can we have a fix for this? cc : @tdcmeehan

@tdcmeehan
Copy link
Contributor

This refactoring only made the directory structure slightly larger, by introducing the delta_v1/delta_v3 prefix. Otherwise it's the same directory structure, which is what is specified by the Delta format. While we could collapse this to something smaller like v3-partition-values and remove that prefix directory, long term I'd recommend finding a way to increase the limit on the directory structure so you don't encounter this again in the future @abhiseksaikia.

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

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants