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

Support sorted writes in the Iceberg connector #14891

Merged
merged 5 commits into from
Feb 28, 2023

Conversation

alexjo2144
Copy link
Member

@alexjo2144 alexjo2144 commented Nov 3, 2022

Description

Support sorting files during inserts to the Iceberg connector. This reuses the SortingFileWriter from the Hive connector.

Non-technical explanation

Sorting enables better performance during selective read queries, where a small range of values is needed from a high carnality column.

Release notes

( ) This is not user-visible or docs only and no release notes are required.
( ) Release notes are required, please propose a release note for me.
(x) Release notes are required, with the following suggested text:

# Iceberg
* Support for Iceberg table sort orders. Tables can be created or altered to add a list of `sorted_by` columns which will be used to order files written to the table.

@cla-bot cla-bot bot added the cla-signed label Nov 3, 2022
@alexjo2144
Copy link
Member Author

alexjo2144 commented Nov 3, 2022

Note to self: rework the "Allow updating the sorted_by Iceberg table property" commit to give @osscm author credit.

@alexjo2144 alexjo2144 marked this pull request as draft November 3, 2022 20:48
@alexjo2144 alexjo2144 requested review from homar, ebyhr and findepi November 3, 2022 20:48
@alexjo2144 alexjo2144 self-assigned this Nov 4, 2022
@findinpath
Copy link
Contributor

Add compatibility tests with spark.

@findinpath
Copy link
Contributor

findinpath commented Nov 7, 2022

Sorting enables better performance during selective read queries, where a small range of values is needed from a high cardinality column.

I'd appreciate having a demo test that emphasises on the fact that the number of files being read is smaller when working with sorted_by. Source of inspiration: io.trino.plugin.iceberg.TestIcebergMetadataFileOperations

I don't see sortOrder in the IcebergTableHandle. Will eventually the sorted reads come as a follow-up PR?

assertThat(catalog.listTables(SESSION, Optional.empty())).contains(schemaTableName);

Table icebergTable = catalog.loadTable(SESSION, schemaTableName);
assertEquals(icebergTable.name(), quotedTableName(schemaTableName));
Copy link
Contributor

Choose a reason for hiding this comment

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

The assertions related to the columns were already made in io.trino.plugin.iceberg.catalog.BaseTrinoCatalogTest#testCreateTable . Are they relevant here as well?

return writerSortBufferSize;
}

@Config("hive.writer-sort-buffer-size")
Copy link
Contributor

Choose a reason for hiding this comment

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

Pre-existing: It would be beneficial to have the purpose of this property documented (in the code).


dropTable("test_sorted_with_partition_table");
assertUpdate("INSERT INTO " + tableName + " VALUES (true, 1, 5), (false, 2, 4), (true, 3, 3), (false, 4, 2), (true, 5, 1)", 5);
assertQuery("SELECT * FROM " + tableName, "VALUES (true, 1, 5), (false, 2, 4), (true, 3, 3), (false, 4, 2), (true, 5, 1)");
Copy link
Contributor

Choose a reason for hiding this comment

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

What do we showcase in this test?
I think the test would pass also without sorted_by property specified on the table.

Copy link
Contributor

Choose a reason for hiding this comment

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

gentle reminder

Copy link
Contributor

Choose a reason for hiding this comment

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

new reminder :)

@alexjo2144 alexjo2144 force-pushed the iceberg/sorted-writes branch from ca6d0fb to 105b42b Compare November 7, 2022 22:15
@alexjo2144
Copy link
Member Author

Still working on test cases and Marius' comments, but added support for sorting during updates and during optimize.

void verifyFileIsSorted(String path, String sortColumnName)
{
Comparable previousMax = null;
try (ParquetFileReader parquetReader = ParquetFileReader.open(HadoopInputFile.fromPath(new Path(path), ConfigurationInstantiator.newEmptyConfiguration()))) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Naive question: Why not read all the rows from the given parquet file and actually verify that any given row follows the sort order contract in respect to the previous row?

Copy link
Member Author

Choose a reason for hiding this comment

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

Hmmm, we could it just sounds more complex to set up the test.

@alexjo2144 alexjo2144 force-pushed the iceberg/sorted-writes branch 5 times, most recently from 37d76ab to e1191f4 Compare November 14, 2022 21:22
@alexjo2144 alexjo2144 force-pushed the iceberg/sorted-writes branch 2 times, most recently from 2362877 to b0d4cf0 Compare November 15, 2022 19:39
@alexjo2144 alexjo2144 marked this pull request as ready for review November 15, 2022 21:23
@alexjo2144
Copy link
Member Author

@findinpath @findepi @ebyhr I think this is ready for review. Please take a look when you get a chance

Copy link
Contributor

@findinpath findinpath left a comment

Choose a reason for hiding this comment

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

LGTM % comments


private static void parseSortField(SortOrderBuilder<?> builder, String field)
{
boolean matched = tryMatch(field, IDENTITY_ASC_NULLS_FIRST_PATTERN, match -> builder.asc(fromIdentifierToColumn(match.group(1).trim()), NullOrder.NULLS_FIRST)) ||
Copy link
Contributor

Choose a reason for hiding this comment

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

Should we extract fromIdentifierToColumn to a shared utility class?

Copy link
Member Author

Choose a reason for hiding this comment

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

Probably, but I'd rather leave it until we add transform support and clean it up then. I think there are a few other things that should be central between both partition and sort transform parsing.

#15088


dropTable("test_sorted_with_partition_table");
assertUpdate("INSERT INTO " + tableName + " VALUES (true, 1, 5), (false, 2, 4), (true, 3, 3), (false, 4, 2), (true, 5, 1)", 5);
assertQuery("SELECT * FROM " + tableName, "VALUES (true, 1, 5), (false, 2, 4), (true, 3, 3), (false, 4, 2), (true, 5, 1)");
Copy link
Contributor

Choose a reason for hiding this comment

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

gentle reminder

return sortedWritingEnabled;
}

@Config("iceberg.sorted-writing-enabled")
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 a (temporary) fallback in case that the sorted writing does not work as expected?

Copy link
Member Author

Choose a reason for hiding this comment

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

It can also be useful if your writes are very small (streaming ingest, for example) such that sorting them would be a waste of time until they are compacted.

Copy link
Member

Choose a reason for hiding this comment

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

Writes being small or not sounds like a query-dependent, so it warrants session toggle more than a catalog config.

Also, can a writer detect that written data is small and not worth sorting?
OTOH, sorting small amount of data sounds like not a big deal (as long as it happens fully in memory and doesn't add latency), so why would we care?

The config should still remain as a kill switch.

@alexjo2144 alexjo2144 force-pushed the iceberg/sorted-writes branch from e5bbfdc to 762cfee Compare February 23, 2023 14:54
@alexjo2144
Copy link
Member Author

Addressed doc comments.

I do need to fix the tests though. I moved some to the smoke tests so that they run against all supported file systems but they're not working yet.

@alexjo2144 alexjo2144 force-pushed the iceberg/sorted-writes branch 2 times, most recently from c39baaa to c3f4fc1 Compare February 24, 2023 18:17
@alexjo2144 alexjo2144 added the iceberg Iceberg connector label Feb 24, 2023
@alexjo2144 alexjo2144 force-pushed the iceberg/sorted-writes branch 2 times, most recently from 33590e7 to 0f09920 Compare February 27, 2023 17:19
@alexjo2144
Copy link
Member Author

@ebyhr @findepi I think I got everything green with this one. Can you kick off a build with serets?

@ebyhr
Copy link
Member

ebyhr commented Feb 28, 2023

/test-with-secrets sha=0f09920b81b690612b026fcfd6e7c4cb252951ee

@github-actions
Copy link

The CI workflow run with tests that require additional secrets finished as failure: https://github.com/trinodb/trino/actions/runs/4289597275

@findepi
Copy link
Member

findepi commented Feb 28, 2023

Run with secrets failed: #13199 (reopened)

@findepi findepi force-pushed the iceberg/sorted-writes branch from 0f09920 to f9d5336 Compare February 28, 2023 08:48
@findepi
Copy link
Member

findepi commented Feb 28, 2023

rebased to resolve a conflict

@findepi findepi force-pushed the iceberg/sorted-writes branch from f9d5336 to b7adc4c Compare February 28, 2023 09:23
@findepi findepi merged commit da230aa into trinodb:master Feb 28, 2023
@findepi
Copy link
Member

findepi commented Feb 28, 2023

thanks!

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

Successfully merging this pull request may close these issues.

8 participants