getAddresses()
return List.of();
}
+ /**
+ * Returns an optional affinity key so splits reading related content are routed to the
+ * same worker(s) across queries. When empty, scheduling falls back to {@link #getAddresses()}.
+ *
+ * Only remotely accessible splits may supply an affinity key (see {@link #isRemotelyAccessible()}).
+ */
+ default Optional getAffinityKey()
+ {
+ return Optional.empty();
+ }
+
default SplitWeight getSplitWeight()
{
return SplitWeight.standard();
diff --git a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheModule.java b/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheModule.java
index 4d063e05e5d9..15e979705bcc 100644
--- a/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheModule.java
+++ b/lib/trino-filesystem-cache-alluxio/src/main/java/io/trino/filesystem/alluxio/AlluxioFileSystemCacheModule.java
@@ -18,9 +18,8 @@
import com.google.inject.Binder;
import com.google.inject.Provider;
import io.airlift.configuration.AbstractConfigurationAwareModule;
-import io.trino.filesystem.cache.CachingHostAddressProvider;
-import io.trino.filesystem.cache.ConsistentHashingHostAddressProvider;
-import io.trino.filesystem.cache.ConsistentHashingHostAddressProviderConfig;
+import io.trino.filesystem.cache.CacheSplitAffinityProvider;
+import io.trino.filesystem.cache.SplitAffinityProvider;
import io.trino.filesystem.cache.TrinoFileSystemCache;
import io.trino.spi.catalog.CatalogName;
@@ -45,14 +44,13 @@ public AlluxioFileSystemCacheModule(boolean isCoordinator)
protected void setup(Binder binder)
{
configBinder(binder).bindConfig(AlluxioFileSystemCacheConfig.class);
- configBinder(binder).bindConfig(ConsistentHashingHostAddressProviderConfig.class);
binder.bind(AlluxioCacheStats.class).in(SINGLETON);
Provider catalogName = binder.getProvider(CatalogName.class);
newExporter(binder).export(AlluxioCacheStats.class)
.as(generator -> generator.generatedNameOf(AlluxioCacheStats.class, catalogName.get().toString()));
if (isCoordinator) {
- newOptionalBinder(binder, CachingHostAddressProvider.class).setBinding().to(ConsistentHashingHostAddressProvider.class).in(SINGLETON);
+ newOptionalBinder(binder, SplitAffinityProvider.class).setBinding().to(CacheSplitAffinityProvider.class).in(SINGLETON);
}
binder.bind(TrinoFileSystemCache.class).to(AlluxioFileSystemCache.class).in(SINGLETON);
diff --git a/lib/trino-filesystem-manager/src/main/java/io/trino/filesystem/manager/FileSystemModule.java b/lib/trino-filesystem-manager/src/main/java/io/trino/filesystem/manager/FileSystemModule.java
index 8183e434f61c..353658cec3aa 100644
--- a/lib/trino-filesystem-manager/src/main/java/io/trino/filesystem/manager/FileSystemModule.java
+++ b/lib/trino-filesystem-manager/src/main/java/io/trino/filesystem/manager/FileSystemModule.java
@@ -30,9 +30,9 @@
import io.trino.filesystem.azure.AzureFileSystemModule;
import io.trino.filesystem.cache.CacheFileSystemFactory;
import io.trino.filesystem.cache.CacheKeyProvider;
-import io.trino.filesystem.cache.CachingHostAddressProvider;
import io.trino.filesystem.cache.DefaultCacheKeyProvider;
-import io.trino.filesystem.cache.DefaultCachingHostAddressProvider;
+import io.trino.filesystem.cache.NoopSplitAffinityProvider;
+import io.trino.filesystem.cache.SplitAffinityProvider;
import io.trino.filesystem.cache.TrinoFileSystemCache;
import io.trino.filesystem.gcs.GcsFileSystemFactory;
import io.trino.filesystem.gcs.GcsFileSystemModule;
@@ -121,8 +121,8 @@ protected void setup(Binder binder)
factories.addBinding("file").to(LocalFileSystemFactory.class);
}
- newOptionalBinder(binder, CachingHostAddressProvider.class).setDefault().to(DefaultCachingHostAddressProvider.class).in(Scopes.SINGLETON);
newOptionalBinder(binder, CacheKeyProvider.class).setDefault().to(DefaultCacheKeyProvider.class).in(Scopes.SINGLETON);
+ newOptionalBinder(binder, SplitAffinityProvider.class).setDefault().to(NoopSplitAffinityProvider.class).in(Scopes.SINGLETON);
newOptionalBinder(binder, TrinoFileSystemCache.class);
newOptionalBinder(binder, MemoryFileSystemCache.class);
diff --git a/lib/trino-filesystem/pom.xml b/lib/trino-filesystem/pom.xml
index ac7dcb5f4bc4..b522b21e5950 100644
--- a/lib/trino-filesystem/pom.xml
+++ b/lib/trino-filesystem/pom.xml
@@ -18,12 +18,6 @@
-
- com.github.ishugaliy
- allgood-consistent-hash
- 1.0.0
-
-
com.google.guava
guava
@@ -90,11 +84,6 @@
trino-spi
-
- jakarta.annotation
- jakarta.annotation-api
-
-
jakarta.validation
jakarta.validation-api
diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/DefaultCachingHostAddressProvider.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheSplitAffinityProvider.java
similarity index 69%
rename from lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/DefaultCachingHostAddressProvider.java
rename to lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheSplitAffinityProvider.java
index 9fdbe388eabe..1ca4dba0015f 100644
--- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/DefaultCachingHostAddressProvider.java
+++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CacheSplitAffinityProvider.java
@@ -13,16 +13,14 @@
*/
package io.trino.filesystem.cache;
-import io.trino.spi.HostAddress;
+import java.util.Optional;
-import java.util.List;
-
-public class DefaultCachingHostAddressProvider
- implements CachingHostAddressProvider
+public class CacheSplitAffinityProvider
+ implements SplitAffinityProvider
{
@Override
- public List getHosts(String splitKey, List defaultAddresses)
+ public Optional getKey(String path, long offset, long length)
{
- return defaultAddresses;
+ return Optional.of(path + ":" + offset + ":" + length);
}
}
diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CachingHostAddressProvider.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CachingHostAddressProvider.java
deleted file mode 100644
index dbc28b1d7674..000000000000
--- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/CachingHostAddressProvider.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package io.trino.filesystem.cache;
-
-import io.trino.spi.HostAddress;
-
-import java.util.List;
-
-public interface CachingHostAddressProvider
-{
- /**
- * Returns a lists of hosts which are preferred to cache the split with the given path.
- */
- List getHosts(String splitKey, List defaultAddresses);
-
- // Include offset and length in key to allow different splits belonging to the same file to be cached on different nodes
- // This can help with avoiding hotspots when multiple splits are created from a file
- static String getSplitKey(String splitPath, long offset, long length)
- {
- return splitPath + ":" + offset + ":" + length;
- }
-}
diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/NoopSplitAffinityProvider.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/NoopSplitAffinityProvider.java
new file mode 100644
index 000000000000..9e41b3328740
--- /dev/null
+++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/NoopSplitAffinityProvider.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.filesystem.cache;
+
+import java.util.Optional;
+
+public class NoopSplitAffinityProvider
+ implements SplitAffinityProvider
+{
+ @Override
+ public Optional getKey(String path, long offset, long length)
+ {
+ return Optional.empty();
+ }
+}
diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/SplitAffinityProvider.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/SplitAffinityProvider.java
new file mode 100644
index 000000000000..e18de49533e6
--- /dev/null
+++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/cache/SplitAffinityProvider.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.filesystem.cache;
+
+import java.util.Optional;
+
+/**
+ * Produces a stable affinity key for a split, used by the engine to consistently route
+ * splits reading the same content to the same worker(s). A non-empty key opts the split
+ * into cache-aware scheduling via the engine's consistent-hash ring; an empty key leaves
+ * scheduling to the split's own addresses.
+ */
+public interface SplitAffinityProvider
+{
+ Optional getKey(String path, long offset, long length);
+}
diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/cache/TestConsistentHashingCacheHostAddressProvider.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/cache/TestConsistentHashingCacheHostAddressProvider.java
deleted file mode 100644
index c24a50c880fd..000000000000
--- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/cache/TestConsistentHashingCacheHostAddressProvider.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package io.trino.filesystem.cache;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Sets;
-import io.trino.node.InternalNode;
-import io.trino.spi.Node;
-import io.trino.spi.NodeVersion;
-import io.trino.testing.TestingNodeManager;
-import org.junit.jupiter.api.Test;
-
-import java.net.URI;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import static java.lang.Math.abs;
-import static org.assertj.core.api.Assertions.assertThat;
-
-public class TestConsistentHashingCacheHostAddressProvider
-{
- @Test
- public void testConsistentHashing()
- {
- TestingNodeManager nodeManager = TestingNodeManager.builder()
- .addNode(node("test-1"))
- .addNode(node("test-2"))
- .addNode(node("test-3"))
- .build();
- ConsistentHashingHostAddressProvider provider = new ConsistentHashingHostAddressProvider(
- nodeManager,
- new ConsistentHashingHostAddressProviderConfig().setPreferredHostsCount(1));
- provider.refreshHashRing();
- assertFairDistribution(provider, nodeManager.getWorkerNodes());
- nodeManager.removeNode(node("test-2"));
- provider.refreshHashRing();
- assertFairDistribution(provider, nodeManager.getWorkerNodes());
- nodeManager.addNode(node("test-4"));
- nodeManager.addNode(node("test-5"));
- provider.refreshHashRing();
- assertFairDistribution(provider, nodeManager.getWorkerNodes());
- }
-
- @Test
- public void testConsistentHashingFairRedistribution()
- {
- TestingNodeManager nodeManager = TestingNodeManager.builder()
- .addNode(node("test-1"))
- .addNode(node("test-2"))
- .addNode(node("test-3"))
- .build();
- ConsistentHashingHostAddressProvider provider = new ConsistentHashingHostAddressProvider(
- nodeManager,
- new ConsistentHashingHostAddressProviderConfig().setPreferredHostsCount(1));
- provider.refreshHashRing();
- Map> distribution = getDistribution(provider);
- nodeManager.removeNode(node("test-1"));
- provider.refreshHashRing();
- Map> removeOne = getDistribution(provider);
- assertMinimalRedistribution(distribution, removeOne);
- nodeManager.addNode(node("test-1"));
- provider.refreshHashRing();
- Map> addOne = getDistribution(provider);
- assertMinimalRedistribution(removeOne, addOne);
- assertThat(addOne).isEqualTo(distribution);
- nodeManager.addNode(node("test-4"));
- provider.refreshHashRing();
- Map> addTwo = getDistribution(provider);
- assertMinimalRedistribution(addOne, addTwo);
- }
-
- private static void assertFairDistribution(CachingHostAddressProvider cachingHostAddressProvider, Set nodeNames)
- {
- int n = 1000;
- Map counts = new HashMap<>();
- for (int i = 0; i < n; i++) {
- counts.merge(cachingHostAddressProvider.getHosts(String.valueOf(i), ImmutableList.of()).get(0).getHostText(), 1, Math::addExact);
- }
- assertThat(nodeNames.stream().map(m -> m.getHostAndPort().getHostText()).collect(Collectors.toSet())).isEqualTo(counts.keySet());
- counts.values().forEach(c -> assertThat(abs(c - n / nodeNames.size()) < 0.1 * n).isTrue());
- }
-
- private void assertMinimalRedistribution(Map> oldDistribution, Map> newDistribution)
- {
- oldDistribution.entrySet().stream().filter(e -> newDistribution.containsKey(e.getKey())).forEach(entry -> {
- int sameKeySize = Sets.intersection(newDistribution.get(entry.getKey()), entry.getValue()).size();
- int oldKeySize = entry.getValue().size();
- assertThat(abs(sameKeySize - oldKeySize) < oldKeySize / oldDistribution.size()).isTrue();
- });
- }
-
- private Map> getDistribution(ConsistentHashingHostAddressProvider provider)
- {
- int n = 1000;
- Map> distribution = new HashMap<>();
- for (int i = 0; i < n; i++) {
- String host = provider.getHosts(String.valueOf(i), ImmutableList.of()).get(0).getHostText();
- distribution.computeIfAbsent(host, (k) -> new HashSet<>()).add(i);
- }
- return distribution;
- }
-
- private static Node node(String nodeName)
- {
- return new InternalNode(nodeName, URI.create("http://" + nodeName + "/"), NodeVersion.UNKNOWN, false);
- }
-}
diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java
index 9c4cce955d6e..0749d5bfdec3 100644
--- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java
+++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java
@@ -168,7 +168,7 @@ public ConnectorPageSource createPageSource(
.filter(column -> (column.columnType() == REGULAR) || column.baseColumnName().equals(ROW_ID_COLUMN_NAME))
.collect(toImmutableList());
- Map> partitionKeys = split.getPartitionKeys();
+ Map> partitionKeys = split.partitionKeys();
ColumnMappingMode columnMappingMode = getColumnMappingMode(table.getMetadataEntry(), table.getProtocolEntry());
Optional> partitionValues = Optional.empty();
if (deltaLakeColumns.stream().anyMatch(column -> column.baseColumnName().equals(ROW_ID_COLUMN_NAME))) {
@@ -195,7 +195,7 @@ public ConnectorPageSource createPageSource(
// is available now, without having to access parquet file footer for row-group stats.
TupleDomain filteredSplitPredicate = TupleDomain.intersect(ImmutableList.of(
table.getNonPartitionConstraint(),
- split.getStatisticsPredicate(),
+ split.statisticsPredicate(),
dynamicFilter.getCurrentPredicate().transformKeys(DeltaLakeColumnHandle.class::cast)));
if (filteredSplitPredicate.isNone()) {
return new EmptyPageSource();
@@ -203,34 +203,34 @@ public ConnectorPageSource createPageSource(
Map partitionColumnDomains = filteredSplitPredicate.getDomains().orElseThrow().entrySet().stream()
.filter(entry -> entry.getKey().columnType() == DeltaLakeColumnType.PARTITION_KEY)
.collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue));
- if (!partitionMatchesPredicate(split.getPartitionKeys(), partitionColumnDomains)) {
+ if (!partitionMatchesPredicate(split.partitionKeys(), partitionColumnDomains)) {
return new EmptyPageSource();
}
// Skip reading the file if none of the actual file columns are being read
if (filteredSplitPredicate.isAll() &&
- split.getStart() == 0 && split.getLength() == split.getFileSize() &&
- split.getFileRowCount().isPresent() &&
- split.getDeletionVector().isEmpty() &&
+ split.start() == 0 && split.length() == split.fileSize() &&
+ split.fileRowCount().isPresent() &&
+ split.deletionVector().isEmpty() &&
(regularColumns.isEmpty() || onlyRowIdColumn(regularColumns))) {
return projectColumns(
deltaLakeColumns,
ImmutableSet.of(),
partitionKeys,
partitionValues,
- generatePages(split.getFileRowCount().get(), onlyRowIdColumn(regularColumns)),
- split.getPath(),
- split.getFileSize(),
- split.getFileModifiedTime());
+ generatePages(split.fileRowCount().get(), onlyRowIdColumn(regularColumns)),
+ split.path(),
+ split.fileSize(),
+ split.fileModifiedTime());
}
- Location location = Location.of(split.getPath());
+ Location location = Location.of(split.path());
TrinoFileSystem fileSystem = fileSystemFactory.create(session, table);
- TrinoInputFile inputFile = fileSystem.newInputFile(location, split.getFileSize());
+ TrinoInputFile inputFile = fileSystem.newInputFile(location, split.fileSize());
ParquetReaderOptions options = ParquetReaderOptions.builder(parquetReaderOptions)
.withMaxReadBlockSize(getParquetMaxReadBlockSize(session))
.withMaxReadBlockRowCount(getParquetMaxReadBlockRowCount(session))
.withSmallFileThreshold(getParquetSmallFileThreshold(session))
- .withUseColumnIndex(!table.isMerge() && split.getDeletionVector().isEmpty() && isParquetUseColumnIndex(session))
+ .withUseColumnIndex(!table.isMerge() && split.deletionVector().isEmpty() && isParquetUseColumnIndex(session))
.withIgnoreStatistics(isParquetIgnoreStatistics(session))
.withVectorizedDecodingEnabled(isParquetVectorizedDecodingEnabled(session))
.build();
@@ -248,7 +248,7 @@ public ConnectorPageSource createPageSource(
hiveColumnHandlesBuilder::add,
() -> missingColumnNamesBuilder.add(column.baseColumnName()));
}
- if (split.getDeletionVector().isPresent() && !regularColumns.contains(rowPositionColumnHandle())) {
+ if (split.deletionVector().isPresent() && !regularColumns.contains(rowPositionColumnHandle())) {
hiveColumnHandlesBuilder.add(PARQUET_ROW_INDEX_COLUMN);
}
List hiveColumnHandles = hiveColumnHandlesBuilder.build();
@@ -258,8 +258,8 @@ public ConnectorPageSource createPageSource(
ConnectorPageSource delegate = ParquetPageSourceFactory.createPageSource(
inputFile,
- split.getStart(),
- split.getLength(),
+ split.start(),
+ split.length(),
hiveColumnHandles,
ImmutableList.of(parquetPredicate),
true,
@@ -269,15 +269,15 @@ public ConnectorPageSource createPageSource(
Optional.empty(),
Optional.empty(),
domainCompactionThreshold,
- OptionalLong.of(split.getFileSize()));
+ OptionalLong.of(split.fileSize()));
- if (split.getDeletionVector().isPresent()) {
+ if (split.deletionVector().isPresent()) {
var pageFilterSupplier = Suppliers.memoize(() -> {
List requiredColumns = ImmutableList.builderWithExpectedSize(regularColumns.size() + 1)
.addAll(regularColumns)
.add(rowPositionColumnHandle())
.build();
- PositionDeleteFilter deleteFilter = readDeletes(fileSystem, Location.of(table.location()), split.getDeletionVector().get());
+ PositionDeleteFilter deleteFilter = readDeletes(fileSystem, Location.of(table.location()), split.deletionVector().get());
return deleteFilter.createPredicate(requiredColumns);
});
@@ -292,9 +292,9 @@ public ConnectorPageSource createPageSource(
partitionKeys,
partitionValues,
delegate,
- split.getPath(),
- split.getFileSize(),
- split.getFileModifiedTime());
+ split.path(),
+ split.fileSize(),
+ split.fileModifiedTime());
}
public static ConnectorPageSource projectColumns(
diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplit.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplit.java
index 2cb0d556d230..24b242b89e4e 100644
--- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplit.java
+++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplit.java
@@ -13,20 +13,14 @@
*/
package io.trino.plugin.deltalake;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.collect.ImmutableList;
import io.airlift.slice.SizeOf;
import io.trino.plugin.deltalake.transactionlog.DeletionVectorEntry;
-import io.trino.spi.HostAddress;
import io.trino.spi.SplitWeight;
import io.trino.spi.connector.ConnectorSplit;
import io.trino.spi.predicate.TupleDomain;
-import java.util.List;
import java.util.Map;
-import java.util.Objects;
import java.util.Optional;
import static com.google.common.base.MoreObjects.toStringHelper;
@@ -36,148 +30,45 @@
import static io.airlift.slice.SizeOf.sizeOf;
import static java.util.Objects.requireNonNull;
-public class DeltaLakeSplit
+public record DeltaLakeSplit(
+ @JsonProperty("path") String path,
+ @JsonProperty("start") long start,
+ @JsonProperty("length") long length,
+ @JsonProperty("fileSize") long fileSize,
+ @JsonProperty("rowCount") Optional fileRowCount,
+ @JsonProperty("fileModifiedTime") long fileModifiedTime,
+ @JsonProperty("deletionVector") Optional deletionVector,
+ @JsonProperty("affinityKey") Optional affinityKey,
+ @JsonProperty("splitWeight") SplitWeight splitWeight,
+ @JsonProperty("statisticsPredicate") TupleDomain statisticsPredicate,
+ @JsonProperty("partitionKeys") Map> partitionKeys)
implements ConnectorSplit
{
private static final int INSTANCE_SIZE = instanceSize(DeltaLakeSplit.class);
- private final String path;
- private final long start;
- private final long length;
- private final long fileSize;
- private final Optional fileRowCount;
- private final long fileModifiedTime;
- private final Optional deletionVector;
- private final List addresses;
- private final SplitWeight splitWeight;
- private final TupleDomain statisticsPredicate;
- private final Map> partitionKeys;
-
- @JsonCreator
- public DeltaLakeSplit(
- @JsonProperty("path") String path,
- @JsonProperty("start") long start,
- @JsonProperty("length") long length,
- @JsonProperty("fileSize") long fileSize,
- @JsonProperty("rowCount") Optional fileRowCount,
- @JsonProperty("fileModifiedTime") long fileModifiedTime,
- @JsonProperty("deletionVector") Optional deletionVector,
- @JsonProperty("splitWeight") SplitWeight splitWeight,
- @JsonProperty("statisticsPredicate") TupleDomain statisticsPredicate,
- @JsonProperty("partitionKeys") Map> partitionKeys)
+ public DeltaLakeSplit
{
- this(
- path,
- start,
- length,
- fileSize,
- fileRowCount,
- fileModifiedTime,
- deletionVector,
- ImmutableList.of(),
- splitWeight,
- statisticsPredicate,
- partitionKeys);
+ requireNonNull(path, "path is null");
+ requireNonNull(fileRowCount, "rowCount is null");
+ requireNonNull(deletionVector, "deletionVector is null");
+ requireNonNull(affinityKey, "affinityKey is null");
+ requireNonNull(splitWeight, "splitWeight is null");
+ requireNonNull(statisticsPredicate, "statisticsPredicate is null");
+ requireNonNull(partitionKeys, "partitionKeys is null");
}
- public DeltaLakeSplit(
- String path,
- long start,
- long length,
- long fileSize,
- Optional fileRowCount,
- long fileModifiedTime,
- Optional deletionVector,
- List addresses,
- SplitWeight splitWeight,
- TupleDomain statisticsPredicate,
- Map> partitionKeys)
- {
- this.path = requireNonNull(path, "path is null");
- this.start = start;
- this.length = length;
- this.fileSize = fileSize;
- this.fileRowCount = requireNonNull(fileRowCount, "rowCount is null");
- this.fileModifiedTime = fileModifiedTime;
- this.deletionVector = requireNonNull(deletionVector, "deletionVector is null");
- this.addresses = requireNonNull(addresses, "addresses is null");
- this.splitWeight = requireNonNull(splitWeight, "splitWeight is null");
- this.statisticsPredicate = requireNonNull(statisticsPredicate, "statisticsPredicate is null");
- this.partitionKeys = requireNonNull(partitionKeys, "partitionKeys is null");
- }
-
- // do not serialize addresses as they are not needed on workers
- @JsonIgnore
@Override
- public List getAddresses()
+ public Optional getAffinityKey()
{
- return addresses;
+ return affinityKey;
}
- @JsonProperty
@Override
public SplitWeight getSplitWeight()
{
return splitWeight;
}
- @JsonProperty
- public String getPath()
- {
- return path;
- }
-
- @JsonProperty
- public long getStart()
- {
- return start;
- }
-
- @JsonProperty
- public long getLength()
- {
- return length;
- }
-
- @JsonProperty
- public long getFileSize()
- {
- return fileSize;
- }
-
- @JsonProperty
- public Optional getFileRowCount()
- {
- return fileRowCount;
- }
-
- @JsonProperty
- public long getFileModifiedTime()
- {
- return fileModifiedTime;
- }
-
- @JsonProperty
- public Optional getDeletionVector()
- {
- return deletionVector;
- }
-
- /**
- * A TupleDomain representing the min/max statistics from the file this split was generated from. This does not contain any partitioning information.
- */
- @JsonProperty
- public TupleDomain getStatisticsPredicate()
- {
- return statisticsPredicate;
- }
-
- @JsonProperty
- public Map> getPartitionKeys()
- {
- return partitionKeys;
- }
-
@Override
public long getRetainedSizeInBytes()
{
@@ -185,6 +76,7 @@ public long getRetainedSizeInBytes()
+ estimatedSizeOf(path)
+ sizeOf(fileRowCount, value -> LONG_INSTANCE_SIZE)
+ sizeOf(deletionVector, DeletionVectorEntry::sizeInBytes)
+ + sizeOf(affinityKey, SizeOf::estimatedSizeOf)
+ splitWeight.getRetainedSizeInBytes()
+ statisticsPredicate.getRetainedSizeInBytes(DeltaLakeColumnHandle::retainedSizeInBytes)
+ estimatedSizeOf(partitionKeys, SizeOf::estimatedSizeOf, value -> sizeOf(value, SizeOf::estimatedSizeOf));
@@ -201,37 +93,8 @@ public String toString()
.add("rowCount", fileRowCount)
.add("fileModifiedTime", fileModifiedTime)
.add("deletionVector", deletionVector)
- .add("addresses", addresses)
.add("statisticsPredicate", statisticsPredicate)
.add("partitionKeys", partitionKeys)
.toString();
}
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- DeltaLakeSplit that = (DeltaLakeSplit) o;
- return start == that.start &&
- length == that.length &&
- fileSize == that.fileSize &&
- fileModifiedTime == that.fileModifiedTime &&
- path.equals(that.path) &&
- fileRowCount.equals(that.fileRowCount) &&
- deletionVector.equals(that.deletionVector) &&
- Objects.equals(addresses, that.addresses) &&
- Objects.equals(statisticsPredicate, that.statisticsPredicate) &&
- Objects.equals(partitionKeys, that.partitionKeys);
- }
-
- @Override
- public int hashCode()
- {
- return Objects.hash(path, start, length, fileSize, fileRowCount, fileModifiedTime, deletionVector, addresses, statisticsPredicate, partitionKeys);
- }
}
diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitManager.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitManager.java
index 3e212c4f31d1..3c0136e34b67 100644
--- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitManager.java
+++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitManager.java
@@ -17,7 +17,7 @@
import com.google.inject.Inject;
import io.airlift.units.DataSize;
import io.trino.filesystem.Location;
-import io.trino.filesystem.cache.CachingHostAddressProvider;
+import io.trino.filesystem.cache.SplitAffinityProvider;
import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitSource;
import io.trino.plugin.deltalake.functions.tablechanges.TableChangesSplitSource;
import io.trino.plugin.deltalake.functions.tablechanges.TableChangesTableFunctionHandle;
@@ -56,7 +56,6 @@
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.ImmutableMap.toImmutableMap;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
-import static io.trino.filesystem.cache.CachingHostAddressProvider.getSplitKey;
import static io.trino.plugin.deltalake.DeltaLakeAnalyzeProperties.AnalyzeMode.FULL_REFRESH;
import static io.trino.plugin.deltalake.DeltaLakeMetadata.createStatisticsPredicate;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getDynamicFilteringWaitTimeout;
@@ -86,7 +85,7 @@ public class DeltaLakeSplitManager
private final double minimumAssignedSplitWeight;
private final DeltaLakeFileSystemFactory fileSystemFactory;
private final DeltaLakeTransactionManager deltaLakeTransactionManager;
- private final CachingHostAddressProvider cachingHostAddressProvider;
+ private final SplitAffinityProvider splitAffinityProvider;
@Inject
public DeltaLakeSplitManager(
@@ -96,7 +95,7 @@ public DeltaLakeSplitManager(
DeltaLakeConfig config,
DeltaLakeFileSystemFactory fileSystemFactory,
DeltaLakeTransactionManager deltaLakeTransactionManager,
- CachingHostAddressProvider cachingHostAddressProvider)
+ SplitAffinityProvider splitAffinityProvider)
{
this.typeManager = requireNonNull(typeManager, "typeManager is null");
this.transactionLogAccess = requireNonNull(transactionLogAccess, "transactionLogAccess is null");
@@ -106,7 +105,7 @@ public DeltaLakeSplitManager(
this.minimumAssignedSplitWeight = config.getMinimumAssignedSplitWeight();
this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null");
this.deltaLakeTransactionManager = requireNonNull(deltaLakeTransactionManager, "deltaLakeTransactionManager is null");
- this.cachingHostAddressProvider = requireNonNull(cachingHostAddressProvider, "cacheHostAddressProvider is null");
+ this.splitAffinityProvider = requireNonNull(splitAffinityProvider, "splitAffinityProvider is null");
}
@Override
@@ -309,7 +308,7 @@ private List splitsForFile(
addFileEntry.getStats().flatMap(DeltaLakeFileStatistics::getNumRecords),
addFileEntry.getModificationTime(),
addFileEntry.getDeletionVector(),
- cachingHostAddressProvider.getHosts(splitPath, ImmutableList.of()),
+ splitAffinityProvider.getKey(splitPath, 0, fileSize),
SplitWeight.standard(),
statisticsPredicate,
partitionKeys));
@@ -321,6 +320,7 @@ private List splitsForFile(
long maxSplitSize = getMaxSplitSize(session).toBytes();
long splitSize = Math.min(maxSplitSize, fileSize - currentOffset);
+ Optional affinityKey = splitAffinityProvider.getKey(splitPath, currentOffset, splitSize);
splits.add(new DeltaLakeSplit(
splitPath,
currentOffset,
@@ -329,7 +329,7 @@ private List splitsForFile(
Optional.empty(),
addFileEntry.getModificationTime(),
addFileEntry.getDeletionVector(),
- cachingHostAddressProvider.getHosts(getSplitKey(splitPath, currentOffset, splitSize), ImmutableList.of()),
+ affinityKey,
SplitWeight.fromProportion(clamp((double) splitSize / maxSplitSize, minimumAssignedSplitWeight, 1.0)),
statisticsPredicate,
partitionKeys));
diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitSource.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitSource.java
index 075c1665b1a9..690a0dd0149e 100644
--- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitSource.java
+++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitSource.java
@@ -142,11 +142,11 @@ public CompletableFuture getNextBatch(int maxSize)
.collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue));
List filteredSplits = splits.stream()
.map(DeltaLakeSplit.class::cast)
- .filter(split -> partitionMatchesPredicate(split.getPartitionKeys(), partitionColumnDomains) &&
- split.getStatisticsPredicate().overlaps(dynamicFilterPredicate))
+ .filter(split -> partitionMatchesPredicate(split.partitionKeys(), partitionColumnDomains) &&
+ split.statisticsPredicate().overlaps(dynamicFilterPredicate))
.collect(toImmutableList());
if (recordScannedFiles) {
- filteredSplits.forEach(split -> scannedFilePaths.add(new DeltaLakeScannedDataFile(((DeltaLakeSplit) split).getPath(), ((DeltaLakeSplit) split).getPartitionKeys())));
+ filteredSplits.forEach(split -> scannedFilePaths.add(new DeltaLakeScannedDataFile(((DeltaLakeSplit) split).path(), ((DeltaLakeSplit) split).partitionKeys())));
}
return new ConnectorSplitBatch(filteredSplits, noMoreSplits);
},
diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java
index e96e7afc6087..1ec70c88ac1c 100644
--- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java
+++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java
@@ -23,8 +23,8 @@
import io.airlift.bootstrap.Bootstrap;
import io.airlift.json.JsonModule;
import io.trino.filesystem.TrinoFileSystemFactory;
-import io.trino.filesystem.cache.CachingHostAddressProvider;
-import io.trino.filesystem.cache.DefaultCachingHostAddressProvider;
+import io.trino.filesystem.cache.NoopSplitAffinityProvider;
+import io.trino.filesystem.cache.SplitAffinityProvider;
import io.trino.filesystem.hdfs.HdfsFileSystemFactory;
import io.trino.hdfs.HdfsEnvironment;
import io.trino.hdfs.TrinoHdfsFileSystemStats;
@@ -196,7 +196,7 @@ public void setUp()
binder.bind(HdfsEnvironment.class).toInstance(HDFS_ENVIRONMENT);
binder.bind(TrinoHdfsFileSystemStats.class).toInstance(HDFS_FILE_SYSTEM_STATS);
binder.bind(TrinoFileSystemFactory.class).to(HdfsFileSystemFactory.class).in(Scopes.SINGLETON);
- binder.bind(CachingHostAddressProvider.class).to(DefaultCachingHostAddressProvider.class).in(Scopes.SINGLETON);
+ binder.bind(SplitAffinityProvider.class).toInstance(new NoopSplitAffinityProvider());
},
new AbstractModule()
{
diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java
index ffa215cbb70d..56499384c915 100644
--- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java
+++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java
@@ -118,6 +118,7 @@ public void testDynamicSplitPruningOnUnpartitionedTable()
Optional.empty(),
0,
Optional.empty(),
+ Optional.empty(),
SplitWeight.standard(),
TupleDomain.all(),
ImmutableMap.of());
@@ -215,6 +216,7 @@ public void testDynamicSplitPruningWithExplicitPartitionFilter()
Optional.empty(),
0,
Optional.empty(),
+ Optional.empty(),
SplitWeight.standard(),
TupleDomain.all(),
ImmutableMap.of(dateColumnName, Optional.of("2023-01-10")));
diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java
index 12cb207d0168..f8bbe30439d4 100644
--- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java
+++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java
@@ -18,7 +18,7 @@
import io.airlift.json.JsonCodecFactory;
import io.airlift.units.DataSize;
import io.trino.filesystem.Location;
-import io.trino.filesystem.cache.DefaultCachingHostAddressProvider;
+import io.trino.filesystem.cache.NoopSplitAffinityProvider;
import io.trino.filesystem.memory.MemoryFileSystemFactory;
import io.trino.metastore.HiveMetastoreFactory;
import io.trino.plugin.base.metrics.FileFormatDataSourceStats;
@@ -126,11 +126,12 @@ private void testAbsolutePathSplits(String absoluteRawEncodedFilePath, String ab
DeltaLakeSplitManager splitManager = setupSplitManager(addFileEntries, deltaLakeConfig);
List splits = getSplits(splitManager, deltaLakeConfig);
+ long maxSplitSize = deltaLakeConfig.getMaxSplitSize().toBytes();
List expected = ImmutableList.of(
- makeSplit(absoluteDecodedParsedFilePath, 0, 5_000, fileSize, minimumAssignedSplitWeight),
- makeSplit(absoluteDecodedParsedFilePath, 5_000, 5_000, fileSize, minimumAssignedSplitWeight),
- makeSplit(absoluteDecodedParsedFilePath, 10_000, 5_000, fileSize, minimumAssignedSplitWeight),
- makeSplit(absoluteDecodedParsedFilePath, 15_000, 5_000, fileSize, minimumAssignedSplitWeight));
+ makeSplit(absoluteDecodedParsedFilePath, 0, 5_000, fileSize, maxSplitSize, minimumAssignedSplitWeight),
+ makeSplit(absoluteDecodedParsedFilePath, 5_000, 5_000, fileSize, maxSplitSize, minimumAssignedSplitWeight),
+ makeSplit(absoluteDecodedParsedFilePath, 10_000, 5_000, fileSize, maxSplitSize, minimumAssignedSplitWeight),
+ makeSplit(absoluteDecodedParsedFilePath, 15_000, 5_000, fileSize, maxSplitSize, minimumAssignedSplitWeight));
assertThat(splits).isEqualTo(expected);
}
@@ -148,10 +149,11 @@ public void testSplitSizes()
DeltaLakeSplitManager splitManager = setupSplitManager(addFileEntries, deltaLakeConfig);
List splits = getSplits(splitManager, deltaLakeConfig);
+ long maxSplitSize = deltaLakeConfig.getMaxSplitSize().toBytes();
List expected = ImmutableList.of(
- makeSplit(FULL_PATH, 0, 20_000, fileSize, minimumAssignedSplitWeight),
- makeSplit(FULL_PATH, 20_000, 20_000, fileSize, minimumAssignedSplitWeight),
- makeSplit(FULL_PATH, 40_000, 10_000, fileSize, minimumAssignedSplitWeight));
+ makeSplit(FULL_PATH, 0, 20_000, fileSize, maxSplitSize, minimumAssignedSplitWeight),
+ makeSplit(FULL_PATH, 20_000, 20_000, fileSize, maxSplitSize, minimumAssignedSplitWeight),
+ makeSplit(FULL_PATH, 40_000, 10_000, fileSize, maxSplitSize, minimumAssignedSplitWeight));
assertThat(splits).isEqualTo(expected);
}
@@ -170,10 +172,11 @@ public void testSplitsFromMultipleFiles()
DeltaLakeSplitManager splitManager = setupSplitManager(addFileEntries, deltaLakeConfig);
List splits = getSplits(splitManager, deltaLakeConfig);
+ long maxSplitSize = deltaLakeConfig.getMaxSplitSize().toBytes();
List expected = ImmutableList.of(
- makeSplit(FULL_PATH, 0, 1_000, firstFileSize, minimumAssignedSplitWeight),
- makeSplit(FULL_PATH, 0, 10_000, secondFileSize, minimumAssignedSplitWeight),
- makeSplit(FULL_PATH, 10_000, 10_000, secondFileSize, minimumAssignedSplitWeight));
+ makeSplit(FULL_PATH, 0, 1_000, firstFileSize, maxSplitSize, minimumAssignedSplitWeight),
+ makeSplit(FULL_PATH, 0, 10_000, secondFileSize, maxSplitSize, minimumAssignedSplitWeight),
+ makeSplit(FULL_PATH, 10_000, 10_000, secondFileSize, maxSplitSize, minimumAssignedSplitWeight));
assertThat(splits).isEqualTo(expected);
}
@@ -246,7 +249,7 @@ public Stream getActiveFiles(
deltaLakeConfig,
new DefaultDeltaLakeFileSystemFactory(HDFS_FILE_SYSTEM_FACTORY, new NoOpVendedCredentialsProvider()),
deltaLakeTransactionManager,
- new DefaultCachingHostAddressProvider());
+ new NoopSplitAffinityProvider());
}
private AddFileEntry addFileEntryOfSize(String path, long fileSize)
@@ -254,10 +257,10 @@ private AddFileEntry addFileEntryOfSize(String path, long fileSize)
return new AddFileEntry(path, ImmutableMap.of(), fileSize, 0, false, Optional.empty(), Optional.empty(), ImmutableMap.of(), Optional.empty());
}
- private DeltaLakeSplit makeSplit(String path, long start, long splitSize, long fileSize, double minimumAssignedSplitWeight)
+ private DeltaLakeSplit makeSplit(String path, long start, long splitSize, long fileSize, long maxSplitSize, double minimumAssignedSplitWeight)
{
- SplitWeight splitWeight = SplitWeight.fromProportion(clamp((double) fileSize / splitSize, minimumAssignedSplitWeight, 1.0));
- return new DeltaLakeSplit(path, start, splitSize, fileSize, Optional.empty(), 0, Optional.empty(), splitWeight, TupleDomain.all(), ImmutableMap.of());
+ SplitWeight splitWeight = SplitWeight.fromProportion(clamp((double) splitSize / maxSplitSize, minimumAssignedSplitWeight, 1.0));
+ return new DeltaLakeSplit(path, start, splitSize, fileSize, Optional.empty(), 0, Optional.empty(), Optional.empty(), splitWeight, TupleDomain.all(), ImmutableMap.of());
}
private List getSplits(DeltaLakeSplitManager splitManager, DeltaLakeConfig deltaLakeConfig)
diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplit.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplit.java
index 125ac10f579d..596fd6074bb0 100644
--- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplit.java
+++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplit.java
@@ -18,6 +18,7 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
+import io.airlift.slice.SizeOf;
import io.trino.metastore.HiveTypeName;
import io.trino.plugin.hive.util.HiveBucketing.BucketingVersion;
import io.trino.spi.HostAddress;
@@ -51,6 +52,7 @@ public class HiveSplit
private final Schema schema;
private final List partitionKeys;
private final List addresses;
+ private final Optional affinityKey;
private final String partitionName;
private final OptionalInt readBucketNumber;
private final OptionalInt tableBucketNumber;
@@ -90,6 +92,7 @@ public HiveSplit(
schema,
partitionKeys,
ImmutableList.of(),
+ Optional.empty(),
readBucketNumber,
tableBucketNumber,
forceLocalScheduling,
@@ -110,6 +113,7 @@ public HiveSplit(
Schema schema,
List partitionKeys,
List addresses,
+ Optional affinityKey,
OptionalInt readBucketNumber,
OptionalInt tableBucketNumber,
boolean forceLocalScheduling,
@@ -127,6 +131,7 @@ public HiveSplit(
requireNonNull(schema, "schema is null");
requireNonNull(partitionKeys, "partitionKeys is null");
requireNonNull(addresses, "addresses is null");
+ requireNonNull(affinityKey, "affinityKey is null");
requireNonNull(readBucketNumber, "readBucketNumber is null");
requireNonNull(tableBucketNumber, "tableBucketNumber is null");
requireNonNull(hiveColumnCoercions, "hiveColumnCoercions is null");
@@ -143,6 +148,7 @@ public HiveSplit(
this.schema = schema;
this.partitionKeys = ImmutableList.copyOf(partitionKeys);
this.addresses = ImmutableList.copyOf(addresses);
+ this.affinityKey = affinityKey;
this.readBucketNumber = readBucketNumber;
this.tableBucketNumber = tableBucketNumber;
this.forceLocalScheduling = forceLocalScheduling;
@@ -209,6 +215,14 @@ public List getAddresses()
return addresses;
}
+ // do not serialize affinity key as it is only used by the scheduler on the coordinator
+ @JsonIgnore
+ @Override
+ public Optional getAffinityKey()
+ {
+ return affinityKey;
+ }
+
@JsonProperty
public OptionalInt getReadBucketNumber()
{
@@ -272,6 +286,7 @@ public long getRetainedSizeInBytes()
+ schema.getRetainedSizeInBytes()
+ estimatedSizeOf(partitionKeys, HivePartitionKey::estimatedSizeInBytes)
+ estimatedSizeOf(addresses, HostAddress::getRetainedSizeInBytes)
+ + sizeOf(affinityKey, SizeOf::estimatedSizeOf)
+ estimatedSizeOf(partitionName)
+ sizeOf(readBucketNumber)
+ sizeOf(tableBucketNumber)
diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java
index 94a739307466..3fc04a2f5ea1 100644
--- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java
+++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java
@@ -24,7 +24,7 @@
import io.airlift.stats.CounterStat;
import io.airlift.units.DataSize;
import io.trino.filesystem.TrinoFileSystemFactory;
-import io.trino.filesystem.cache.CachingHostAddressProvider;
+import io.trino.filesystem.cache.SplitAffinityProvider;
import io.trino.metastore.Column;
import io.trino.metastore.HiveBucketProperty;
import io.trino.metastore.HivePartition;
@@ -122,7 +122,7 @@ public class HiveSplitManager
private final boolean recursiveDfsWalkerEnabled;
private final CounterStat highMemorySplitSourceCounter;
private final TypeManager typeManager;
- private final CachingHostAddressProvider cachingHostAddressProvider;
+ private final SplitAffinityProvider splitAffinityProvider;
private final int maxPartitionsPerScan;
@Inject
@@ -134,7 +134,7 @@ public HiveSplitManager(
@ForHiveSplitManager ExecutorService executorService,
VersionEmbedder versionEmbedder,
TypeManager typeManager,
- CachingHostAddressProvider cachingHostAddressProvider)
+ SplitAffinityProvider splitAffinityProvider)
{
this(
transactionManager,
@@ -151,7 +151,7 @@ public HiveSplitManager(
hiveConfig.getMaxSplitsPerSecond(),
hiveConfig.getRecursiveDirWalkerEnabled(),
typeManager,
- cachingHostAddressProvider,
+ splitAffinityProvider,
hiveConfig.getMaxPartitionsPerScan());
}
@@ -170,7 +170,7 @@ public HiveSplitManager(
@Nullable Integer maxSplitsPerSecond,
boolean recursiveDfsWalkerEnabled,
TypeManager typeManager,
- CachingHostAddressProvider cachingHostAddressProvider,
+ SplitAffinityProvider splitAffinityProvider,
int maxPartitionsPerScan)
{
this.transactionManager = requireNonNull(transactionManager, "transactionManager is null");
@@ -188,7 +188,7 @@ public HiveSplitManager(
this.maxSplitsPerSecond = requireNonNullElse(maxSplitsPerSecond, Integer.MAX_VALUE);
this.recursiveDfsWalkerEnabled = recursiveDfsWalkerEnabled;
this.typeManager = requireNonNull(typeManager, "typeManager is null");
- this.cachingHostAddressProvider = requireNonNull(cachingHostAddressProvider, "cachingHostAddressProvider is null");
+ this.splitAffinityProvider = requireNonNull(splitAffinityProvider, "splitAffinityProvider is null");
this.maxPartitionsPerScan = maxPartitionsPerScan;
}
@@ -291,7 +291,7 @@ public ConnectorSplitSource getSplits(
hiveSplitLoader,
executor,
highMemorySplitSourceCounter,
- cachingHostAddressProvider,
+ splitAffinityProvider,
hiveTable.isRecordScannedFiles());
hiveSplitLoader.start(splitSource);
diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java
index 6332eb98ca31..9f1cf8477e8c 100644
--- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java
+++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java
@@ -19,7 +19,7 @@
import io.airlift.log.Logger;
import io.airlift.stats.CounterStat;
import io.airlift.units.DataSize;
-import io.trino.filesystem.cache.CachingHostAddressProvider;
+import io.trino.filesystem.cache.SplitAffinityProvider;
import io.trino.plugin.hive.InternalHiveSplit.InternalHiveBlock;
import io.trino.plugin.hive.util.AsyncQueue;
import io.trino.plugin.hive.util.AsyncQueue.BorrowResult;
@@ -86,7 +86,7 @@ class HiveSplitSource
private final CounterStat highMemorySplitSourceCounter;
private final AtomicBoolean loggedHighMemoryWarning = new AtomicBoolean();
private final HiveSplitWeightProvider splitWeightProvider;
- private final CachingHostAddressProvider cachingHostAddressProvider;
+ private final SplitAffinityProvider splitAffinityProvider;
private final boolean recordScannedFiles;
private final ImmutableList.Builder