Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
16 changes: 16 additions & 0 deletions docs/src/main/sphinx/connector/iceberg.md
Original file line number Diff line number Diff line change
Expand Up @@ -217,6 +217,22 @@ implementation is used:
- Enable [sorted writing](iceberg-sorted-files) to tables with a specified sort order. Equivalent
session property is `sorted_writing_enabled`.
- `true`
* - `iceberg.temporary-staging-directory-enabled`
- Controls whether the temporary staging directory configured at
`iceberg.temporary-staging-directory-path` is used for write operations.
Temporary staging directory is never used for writes to non-sorted tables on
S3, encrypted HDFS or external location. Writes to sorted tables will
utilize this path for staging temporary files during sorting operation. When
disabled, the target storage will be used for staging while writing sorted
tables which can be inefficient when writing to object stores like S3. Equivalent
session property is `sorted_writing_temporary_staging_directory_enabled`.
- `false`
* - `iceberg.temporary-staging-directory-path`
- Controls the location of temporary staging directory that is used for write
operations. The `${USER}` placeholder can be used to use a different
location for each user. Equivalent session property is
`sorted_writing_temporary_staging_directory_path`.
- `/tmp/presto-${USER}`
Copy link
Member

Choose a reason for hiding this comment

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

presto -> trino

Copy link
Member Author

Choose a reason for hiding this comment

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

@raunaqmorarka I defaulted to the same one as the Hive temp directory. Happy to change too if you prefer

Copy link
Member

Choose a reason for hiding this comment

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

we don't need retain the old naming, please change it here

* - `iceberg.allowed-extra-properties`
- List of extra properties that are allowed to be set on Iceberg tables.
Use `*` to allow all properties.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,8 @@ public class IcebergConfig
private boolean hideMaterializedViewStorageTable = true;
private Optional<String> materializedViewsStorageSchema = Optional.empty();
private boolean sortedWritingEnabled = true;
private boolean temporaryStagingDirectoryEnabled;
private String temporaryStagingDirectoryPath = "/tmp/presto-${USER}";
Copy link
Member

Choose a reason for hiding this comment

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

presto -> trino

private boolean queryPartitionFilterRequired;
private Set<String> queryPartitionFilterRequiredSchemas = ImmutableSet.of();
private int splitManagerThreads = Math.min(Runtime.getRuntime().availableProcessors() * 2, 32);
Expand Down Expand Up @@ -454,6 +456,33 @@ public IcebergConfig setSortedWritingEnabled(boolean sortedWritingEnabled)
return this;
}

public boolean isTemporaryStagingDirectoryEnabled()
{
return temporaryStagingDirectoryEnabled;
}

@Config("iceberg.temporary-staging-directory-enabled")
@ConfigDescription("Should use temporary staging directory for write operations")
public IcebergConfig setTemporaryStagingDirectoryEnabled(boolean temporaryStagingDirectoryEnabled)
{
this.temporaryStagingDirectoryEnabled = temporaryStagingDirectoryEnabled;
return this;
}

@NotNull
public String getTemporaryStagingDirectoryPath()
{
return temporaryStagingDirectoryPath;
}

@Config("iceberg.temporary-staging-directory-path")
@ConfigDescription("Location of temporary staging directory for write operations. Use ${USER} placeholder to use different location for each user")
public IcebergConfig setTemporaryStagingDirectoryPath(String temporaryStagingDirectoryPath)
{
this.temporaryStagingDirectoryPath = temporaryStagingDirectoryPath;
return this;
}

@Config("iceberg.query-partition-filter-required")
@ConfigDescription("Require a filter on at least one partition column")
public IcebergConfig setQueryPartitionFilterRequired(boolean queryPartitionFilterRequired)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,11 @@
import static io.airlift.slice.Slices.wrappedBuffer;
import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_INVALID_METADATA;
import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_TOO_MANY_OPEN_PARTITIONS;
import static io.trino.plugin.iceberg.IcebergSessionProperties.getSortedWritingTempStagingDirPath;
import static io.trino.plugin.iceberg.IcebergSessionProperties.getSortedWritingWriterBufferSize;
import static io.trino.plugin.iceberg.IcebergSessionProperties.getSortedWritingWriterMaxOpenFiles;
import static io.trino.plugin.iceberg.IcebergSessionProperties.isSortedWritingEnabled;
import static io.trino.plugin.iceberg.IcebergSessionProperties.isSortedWritingTempStagingDirEnabled;
import static io.trino.plugin.iceberg.IcebergUtil.getTopLevelColumns;
import static io.trino.plugin.iceberg.PartitionTransforms.getColumnTransform;
import static io.trino.plugin.iceberg.util.Timestamps.getTimestampTz;
Expand Down Expand Up @@ -116,6 +120,8 @@ public class IcebergPageSink
private final boolean sortedWritingEnabled;
private final DataSize sortingFileWriterBufferSize;
private final Integer sortingFileWriterMaxOpenFiles;
private final boolean sortedWritingTempStagingPathEnabled;
private final String sortedWritingTempStagingPath;
private final Location tempDirectory;
private final TypeManager typeManager;
private final PageSorter pageSorter;
Expand Down Expand Up @@ -147,8 +153,6 @@ public IcebergPageSink(
Map<String, String> storageProperties,
int maxOpenWriters,
List<TrinoSortField> sortOrder,
DataSize sortingFileWriterBufferSize,
int sortingFileWriterMaxOpenFiles,
TypeManager typeManager,
PageSorter pageSorter)
{
Expand All @@ -169,15 +173,23 @@ public IcebergPageSink(
this.storageProperties = requireNonNull(storageProperties, "storageProperties is null");
this.sortOrder = requireNonNull(sortOrder, "sortOrder is null");
this.sortedWritingEnabled = isSortedWritingEnabled(session);
this.sortingFileWriterBufferSize = requireNonNull(sortingFileWriterBufferSize, "sortingFileWriterBufferSize is null");
this.sortingFileWriterMaxOpenFiles = sortingFileWriterMaxOpenFiles;
this.tempDirectory = Location.of(locationProvider.newDataLocation("trino-tmp-files"));
this.sortingFileWriterBufferSize = getSortedWritingWriterBufferSize(session);
this.sortingFileWriterMaxOpenFiles = getSortedWritingWriterMaxOpenFiles(session);
this.sortedWritingTempStagingPathEnabled = isSortedWritingTempStagingDirEnabled(session);
this.sortedWritingTempStagingPath = getSortedWritingTempStagingDirPath(session);
this.typeManager = requireNonNull(typeManager, "typeManager is null");
this.pageSorter = requireNonNull(pageSorter, "pageSorter is null");
this.columnTypes = getTopLevelColumns(outputSchema, typeManager).stream()
.map(IcebergColumnHandle::getType)
.collect(toImmutableList());

if (sortedWritingTempStagingPathEnabled) {
String stagingPath = sortedWritingTempStagingPath.replace("${USER}", session.getIdentity().getUser());
this.tempDirectory = setSchemeToFileIfAbsent(Location.of(stagingPath));
} else {
this.tempDirectory = Location.of(locationProvider.newDataLocation("trino-tmp-files"));
}

if (sortedWritingEnabled) {
ImmutableList.Builder<Integer> sortColumnIndexes = ImmutableList.builder();
ImmutableList.Builder<SortOrder> sortOrders = ImmutableList.builder();
Expand Down Expand Up @@ -388,6 +400,14 @@ private int[] getWriterIndexes(Page page)
return writerIndexes;
}

private static Location setSchemeToFileIfAbsent(Location location)
{
if (location.scheme().isPresent()) {
return location;
}
return Location.of("file:///" + location.path());
}

@Override
public void closeIdleWriters()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,6 @@

import com.google.inject.Inject;
import io.airlift.json.JsonCodec;
import io.airlift.units.DataSize;
import io.trino.plugin.hive.SortingFileWriterConfig;
import io.trino.plugin.iceberg.procedure.IcebergOptimizeHandle;
import io.trino.plugin.iceberg.procedure.IcebergTableExecuteHandle;
import io.trino.spi.PageIndexerFactory;
Expand Down Expand Up @@ -52,8 +50,6 @@ public class IcebergPageSinkProvider
private final JsonCodec<CommitTaskData> jsonCodec;
private final IcebergFileWriterFactory fileWriterFactory;
private final PageIndexerFactory pageIndexerFactory;
private final DataSize sortingFileWriterBufferSize;
private final int sortingFileWriterMaxOpenFiles;
private final TypeManager typeManager;
private final PageSorter pageSorter;

Expand All @@ -63,16 +59,13 @@ public IcebergPageSinkProvider(
JsonCodec<CommitTaskData> jsonCodec,
IcebergFileWriterFactory fileWriterFactory,
PageIndexerFactory pageIndexerFactory,
SortingFileWriterConfig sortingFileWriterConfig,
TypeManager typeManager,
PageSorter pageSorter)
{
this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null");
this.jsonCodec = requireNonNull(jsonCodec, "jsonCodec is null");
this.fileWriterFactory = requireNonNull(fileWriterFactory, "fileWriterFactory is null");
this.pageIndexerFactory = requireNonNull(pageIndexerFactory, "pageIndexerFactory is null");
this.sortingFileWriterBufferSize = sortingFileWriterConfig.getWriterSortBufferSize();
this.sortingFileWriterMaxOpenFiles = sortingFileWriterConfig.getMaxOpenSortFiles();
this.typeManager = requireNonNull(typeManager, "typeManager is null");
this.pageSorter = requireNonNull(pageSorter, "pageSorter is null");
}
Expand Down Expand Up @@ -109,8 +102,6 @@ private ConnectorPageSink createPageSink(ConnectorSession session, IcebergWritab
tableHandle.storageProperties(),
maxPartitionsPerWriter(session),
tableHandle.sortOrder(),
sortingFileWriterBufferSize,
sortingFileWriterMaxOpenFiles,
typeManager,
pageSorter);
}
Expand Down Expand Up @@ -140,8 +131,6 @@ public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHa
optimizeHandle.tableStorageProperties(),
maxPartitionsPerWriter(session),
optimizeHandle.sortOrder(),
sortingFileWriterBufferSize,
sortingFileWriterMaxOpenFiles,
typeManager,
pageSorter);
case OPTIMIZE_MANIFESTS:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import io.trino.orc.OrcWriteValidation.OrcWriteValidationMode;
import io.trino.plugin.base.session.SessionPropertiesProvider;
import io.trino.plugin.hive.HiveCompressionOption;
import io.trino.plugin.hive.SortingFileWriterConfig;
import io.trino.plugin.hive.orc.OrcReaderConfig;
import io.trino.plugin.hive.orc.OrcWriterConfig;
import io.trino.plugin.hive.parquet.ParquetReaderConfig;
Expand Down Expand Up @@ -106,6 +107,10 @@ public final class IcebergSessionProperties
public static final String REMOVE_ORPHAN_FILES_MIN_RETENTION = "remove_orphan_files_min_retention";
private static final String MERGE_MANIFESTS_ON_WRITE = "merge_manifests_on_write";
private static final String SORTED_WRITING_ENABLED = "sorted_writing_enabled";
private static final String SORTED_WRITING_WRITER_BUFFER_SIZE = "sorted_writing_write_buffer_size";
private static final String SORTED_WRITING_WRITER_MAX_OPEN_FILES = "sorted_writing_writer_max_open_files";
private static final String SORTED_WRITING_TEMP_STAGING_DIR_ENABLED = "sorted_writing_temporary_staging_directory_enabled";
private static final String SORTED_WRITING_TEMP_STAGING_DIR_PATH = "sorted_writing_temporary_staging_directory_path";
Comment on lines +110 to +113
Copy link
Member

Choose a reason for hiding this comment

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

We don't want to expose any of these as session properties, some of these were removed from hive connector in https://github.com/trinodb/trino/pull/17390/files#diff-7dad18a4dbe9d9a536d45655875f47ffd0d8e3b5d5c6d2f3447246763841fb44L7716

private static final String QUERY_PARTITION_FILTER_REQUIRED = "query_partition_filter_required";
private static final String QUERY_PARTITION_FILTER_REQUIRED_SCHEMAS = "query_partition_filter_required_schemas";
private static final String INCREMENTAL_REFRESH_ENABLED = "incremental_refresh_enabled";
Expand All @@ -121,7 +126,8 @@ public IcebergSessionProperties(
OrcReaderConfig orcReaderConfig,
OrcWriterConfig orcWriterConfig,
ParquetReaderConfig parquetReaderConfig,
ParquetWriterConfig parquetWriterConfig)
ParquetWriterConfig parquetWriterConfig,
SortingFileWriterConfig sortingFileWriterConfig)
{
sessionProperties = ImmutableList.<PropertyMetadata<?>>builder()
.add(dataSizeProperty(
Expand Down Expand Up @@ -368,6 +374,26 @@ public IcebergSessionProperties(
"Enable sorted writing to tables with a specified sort order",
icebergConfig.isSortedWritingEnabled(),
false))
.add(dataSizeProperty(
SORTED_WRITING_WRITER_BUFFER_SIZE,
"Target size of buffer files used during sorting",
sortingFileWriterConfig.getWriterSortBufferSize(),
false))
.add(integerProperty(
SORTED_WRITING_WRITER_MAX_OPEN_FILES,
"Max number of concurrently open buffer files during sorting",
sortingFileWriterConfig.getMaxOpenSortFiles(),
false))
.add(booleanProperty(
SORTED_WRITING_TEMP_STAGING_DIR_ENABLED,
"Should use (if possible) temporary staging directory for write operations",
icebergConfig.isTemporaryStagingDirectoryEnabled(),
false))
.add(stringProperty(
SORTED_WRITING_TEMP_STAGING_DIR_PATH,
"Location of temporary staging directory for write operations. Use ${USER} placeholder to use different location for each user",
icebergConfig.getTemporaryStagingDirectoryPath(),
false))
.add(booleanProperty(
QUERY_PARTITION_FILTER_REQUIRED,
"Require filter on partition column",
Expand Down Expand Up @@ -643,6 +669,26 @@ public static boolean isSortedWritingEnabled(ConnectorSession session)
return session.getProperty(SORTED_WRITING_ENABLED, Boolean.class);
}

public static DataSize getSortedWritingWriterBufferSize(ConnectorSession session)
{
return session.getProperty(SORTED_WRITING_WRITER_BUFFER_SIZE, DataSize.class);
}

public static Integer getSortedWritingWriterMaxOpenFiles(ConnectorSession session)
{
return session.getProperty(SORTED_WRITING_WRITER_MAX_OPEN_FILES, Integer.class);
}

public static boolean isSortedWritingTempStagingDirEnabled(ConnectorSession session)
{
return session.getProperty(SORTED_WRITING_TEMP_STAGING_DIR_ENABLED, Boolean.class);
}

public static String getSortedWritingTempStagingDirPath(ConnectorSession session)
{
return session.getProperty(SORTED_WRITING_TEMP_STAGING_DIR_PATH, String.class);
}

public static boolean isQueryPartitionFilterRequired(ConnectorSession session)
{
return session.getProperty(QUERY_PARTITION_FILTER_REQUIRED, Boolean.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -566,6 +566,33 @@ public void testFileSortingWithLargerTable()
}
}

@Test
public void testFileSortingWithLargerTableAndTempDirForBufferFiles()
Copy link
Member

Choose a reason for hiding this comment

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

testSortedWritingTempStaging

{
// Using a larger table forces buffered data to be written to disk
Session withSmallRowGroups = Session.builder(getSession())
.setCatalogSessionProperty("iceberg", "orc_writer_max_stripe_rows", "200")
.setCatalogSessionProperty("iceberg", "parquet_writer_block_size", "20kB")
.setCatalogSessionProperty("iceberg", "parquet_writer_batch_size", "200")
Comment on lines +574 to +576
Copy link
Member

Choose a reason for hiding this comment

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

These are not needed, sorting is per file, not per row-group

Copy link
Member

Choose a reason for hiding this comment

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

You can use small value of target_max_file_size to produce multiple files

.setCatalogSessionProperty("iceberg", "sorted_writing_temporary_staging_directory_enabled", "true")
.setCatalogSessionProperty("iceberg", "sorted_writing_write_buffer_size", "2kB")
.setCatalogSessionProperty("iceberg", "sorted_writing_writer_max_open_files", "5")
Comment on lines +577 to +579
Copy link
Member

Choose a reason for hiding this comment

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

These should be setup in config properties rather than session

.build();
try (TestTable table = new TestTable(
getQueryRunner()::execute,
"test_sorted_lineitem_table",
"WITH (sorted_by = ARRAY['comment'], format = '" + format.name() + "') AS TABLE tpch.tiny.lineitem WITH NO DATA")) {
assertUpdate(
withSmallRowGroups,
"INSERT INTO " + table.getName() + " TABLE tpch.tiny.lineitem",
"VALUES 60175");
for (Object filePath : computeActual("SELECT file_path from \"" + table.getName() + "$files\"").getOnlyColumnAsSet()) {
assertThat(isFileSorted(Location.of((String) filePath), "comment")).isTrue();
}
assertQuery("SELECT * FROM " + table.getName(), "SELECT * FROM lineitem");
}
}

@Test
public void testDropTableWithMissingMetadataFile()
throws Exception
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import io.trino.parquet.metadata.ParquetMetadata;
import io.trino.parquet.reader.MetadataReader;
import io.trino.plugin.base.metrics.FileFormatDataSourceStats;
import io.trino.plugin.hive.SortingFileWriterConfig;
import io.trino.plugin.hive.TrinoViewHiveMetastore;
import io.trino.plugin.hive.orc.OrcReaderConfig;
import io.trino.plugin.hive.orc.OrcWriterConfig;
Expand Down Expand Up @@ -91,7 +92,8 @@ public final class IcebergTestUtils
new OrcReaderConfig(),
new OrcWriterConfig(),
new ParquetReaderConfig(),
new ParquetWriterConfig()).getSessionProperties())
new ParquetWriterConfig(),
new SortingFileWriterConfig()).getSessionProperties())
.build();

private IcebergTestUtils() {}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,8 @@ public void testDefaults()
.setRegisterTableProcedureEnabled(false)
.setAddFilesProcedureEnabled(false)
.setSortedWritingEnabled(true)
.setTemporaryStagingDirectoryEnabled(false)
.setTemporaryStagingDirectoryPath("/tmp/presto-${USER}")
.setQueryPartitionFilterRequired(false)
.setQueryPartitionFilterRequiredSchemas(ImmutableSet.of())
.setSplitManagerThreads(Integer.toString(Runtime.getRuntime().availableProcessors() * 2))
Expand Down Expand Up @@ -113,6 +115,8 @@ public void testExplicitPropertyMappings()
.put("iceberg.register-table-procedure.enabled", "true")
.put("iceberg.add-files-procedure.enabled", "true")
.put("iceberg.sorted-writing-enabled", "false")
.put("iceberg.temporary-staging-directory-enabled", "true")
.put("iceberg.temporary-staging-directory-path", "/tmp/presto")
.put("iceberg.query-partition-filter-required", "true")
.put("iceberg.query-partition-filter-required-schemas", "bronze,silver")
.put("iceberg.split-manager-threads", "42")
Expand Down Expand Up @@ -152,6 +156,8 @@ public void testExplicitPropertyMappings()
.setRegisterTableProcedureEnabled(true)
.setAddFilesProcedureEnabled(true)
.setSortedWritingEnabled(false)
.setTemporaryStagingDirectoryEnabled(true)
.setTemporaryStagingDirectoryPath("/tmp/presto")
.setQueryPartitionFilterRequired(true)
.setQueryPartitionFilterRequiredSchemas(ImmutableSet.of("bronze", "silver"))
.setSplitManagerThreads("42")
Expand Down
Loading
Loading