Skip to content
Merged
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
Original file line number Diff line number Diff line change
Expand Up @@ -152,18 +152,19 @@ public BucketNodeMap getBucketNodeMap(Session session, PartitioningHandle partit
{
ConnectorBucketNodeMap connectorBucketNodeMap = getConnectorBucketNodeMap(session, partitioningHandle);

ToIntFunction<Split> splitToBucket = getSplitToBucket(session, partitioningHandle);
if (connectorBucketNodeMap.hasFixedMapping()) {
return new FixedBucketNodeMap(getSplitToBucket(session, partitioningHandle), getFixedMapping(connectorBucketNodeMap));
return new FixedBucketNodeMap(splitToBucket, getFixedMapping(connectorBucketNodeMap));
}

if (preferDynamic) {
return new DynamicBucketNodeMap(getSplitToBucket(session, partitioningHandle), connectorBucketNodeMap.getBucketCount());
return new DynamicBucketNodeMap(splitToBucket, connectorBucketNodeMap.getBucketCount());
}

Optional<CatalogHandle> catalogName = partitioningHandle.getCatalogHandle();
checkArgument(catalogName.isPresent(), "No catalog handle for partitioning handle: %s", partitioningHandle);
return new FixedBucketNodeMap(
getSplitToBucket(session, partitioningHandle),
splitToBucket,
createArbitraryBucketToNode(
new ArrayList<>(nodeScheduler.createNodeSelector(session, catalogName).allNodes()),
connectorBucketNodeMap.getBucketCount()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,8 +13,6 @@
*/
package io.trino.plugin.hive;

import com.google.common.collect.ImmutableList;
import io.trino.spi.Node;
import io.trino.spi.NodeManager;
import io.trino.spi.connector.BucketFunction;
import io.trino.spi.connector.ConnectorBucketNodeMap;
Expand All @@ -29,14 +27,9 @@

import javax.inject.Inject;

import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.function.ToIntFunction;
import java.util.stream.Stream;

import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.trino.spi.connector.ConnectorBucketNodeMap.createBucketNodeMap;
import static java.util.Objects.requireNonNull;

Expand Down Expand Up @@ -85,7 +78,7 @@ public ConnectorBucketNodeMap getBucketNodeMap(ConnectorTransactionHandle transa
return createBucketNodeMap(handle.getBucketCount());
}

// Create a bucket to node mapping. Consecutive buckets are assigned
// Allocate a fixed number of buckets. Trino will assign consecutive buckets
// to shuffled nodes (e.g "1 -> node2, 2 -> node1, 3 -> node2, 4 -> node1, ...").
// Hash function generates consecutive bucket numbers within a partition
// (e.g "(part1, bucket1) -> 1234, (part1, bucket2) -> 1235, ...").
Expand All @@ -95,23 +88,7 @@ public ConnectorBucketNodeMap getBucketNodeMap(ConnectorTransactionHandle transa
// However, number of partitions is not known here
Comment thread
sopel39 marked this conversation as resolved.
Outdated
// If number of workers < ( P * B), we need multiple writers per node to fully
// parallelize the write within a worker
return createBucketNodeMap(createArbitraryBucketToNode(
ImmutableList.copyOf(nodeManager.getRequiredWorkerNodes()),
nodeManager.getRequiredWorkerNodes().size() * PARTITIONED_BUCKETS_PER_NODE));
}

private static List<Node> createArbitraryBucketToNode(List<Node> nodes, int bucketCount)
{
return cyclingShuffledStream(nodes)
.limit(bucketCount)
.collect(toImmutableList());
}

private static <T> Stream<T> cyclingShuffledStream(Collection<T> collection)
{
List<T> list = new ArrayList<>(collection);
Collections.shuffle(list);
return Stream.generate(() -> list).flatMap(List::stream);
return createBucketNodeMap(nodeManager.getRequiredWorkerNodes().size() * PARTITIONED_BUCKETS_PER_NODE);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5678,8 +5678,7 @@ protected void insertPartitionedBucketedTableLayout(boolean transactional)
assertEquals(insertLayout.get().getPartitionColumns(), ImmutableList.of("column1", "column2"));
ConnectorBucketNodeMap connectorBucketNodeMap = nodePartitioningProvider.getBucketNodeMap(transaction.getTransactionHandle(), session, partitioningHandle);
assertEquals(connectorBucketNodeMap.getBucketCount(), 32);
assertTrue(connectorBucketNodeMap.hasFixedMapping());
assertEquals(connectorBucketNodeMap.getFixedMapping().size(), 32);
assertFalse(connectorBucketNodeMap.hasFixedMapping());
}
}
finally {
Expand Down Expand Up @@ -5773,8 +5772,7 @@ public void testCreatePartitionedBucketedTableLayout()
assertEquals(newTableLayout.get().getPartitionColumns(), ImmutableList.of("column1", "column2"));
ConnectorBucketNodeMap connectorBucketNodeMap = nodePartitioningProvider.getBucketNodeMap(transaction.getTransactionHandle(), session, partitioningHandle);
assertEquals(connectorBucketNodeMap.getBucketCount(), 32);
assertTrue(connectorBucketNodeMap.hasFixedMapping());
assertEquals(connectorBucketNodeMap.getFixedMapping().size(), 32);
assertFalse(connectorBucketNodeMap.hasFixedMapping());
}
}

Expand Down