Skip to content

Commit a9f8332

Browse files
committed
[Iceberg] Session property for target split size
1 parent 81e31fe commit a9f8332

15 files changed

+215
-56
lines changed

presto-docs/src/main/sphinx/connector/iceberg.rst

+9-3
Original file line numberDiff line numberDiff line change
@@ -357,9 +357,9 @@ connector using a WITH clause:
357357

358358
The following table properties are available, which are specific to the Presto Iceberg connector:
359359

360-
======================================= =============================================================== ============
360+
======================================= =============================================================== =========================
361361
Property Name Description Default
362-
======================================= =============================================================== ============
362+
======================================= =============================================================== =========================
363363
``format`` Optionally specifies the format of table data files, ``PARQUET``
364364
either ``PARQUET`` or ``ORC``.
365365

@@ -388,7 +388,10 @@ Property Name Description
388388

389389
``metrics_max_inferred_column`` Optionally specifies the maximum number of columns for which ``100``
390390
metrics are collected.
391-
======================================= =============================================================== ============
391+
392+
``read.target.split-size`` The target size for an individual split when generating splits ``134217728`` (128MB)
393+
for a table scan. Must be specified in bytes.
394+
======================================= =============================================================== =========================
392395

393396
The table definition below specifies format ``ORC``, partitioning by columns ``c1`` and ``c2``,
394397
and a file system location of ``s3://test_bucket/test_schema/test_table``:
@@ -421,6 +424,9 @@ Property Name Description
421424
``iceberg.rows_for_metadata_optimization_threshold`` Overrides the behavior of the connector property
422425
``iceberg.rows-for-metadata-optimization-threshold`` in the current
423426
session.
427+
``iceberg.target_split_size`` Overrides the target split size for all tables in a query in bytes.
428+
Set to 0 to use the value in each Iceberg table's
429+
``read.target.split-size`` property.
424430
===================================================== ======================================================================
425431

426432
Caching Support

presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java

+5
Original file line numberDiff line numberDiff line change
@@ -179,6 +179,7 @@
179179
import static org.apache.iceberg.SnapshotSummary.DELETED_RECORDS_PROP;
180180
import static org.apache.iceberg.SnapshotSummary.REMOVED_EQ_DELETES_PROP;
181181
import static org.apache.iceberg.SnapshotSummary.REMOVED_POS_DELETES_PROP;
182+
import static org.apache.iceberg.TableProperties.SPLIT_SIZE;
182183

183184
public abstract class IcebergAbstractMetadata
184185
implements ConnectorMetadata
@@ -611,6 +612,7 @@ protected ImmutableMap<String, Object> createMetadataProperties(Table icebergTab
611612
properties.put(METADATA_PREVIOUS_VERSIONS_MAX, IcebergUtil.getMetadataPreviousVersionsMax(icebergTable));
612613
properties.put(METADATA_DELETE_AFTER_COMMIT, IcebergUtil.isMetadataDeleteAfterCommit(icebergTable));
613614
properties.put(METRICS_MAX_INFERRED_COLUMN, IcebergUtil.getMetricsMaxInferredColumn(icebergTable));
615+
properties.put(SPLIT_SIZE, IcebergUtil.getSplitSize(icebergTable));
614616

615617
return properties.build();
616618
}
@@ -1007,6 +1009,9 @@ public void setTableProperties(ConnectorSession session, ConnectorTableHandle ta
10071009
case COMMIT_RETRIES:
10081010
updateProperties.set(TableProperties.COMMIT_NUM_RETRIES, String.valueOf(entry.getValue()));
10091011
break;
1012+
case SPLIT_SIZE:
1013+
updateProperties.set(TableProperties.SPLIT_SIZE, entry.getValue().toString());
1014+
break;
10101015
default:
10111016
throw new PrestoException(NOT_SUPPORTED, "Updating property " + entry.getKey() + " is not supported currently");
10121017
}

presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSessionProperties.java

+12
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,7 @@
4040
import static com.facebook.presto.spi.session.PropertyMetadata.booleanProperty;
4141
import static com.facebook.presto.spi.session.PropertyMetadata.doubleProperty;
4242
import static com.facebook.presto.spi.session.PropertyMetadata.integerProperty;
43+
import static com.facebook.presto.spi.session.PropertyMetadata.longProperty;
4344
import static com.facebook.presto.spi.session.PropertyMetadata.stringProperty;
4445

4546
public final class IcebergSessionProperties
@@ -65,6 +66,7 @@ public final class IcebergSessionProperties
6566
public static final String STATISTIC_SNAPSHOT_RECORD_DIFFERENCE_WEIGHT = "statistic_snapshot_record_difference_weight";
6667
public static final String ROWS_FOR_METADATA_OPTIMIZATION_THRESHOLD = "rows_for_metadata_optimization_threshold";
6768
public static final String STATISTICS_KLL_SKETCH_K_PARAMETER = "statistics_kll_sketch_k_parameter";
69+
public static final String TARGET_SPLIT_SIZE = "target_split_size";
6870

6971
private final List<PropertyMetadata<?>> sessionProperties;
7072

@@ -189,6 +191,11 @@ public IcebergSessionProperties(
189191
.add(integerProperty(STATISTICS_KLL_SKETCH_K_PARAMETER,
190192
"The K parameter for the Apache DataSketches KLL sketch when computing histogram statistics",
191193
icebergConfig.getStatisticsKllSketchKParameter(),
194+
false))
195+
.add(longProperty(
196+
TARGET_SPLIT_SIZE,
197+
"The target split size. Set to 0 to use the iceberg table's read.split.target-size property",
198+
0L,
192199
false));
193200

194201
nessieConfig.ifPresent((config) -> propertiesBuilder
@@ -323,4 +330,9 @@ public static int getStatisticsKllSketchKParameter(ConnectorSession session)
323330
{
324331
return session.getProperty(STATISTICS_KLL_SKETCH_K_PARAMETER, Integer.class);
325332
}
333+
334+
public static Long getTargetSplitSize(ConnectorSession session)
335+
{
336+
return session.getProperty(TARGET_SPLIT_SIZE, Long.class);
337+
}
326338
}

presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSplitManager.java

+3-2
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,7 @@
4747
import static com.facebook.presto.iceberg.IcebergUtil.getIcebergTable;
4848
import static com.facebook.presto.iceberg.IcebergUtil.getMetadataColumnConstraints;
4949
import static com.facebook.presto.iceberg.IcebergUtil.getNonMetadataColumnConstraints;
50+
import static com.facebook.presto.iceberg.IcebergUtil.getTargetSplitSize;
5051
import static java.util.Objects.requireNonNull;
5152

5253
public class IcebergSplitManager
@@ -95,7 +96,7 @@ public ConnectorSplitSource getSplits(
9596
IncrementalChangelogScan scan = icebergTable.newIncrementalChangelogScan()
9697
.fromSnapshotExclusive(fromSnapshot)
9798
.toSnapshot(toSnapshot);
98-
return new ChangelogSplitSource(session, typeManager, icebergTable, scan, scan.targetSplitSize());
99+
return new ChangelogSplitSource(session, typeManager, icebergTable, scan, getTargetSplitSize(session, scan).toBytes());
99100
}
100101
else if (table.getIcebergTableName().getTableType() == EQUALITY_DELETES) {
101102
CloseableIterable<DeleteFile> deleteFiles = IcebergUtil.getDeleteFiles(icebergTable,
@@ -117,7 +118,7 @@ else if (table.getIcebergTableName().getTableType() == EQUALITY_DELETES) {
117118
IcebergSplitSource splitSource = new IcebergSplitSource(
118119
session,
119120
tableScan,
120-
TableScanUtil.splitFiles(tableScan.planFiles(), tableScan.targetSplitSize()),
121+
TableScanUtil.splitFiles(tableScan.planFiles(), getTargetSplitSize(session, tableScan).toBytes()),
121122
getMinimumAssignedSplitWeight(session),
122123
getMetadataColumnConstraints(layoutHandle.getValidPredicate()));
123124
return splitSource;

presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSplitSource.java

+2-1
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@
4141
import static com.facebook.presto.iceberg.FileFormat.fromIcebergFileFormat;
4242
import static com.facebook.presto.iceberg.IcebergUtil.getDataSequenceNumber;
4343
import static com.facebook.presto.iceberg.IcebergUtil.getPartitionKeys;
44+
import static com.facebook.presto.iceberg.IcebergUtil.getTargetSplitSize;
4445
import static com.facebook.presto.iceberg.IcebergUtil.metadataColumnsMatchPredicates;
4546
import static com.facebook.presto.iceberg.IcebergUtil.partitionDataFromStructLike;
4647
import static com.google.common.collect.ImmutableList.toImmutableList;
@@ -131,7 +132,7 @@ private ConnectorSplit toIcebergSplit(FileScanTask task)
131132
PartitionSpecParser.toJson(spec),
132133
partitionData.map(PartitionData::toJson),
133134
getNodeSelectionStrategy(session),
134-
SplitWeight.fromProportion(Math.min(Math.max((double) task.length() / tableScan.targetSplitSize(), minimumAssignedSplitWeight), 1.0)),
135+
SplitWeight.fromProportion(Math.min(Math.max((double) task.length() / getTargetSplitSize(session, tableScan).toBytes(), minimumAssignedSplitWeight), 1.0)),
135136
task.deletes().stream().map(DeleteFile::fromIceberg).collect(toImmutableList()),
136137
Optional.empty(),
137138
getDataSequenceNumber(task.file()));

presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableProperties.java

+11
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
import static com.facebook.presto.common.type.VarcharType.createUnboundedVarcharType;
3030
import static com.facebook.presto.spi.session.PropertyMetadata.booleanProperty;
3131
import static com.facebook.presto.spi.session.PropertyMetadata.integerProperty;
32+
import static com.facebook.presto.spi.session.PropertyMetadata.longProperty;
3233
import static com.facebook.presto.spi.session.PropertyMetadata.stringProperty;
3334
import static com.google.common.collect.ImmutableList.toImmutableList;
3435
import static java.util.Locale.ENGLISH;
@@ -44,6 +45,7 @@ public class IcebergTableProperties
4445
public static final String METADATA_PREVIOUS_VERSIONS_MAX = "metadata_previous_versions_max";
4546
public static final String METADATA_DELETE_AFTER_COMMIT = "metadata_delete_after_commit";
4647
public static final String METRICS_MAX_INFERRED_COLUMN = "metrics_max_inferred_column";
48+
public static final String TARGET_SPLIT_SIZE = TableProperties.SPLIT_SIZE;
4749
private static final String DEFAULT_FORMAT_VERSION = "2";
4850

4951
private final List<PropertyMetadata<?>> tableProperties;
@@ -112,6 +114,10 @@ public IcebergTableProperties(IcebergConfig icebergConfig)
112114
"The maximum number of columns for which metrics are collected",
113115
icebergConfig.getMetricsMaxInferredColumn(),
114116
false))
117+
.add(longProperty(TARGET_SPLIT_SIZE,
118+
"Desired size of split to generate during query scan planning",
119+
TableProperties.SPLIT_SIZE_DEFAULT,
120+
false))
115121
.build();
116122

117123
columnProperties = ImmutableList.of(stringProperty(
@@ -177,4 +183,9 @@ public static Integer getMetricsMaxInferredColumn(Map<String, Object> tablePrope
177183
{
178184
return (Integer) tableProperties.get(METRICS_MAX_INFERRED_COLUMN);
179185
}
186+
187+
public static Long getTargetSplitSize(Map<String, Object> tableProperties)
188+
{
189+
return (Long) tableProperties.get(TableProperties.SPLIT_SIZE);
190+
}
180191
}

presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java

+37-12
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,7 @@
4949
import com.google.common.collect.ImmutableMap;
5050
import com.google.common.collect.ImmutableSet;
5151
import com.google.common.collect.Sets;
52+
import io.airlift.units.DataSize;
5253
import org.apache.iceberg.BaseTable;
5354
import org.apache.iceberg.ContentFile;
5455
import org.apache.iceberg.ContentScanTask;
@@ -61,6 +62,7 @@
6162
import org.apache.iceberg.PartitionField;
6263
import org.apache.iceberg.PartitionSpec;
6364
import org.apache.iceberg.RowLevelOperationMode;
65+
import org.apache.iceberg.Scan;
6466
import org.apache.iceberg.Schema;
6567
import org.apache.iceberg.Snapshot;
6668
import org.apache.iceberg.SortOrder;
@@ -130,7 +132,6 @@
130132
import static com.facebook.presto.iceberg.ExpressionConverter.toIcebergExpression;
131133
import static com.facebook.presto.iceberg.FileContent.POSITION_DELETES;
132134
import static com.facebook.presto.iceberg.FileContent.fromIcebergFileContent;
133-
import static com.facebook.presto.iceberg.FileFormat.PARQUET;
134135
import static com.facebook.presto.iceberg.IcebergColumnHandle.DATA_SEQUENCE_NUMBER_COLUMN_HANDLE;
135136
import static com.facebook.presto.iceberg.IcebergColumnHandle.PATH_COLUMN_HANDLE;
136137
import static com.facebook.presto.iceberg.IcebergErrorCode.ICEBERG_INVALID_FORMAT_VERSION;
@@ -196,6 +197,8 @@
196197
import static org.apache.iceberg.TableProperties.METRICS_MAX_INFERRED_COLUMN_DEFAULTS_DEFAULT;
197198
import static org.apache.iceberg.TableProperties.ORC_COMPRESSION;
198199
import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION;
200+
import static org.apache.iceberg.TableProperties.SPLIT_SIZE;
201+
import static org.apache.iceberg.TableProperties.SPLIT_SIZE_DEFAULT;
199202
import static org.apache.iceberg.TableProperties.UPDATE_MODE;
200203
import static org.apache.iceberg.TableProperties.WRITE_LOCATION_PROVIDER_IMPL;
201204
import static org.apache.iceberg.types.Type.TypeID.BINARY;
@@ -856,10 +859,10 @@ public static long getDataSequenceNumber(ContentFile<?> file)
856859
* @param requestedSchema If provided, only delete files with this schema will be provided
857860
*/
858861
public static CloseableIterable<DeleteFile> getDeleteFiles(Table table,
859-
long snapshot,
860-
TupleDomain<IcebergColumnHandle> filter,
861-
Optional<Set<Integer>> requestedPartitionSpec,
862-
Optional<Set<Integer>> requestedSchema)
862+
long snapshot,
863+
TupleDomain<IcebergColumnHandle> filter,
864+
Optional<Set<Integer>> requestedPartitionSpec,
865+
Optional<Set<Integer>> requestedSchema)
863866
{
864867
Expression filterExpression = toIcebergExpression(filter);
865868
CloseableIterable<FileScanTask> fileTasks = table.newScan().useSnapshot(snapshot).filter(filterExpression).planFiles();
@@ -1035,9 +1038,9 @@ private static class DeleteFilesIterator
10351038
private DeleteFile currentFile;
10361039

10371040
private DeleteFilesIterator(Map<Integer, PartitionSpec> partitionSpecsById,
1038-
CloseableIterator<FileScanTask> fileTasks,
1039-
Optional<Set<Integer>> requestedPartitionSpec,
1040-
Optional<Set<Integer>> requestedSchema)
1041+
CloseableIterator<FileScanTask> fileTasks,
1042+
Optional<Set<Integer>> requestedPartitionSpec,
1043+
Optional<Set<Integer>> requestedSchema)
10411044
{
10421045
this.partitionSpecsById = partitionSpecsById;
10431046
this.fileTasks = fileTasks;
@@ -1151,6 +1154,9 @@ public static Map<String, String> populateTableProperties(ConnectorTableMetadata
11511154

11521155
Integer metricsMaxInferredColumn = IcebergTableProperties.getMetricsMaxInferredColumn(tableMetadata.getProperties());
11531156
propertiesBuilder.put(METRICS_MAX_INFERRED_COLUMN_DEFAULTS, String.valueOf(metricsMaxInferredColumn));
1157+
1158+
propertiesBuilder.put(SPLIT_SIZE, String.valueOf(IcebergTableProperties.getTargetSplitSize(tableMetadata.getProperties())));
1159+
11541160
return propertiesBuilder.build();
11551161
}
11561162

@@ -1221,8 +1227,8 @@ public static Optional<PartitionData> partitionDataFromStructLike(PartitionSpec
12211227

12221228
/**
12231229
* Get the metadata location for target {@link Table},
1224-
* considering iceberg table properties {@code WRITE_METADATA_LOCATION}
1225-
* */
1230+
* considering iceberg table properties {@code WRITE_METADATA_LOCATION}
1231+
*/
12261232
public static String metadataLocation(Table icebergTable)
12271233
{
12281234
String metadataLocation = icebergTable.properties().get(TableProperties.WRITE_METADATA_LOCATION);
@@ -1237,8 +1243,8 @@ public static String metadataLocation(Table icebergTable)
12371243

12381244
/**
12391245
* Get the data location for target {@link Table},
1240-
* considering iceberg table properties {@code WRITE_DATA_LOCATION}, {@code OBJECT_STORE_PATH} and {@code WRITE_FOLDER_STORAGE_LOCATION}
1241-
* */
1246+
* considering iceberg table properties {@code WRITE_DATA_LOCATION}, {@code OBJECT_STORE_PATH} and {@code WRITE_FOLDER_STORAGE_LOCATION}
1247+
*/
12421248
public static String dataLocation(Table icebergTable)
12431249
{
12441250
Map<String, String> properties = icebergTable.properties();
@@ -1254,4 +1260,23 @@ public static String dataLocation(Table icebergTable)
12541260
}
12551261
return dataLocation;
12561262
}
1263+
1264+
public static Long getSplitSize(Table table)
1265+
{
1266+
return Long.parseLong(table.properties()
1267+
.getOrDefault(SPLIT_SIZE,
1268+
String.valueOf(SPLIT_SIZE_DEFAULT)));
1269+
}
1270+
1271+
public static DataSize getTargetSplitSize(long sessionValueProperty, long icebergScanTargetSplitSize)
1272+
{
1273+
return Optional.of(DataSize.succinctBytes(sessionValueProperty))
1274+
.filter(size -> !size.equals(DataSize.succinctBytes(0)))
1275+
.orElse(DataSize.succinctBytes(icebergScanTargetSplitSize));
1276+
}
1277+
1278+
public static DataSize getTargetSplitSize(ConnectorSession session, Scan<?, ?, ?> scan)
1279+
{
1280+
return getTargetSplitSize(IcebergSessionProperties.getTargetSplitSize(session), scan.targetSplitSize());
1281+
}
12571282
}

presto-iceberg/src/main/java/com/facebook/presto/iceberg/changelog/ChangelogSplitSource.java

+2-1
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
import com.facebook.presto.iceberg.FileFormat;
1818
import com.facebook.presto.iceberg.IcebergColumnHandle;
1919
import com.facebook.presto.iceberg.IcebergSplit;
20+
import com.facebook.presto.iceberg.IcebergUtil;
2021
import com.facebook.presto.iceberg.PartitionData;
2122
import com.facebook.presto.spi.ConnectorSession;
2223
import com.facebook.presto.spi.ConnectorSplit;
@@ -144,7 +145,7 @@ private IcebergSplit splitFromContentScanTask(ContentScanTask<DataFile> task, Ch
144145
PartitionSpecParser.toJson(spec),
145146
partitionData.map(PartitionData::toJson),
146147
getNodeSelectionStrategy(session),
147-
SplitWeight.fromProportion(Math.min(Math.max((double) task.length() / tableScan.targetSplitSize(), minimumAssignedSplitWeight), 1.0)),
148+
SplitWeight.fromProportion(Math.min(Math.max((double) task.length() / IcebergUtil.getTargetSplitSize(session, tableScan).toBytes(), minimumAssignedSplitWeight), 1.0)),
148149
ImmutableList.of(),
149150
Optional.of(new ChangelogSplitInfo(fromIcebergChangelogOperation(changeTask.operation()),
150151
changeTask.changeOrdinal(),

0 commit comments

Comments
 (0)