-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Attach weights to IcebergSplits #12579
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||
|---|---|---|---|---|
|
|
@@ -73,6 +73,7 @@ public final class IcebergSessionProperties | |||
| private static final String PROJECTION_PUSHDOWN_ENABLED = "projection_pushdown_enabled"; | ||||
| private static final String TARGET_MAX_FILE_SIZE = "target_max_file_size"; | ||||
| private static final String HIVE_CATALOG_NAME = "hive_catalog_name"; | ||||
| private static final String MINIMUM_ASSIGNED_SPLIT_WEIGHT = "minimum_assigned_split_weight"; | ||||
| public static final String EXPIRE_SNAPSHOTS_MIN_RETENTION = "expire_snapshots_min_retention"; | ||||
| public static final String REMOVE_ORPHAN_FILES_MIN_RETENTION = "remove_orphan_files_min_retention"; | ||||
|
|
||||
|
|
@@ -229,6 +230,11 @@ public IcebergSessionProperties( | |||
| // Session-level redirections configuration does not work well with views, as view body is analyzed in context | ||||
| // of a session with properties stripped off. Thus, this property is more of a test-only, or at most POC usefulness. | ||||
| true)) | ||||
| .add(doubleProperty( | ||||
| MINIMUM_ASSIGNED_SPLIT_WEIGHT, | ||||
| "Minimum assigned split weight", | ||||
| icebergConfig.getMinimumAssignedSplitWeight(), | ||||
| false)) | ||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think we should have a session property for this. Per my understanding, this is a safety toggle.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. (btw if you convince me this stays, it needs validator as in trino/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSessionProperties.java Line 485 in a76ee40
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. i think we should remove this.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @alexjo2144 we can go ahead with this PR, and i can update mine PR to remove from both |
||||
| .add(durationProperty( | ||||
| EXPIRE_SNAPSHOTS_MIN_RETENTION, | ||||
| "Minimal retention period for expire_snapshot procedure", | ||||
|
|
@@ -394,4 +400,9 @@ public static Duration getRemoveOrphanFilesMinRetention(ConnectorSession session | |||
| { | ||||
| return session.getProperty(REMOVE_ORPHAN_FILES_MIN_RETENTION, Duration.class); | ||||
| } | ||||
|
|
||||
| public static double getMinimumAssignedSplitWeight(ConnectorSession session) | ||||
| { | ||||
| return session.getProperty(MINIMUM_ASSIGNED_SPLIT_WEIGHT, Double.class); | ||||
| } | ||||
| } | ||||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,7 @@ | |
| import com.google.common.collect.ImmutableMap; | ||
| import io.trino.plugin.iceberg.delete.TrinoDeleteFile; | ||
| import io.trino.spi.HostAddress; | ||
| import io.trino.spi.SplitWeight; | ||
|
alexjo2144 marked this conversation as resolved.
|
||
| import io.trino.spi.connector.ConnectorSplit; | ||
| import org.openjdk.jol.info.ClassLayout; | ||
|
|
||
|
|
@@ -43,6 +44,7 @@ public class IcebergSplit | |
| private final String partitionSpecJson; | ||
| private final String partitionDataJson; | ||
| private final List<TrinoDeleteFile> deletes; | ||
| private final SplitWeight splitWeight; | ||
|
|
||
| @JsonCreator | ||
| public IcebergSplit( | ||
|
|
@@ -55,7 +57,8 @@ public IcebergSplit( | |
| @JsonProperty("addresses") List<HostAddress> addresses, | ||
| @JsonProperty("partitionSpecJson") String partitionSpecJson, | ||
| @JsonProperty("partitionDataJson") String partitionDataJson, | ||
| @JsonProperty("deletes") List<TrinoDeleteFile> deletes) | ||
| @JsonProperty("deletes") List<TrinoDeleteFile> deletes, | ||
| @JsonProperty("splitWeight") SplitWeight splitWeight) | ||
| { | ||
| this.path = requireNonNull(path, "path is null"); | ||
| this.start = start; | ||
|
|
@@ -67,6 +70,7 @@ public IcebergSplit( | |
| this.partitionSpecJson = requireNonNull(partitionSpecJson, "partitionSpecJson is null"); | ||
| this.partitionDataJson = requireNonNull(partitionDataJson, "partitionDataJson is null"); | ||
| this.deletes = ImmutableList.copyOf(requireNonNull(deletes, "deletes is null")); | ||
| this.splitWeight = requireNonNull(splitWeight, "splitWeight is null"); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -136,6 +140,13 @@ public List<TrinoDeleteFile> getDeletes() | |
| return deletes; | ||
| } | ||
|
|
||
| @JsonProperty | ||
| @Override | ||
| public SplitWeight getSplitWeight() | ||
| { | ||
| return splitWeight; | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. besides being simpler, it's better to avoid a new field, so that it's clear what the weight is, and that it's reasonable & consistent with other split's state
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We could try
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. My understanding from skimming the code is that if all splits return
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That's correct. The internal representation is an integer value, normalized where a single standard split weight value is
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. To address the initial feedback point, you need to know the split's length in bytes relative to the configured target split size, which means that you would need to compute the value and store it as a field unless the target split size was also hard-coded.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I now understand this isn't a good idea.
that's weird concept from SPI perspective, since "configured target split size" is not a SPI concept itself @pettyjamesm if a connector uses (and where is it documented? :) )
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. A split with a weight of It's not explicitly documented, although the PrestoDB PR which merged after the Trino PR did include references to this behavior in the NodeScheduler Properties Reference. We should probably port those documentation changes.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
thanks, please do |
||
| } | ||
|
|
||
| @Override | ||
| public Object getInfo() | ||
| { | ||
|
|
@@ -154,7 +165,8 @@ public long getRetainedSizeInBytes() | |
| + estimatedSizeOf(addresses, HostAddress::getRetainedSizeInBytes) | ||
| + estimatedSizeOf(partitionSpecJson) | ||
| + estimatedSizeOf(partitionDataJson) | ||
| + estimatedSizeOf(deletes, TrinoDeleteFile::getRetainedSizeInBytes); | ||
| + estimatedSizeOf(deletes, TrinoDeleteFile::getRetainedSizeInBytes) | ||
| + splitWeight.getRetainedSizeInBytes(); | ||
| } | ||
|
|
||
| @Override | ||
|
|
||
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.
Capture in code comment why 0.05.
Add similar in Hive.