diff --git a/client/trino-client/src/main/java/io/trino/client/ServerInfo.java b/client/trino-client/src/main/java/io/trino/client/ServerInfo.java index be8ec4c6ad10..eac1b980fed6 100644 --- a/client/trino-client/src/main/java/io/trino/client/ServerInfo.java +++ b/client/trino-client/src/main/java/io/trino/client/ServerInfo.java @@ -35,6 +35,7 @@ public class ServerInfo // optional to maintain compatibility with older servers private final Optional uptime; private final Optional coordinatorId; + private final Optional nodeId; @JsonCreator public ServerInfo( @@ -43,7 +44,8 @@ public ServerInfo( @JsonProperty("coordinator") boolean coordinator, @JsonProperty("starting") boolean starting, @JsonProperty("uptime") Optional uptime, - @JsonProperty("coordinatorId") Optional coordinatorId) + @JsonProperty("coordinatorId") Optional coordinatorId, + @JsonProperty("nodeId") Optional nodeId) { this.nodeVersion = requireNonNull(nodeVersion, "nodeVersion is null"); this.environment = requireNonNull(environment, "environment is null"); @@ -51,6 +53,7 @@ public ServerInfo( this.starting = starting; this.uptime = requireNonNull(uptime, "uptime is null"); this.coordinatorId = requireNonNull(coordinatorId, "coordinatorId is null"); + this.nodeId = requireNonNull(nodeId, "nodeId is null"); } @JsonProperty @@ -89,6 +92,12 @@ public Optional getCoordinatorId() return coordinatorId; } + @JsonProperty + public Optional getNodeId() + { + return nodeId; + } + @Override public boolean equals(Object o) { @@ -119,6 +128,7 @@ public String toString() .add("coordinator", coordinator) .add("uptime", uptime.orElse(null)) .add("coordinatorId", coordinatorId.orElse(null)) + .add("nodeId", nodeId.orElse(null)) .omitNullValues() .toString(); } diff --git a/client/trino-client/src/test/java/io/trino/client/TestServerInfo.java b/client/trino-client/src/test/java/io/trino/client/TestServerInfo.java index 9f5d06bc3e46..42f1312c3545 100644 --- a/client/trino-client/src/test/java/io/trino/client/TestServerInfo.java +++ b/client/trino-client/src/test/java/io/trino/client/TestServerInfo.java @@ -31,15 +31,15 @@ public class TestServerInfo @Test public void testJsonRoundTrip() { - assertJsonRoundTrip(new ServerInfo(UNKNOWN, "test", true, false, Optional.of(new Duration(2, MINUTES)), Optional.of("3sruz"))); - assertJsonRoundTrip(new ServerInfo(UNKNOWN, "test", true, false, Optional.of(new Duration(2, MINUTES)), Optional.empty())); - assertJsonRoundTrip(new ServerInfo(UNKNOWN, "test", true, false, Optional.empty(), Optional.empty())); + assertJsonRoundTrip(new ServerInfo(UNKNOWN, "test", true, false, Optional.of(new Duration(2, MINUTES)), Optional.of("3sruz"), Optional.of("node-id"))); + assertJsonRoundTrip(new ServerInfo(UNKNOWN, "test", true, false, Optional.of(new Duration(2, MINUTES)), Optional.empty(), Optional.of("node-id"))); + assertJsonRoundTrip(new ServerInfo(UNKNOWN, "test", true, false, Optional.empty(), Optional.empty(), Optional.empty())); } @Test public void testBackwardsCompatible() { - ServerInfo newServerInfo = new ServerInfo(UNKNOWN, "test", true, false, Optional.empty(), Optional.empty()); + ServerInfo newServerInfo = new ServerInfo(UNKNOWN, "test", true, false, Optional.empty(), Optional.empty(), Optional.empty()); ServerInfo legacyServerInfo = SERVER_INFO_CODEC.fromJson("{\"nodeVersion\":{\"version\":\"\"},\"environment\":\"test\",\"coordinator\":true}"); assertThat(newServerInfo).isEqualTo(legacyServerInfo); } diff --git a/core/trino-main/src/main/java/io/trino/connector/CatalogPruneTask.java b/core/trino-main/src/main/java/io/trino/connector/CatalogPruneTask.java index a9930d864712..f2fddcfbebac 100644 --- a/core/trino-main/src/main/java/io/trino/connector/CatalogPruneTask.java +++ b/core/trino-main/src/main/java/io/trino/connector/CatalogPruneTask.java @@ -18,20 +18,17 @@ import com.google.common.collect.ImmutableSet; import com.google.inject.Inject; import io.airlift.concurrent.ThreadPoolExecutorMBean; -import io.airlift.discovery.client.ServiceDescriptor; -import io.airlift.discovery.client.ServiceSelector; -import io.airlift.discovery.client.ServiceType; import io.airlift.http.client.HttpClient; import io.airlift.http.client.Request; import io.airlift.http.client.Response; import io.airlift.http.client.ResponseHandler; import io.airlift.json.JsonCodec; import io.airlift.log.Logger; -import io.airlift.node.NodeInfo; import io.airlift.units.Duration; import io.trino.metadata.CatalogManager; -import io.trino.metadata.ForNodeManager; -import io.trino.server.InternalCommunicationConfig; +import io.trino.node.AllNodes; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.spi.connector.CatalogHandle; import io.trino.transaction.TransactionManager; import jakarta.annotation.PostConstruct; @@ -44,6 +41,7 @@ import java.util.Set; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Stream; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.net.HttpHeaders.CONTENT_TYPE; @@ -64,8 +62,8 @@ public class CatalogPruneTask private final TransactionManager transactionManager; private final CatalogManager catalogManager; private final ConnectorServicesProvider connectorServicesProvider; - private final NodeInfo nodeInfo; - private final ServiceSelector selector; + private final InternalNode currentNode; + private final InternalNodeManager internalNodeManager; private final HttpClient httpClient; private final ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1, daemonThreadsNamed("catalog-prune")); @@ -73,7 +71,6 @@ public class CatalogPruneTask private final boolean enabled; private final Duration updateInterval; - private final boolean httpsRequired; private final AtomicBoolean started = new AtomicBoolean(); @@ -82,23 +79,20 @@ public CatalogPruneTask( TransactionManager transactionManager, CatalogManager catalogManager, ConnectorServicesProvider connectorServicesProvider, - NodeInfo nodeInfo, - @ServiceType("trino") ServiceSelector selector, - @ForNodeManager HttpClient httpClient, - CatalogPruneTaskConfig catalogPruneTaskConfig, - InternalCommunicationConfig internalCommunicationConfig) + InternalNode currentNode, + InternalNodeManager internalNodeManager, + @ForCatalogPrune HttpClient httpClient, + CatalogPruneTaskConfig catalogPruneTaskConfig) { this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); this.catalogManager = requireNonNull(catalogManager, "catalogManager is null"); this.connectorServicesProvider = requireNonNull(connectorServicesProvider, "connectorServicesProvider is null"); - this.nodeInfo = requireNonNull(nodeInfo, "nodeInfo is null"); - this.selector = requireNonNull(selector, "selector is null"); + this.currentNode = requireNonNull(currentNode, "currentNode is null"); + this.internalNodeManager = requireNonNull(internalNodeManager, "internalNodeManager is null"); this.httpClient = requireNonNull(httpClient, "httpClient is null"); this.enabled = catalogPruneTaskConfig.isEnabled(); updateInterval = catalogPruneTaskConfig.getUpdateInterval(); - - this.httpsRequired = internalCommunicationConfig.isHttpsRequired(); } @PostConstruct @@ -133,8 +127,11 @@ public ThreadPoolExecutorMBean getExecutor() @VisibleForTesting public void pruneWorkerCatalogs() { - Set online = selector.selectAllServices().stream() - .filter(descriptor -> !nodeInfo.getNodeId().equals(descriptor.getNodeId())) + AllNodes allNodes = internalNodeManager.getAllNodes(); + Set online = Stream.of(allNodes.activeNodes(), allNodes.inactiveNodes(), allNodes.drainingNodes(), allNodes.drainedNodes(), allNodes.shuttingDownNodes()) + .flatMap(Set::stream) + .map(InternalNode::getInternalUri) + .filter(uri -> !uri.equals(currentNode.getInternalUri())) .collect(toImmutableSet()); // send message to workers to trigger prune @@ -145,13 +142,9 @@ public void pruneWorkerCatalogs() connectorServicesProvider.pruneCatalogs(ImmutableSet.of()); } - void pruneWorkerCatalogs(Set online, List activeCatalogs) + void pruneWorkerCatalogs(Set online, List activeCatalogs) { - for (ServiceDescriptor service : online) { - URI uri = getHttpUri(service); - if (uri == null) { - continue; - } + for (URI uri : online) { uri = uriBuilderFrom(uri).appendPath("/v1/task/pruneCatalogs").build(); Request request = preparePost() .setUri(uri) @@ -186,13 +179,4 @@ private List getActiveCatalogs() activeCatalogs.addAll(catalogManager.getActiveCatalogs()); return ImmutableList.copyOf(activeCatalogs.build()); } - - private URI getHttpUri(ServiceDescriptor descriptor) - { - String url = descriptor.getProperties().get(httpsRequired ? "https" : "http"); - if (url != null) { - return URI.create(url); - } - return null; - } } diff --git a/core/trino-main/src/main/java/io/trino/connector/DefaultCatalogFactory.java b/core/trino-main/src/main/java/io/trino/connector/DefaultCatalogFactory.java index ae13f58c2ce0..bc0ef7daa22c 100644 --- a/core/trino-main/src/main/java/io/trino/connector/DefaultCatalogFactory.java +++ b/core/trino-main/src/main/java/io/trino/connector/DefaultCatalogFactory.java @@ -24,8 +24,9 @@ import io.trino.connector.system.SystemConnector; import io.trino.connector.system.SystemTablesProvider; import io.trino.execution.scheduler.NodeSchedulerConfig; -import io.trino.metadata.InternalNodeManager; import io.trino.metadata.Metadata; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.security.AccessControl; import io.trino.spi.PageIndexerFactory; import io.trino.spi.PageSorter; @@ -58,6 +59,7 @@ public class DefaultCatalogFactory private final Metadata metadata; private final AccessControl accessControl; + private final InternalNode currentNode; private final InternalNodeManager nodeManager; private final PageSorter pageSorter; private final PageIndexerFactory pageIndexerFactory; @@ -76,6 +78,7 @@ public class DefaultCatalogFactory public DefaultCatalogFactory( Metadata metadata, AccessControl accessControl, + InternalNode currentNode, InternalNodeManager nodeManager, PageSorter pageSorter, PageIndexerFactory pageIndexerFactory, @@ -89,6 +92,7 @@ public DefaultCatalogFactory( { this.metadata = requireNonNull(metadata, "metadata is null"); this.accessControl = requireNonNull(accessControl, "accessControl is null"); + this.currentNode = requireNonNull(currentNode, "currentNode is null"); this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.pageSorter = requireNonNull(pageSorter, "pageSorter is null"); this.pageIndexerFactory = requireNonNull(pageIndexerFactory, "pageIndexerFactory is null"); @@ -147,13 +151,13 @@ private CatalogConnector createCatalog(CatalogHandle catalogHandle, ConnectorNam createInformationSchemaCatalogHandle(catalogHandle), new InformationSchemaConnector( catalogHandle.getCatalogName().toString(), - nodeManager, + currentNode, metadata, accessControl, maxPrefetchedInformationSchemaPrefixes)); SystemTablesProvider systemTablesProvider; - if (nodeManager.getCurrentNode().isCoordinator()) { + if (currentNode.isCoordinator()) { systemTablesProvider = new CoordinatorSystemTablesProvider( transactionManager, metadata, @@ -168,6 +172,7 @@ private CatalogConnector createCatalog(CatalogHandle catalogHandle, ConnectorNam tracer, createSystemTablesCatalogHandle(catalogHandle), new SystemConnector( + currentNode, nodeManager, systemTablesProvider, transactionId -> transactionManager.getConnectorTransaction(transactionId, catalogHandle), @@ -193,7 +198,7 @@ private Connector createConnector( catalogHandle, openTelemetry, createTracer(catalogHandle), - new DefaultNodeManager(nodeManager, schedulerIncludeCoordinator), + new DefaultNodeManager(currentNode, nodeManager, schedulerIncludeCoordinator), versionEmbedder, typeManager, new InternalMetadataProvider(metadata, typeManager), diff --git a/core/trino-main/src/main/java/io/trino/connector/DefaultNodeManager.java b/core/trino-main/src/main/java/io/trino/connector/DefaultNodeManager.java index 7bd314224a4e..5e8c20770cc8 100644 --- a/core/trino-main/src/main/java/io/trino/connector/DefaultNodeManager.java +++ b/core/trino-main/src/main/java/io/trino/connector/DefaultNodeManager.java @@ -14,23 +14,26 @@ package io.trino.connector; import com.google.common.collect.ImmutableSet; -import io.trino.metadata.InternalNodeManager; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.spi.Node; import io.trino.spi.NodeManager; import java.util.Set; -import static io.trino.metadata.NodeState.ACTIVE; +import static io.trino.node.NodeState.ACTIVE; import static java.util.Objects.requireNonNull; public class DefaultNodeManager implements NodeManager { + private final InternalNode currentNode; private final InternalNodeManager nodeManager; private final boolean schedulerIncludeCoordinator; - public DefaultNodeManager(InternalNodeManager nodeManager, boolean schedulerIncludeCoordinator) + public DefaultNodeManager(InternalNode currentNode, InternalNodeManager nodeManager, boolean schedulerIncludeCoordinator) { + this.currentNode = requireNonNull(currentNode, "currentNode is null"); this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.schedulerIncludeCoordinator = schedulerIncludeCoordinator; } @@ -42,7 +45,7 @@ public Set getAllNodes() .addAll(nodeManager.getNodes(ACTIVE)) // append current node (before connector is registered with the node // in the discovery service) since current node should have connector always loaded - .add(nodeManager.getCurrentNode()) + .add(currentNode) .build(); } @@ -55,10 +58,10 @@ public Set getWorkerNodes() nodeManager.getNodes(ACTIVE).stream() .filter(node -> !node.isCoordinator() || schedulerIncludeCoordinator) .forEach(nodes::add); - if (!nodeManager.getCurrentNode().isCoordinator() || schedulerIncludeCoordinator) { + if (!currentNode.isCoordinator() || schedulerIncludeCoordinator) { // append current node (before connector is registered with the node // in discovery service) since current node should have connector always loaded - nodes.add(getCurrentNode()); + nodes.add(currentNode); } return nodes.build(); } @@ -66,6 +69,6 @@ public Set getWorkerNodes() @Override public Node getCurrentNode() { - return nodeManager.getCurrentNode(); + return currentNode; } } diff --git a/core/trino-main/src/main/java/io/trino/connector/DynamicCatalogManagerModule.java b/core/trino-main/src/main/java/io/trino/connector/DynamicCatalogManagerModule.java index 5c6d5f8ef191..4dbe6a975dbc 100644 --- a/core/trino-main/src/main/java/io/trino/connector/DynamicCatalogManagerModule.java +++ b/core/trino-main/src/main/java/io/trino/connector/DynamicCatalogManagerModule.java @@ -17,12 +17,15 @@ import com.google.inject.Inject; import com.google.inject.Scopes; import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.airlift.units.Duration; import io.trino.connector.system.GlobalSystemConnector; import io.trino.metadata.CatalogManager; import io.trino.server.ServerConfig; import io.trino.spi.catalog.CatalogStore; import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.trino.server.InternalCommunicationHttpClientModule.internalHttpClientModule; +import static java.util.concurrent.TimeUnit.SECONDS; public class DynamicCatalogManagerModule extends AbstractConfigurationAwareModule @@ -41,6 +44,11 @@ protected void setup(Binder binder) configBinder(binder).bindConfig(CatalogPruneTaskConfig.class); binder.bind(CatalogPruneTask.class).in(Scopes.SINGLETON); + install(internalHttpClientModule("catalog-prune", ForCatalogPrune.class) + .withConfigDefaults(config -> { + config.setIdleTimeout(new Duration(30, SECONDS)); + config.setRequestTimeout(new Duration(10, SECONDS)); + }).build()); } else { binder.bind(WorkerDynamicCatalogManager.class).in(Scopes.SINGLETON); diff --git a/core/trino-main/src/main/java/io/trino/connector/ForCatalogPrune.java b/core/trino-main/src/main/java/io/trino/connector/ForCatalogPrune.java new file mode 100644 index 000000000000..a2bb5a0d2ed2 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/connector/ForCatalogPrune.java @@ -0,0 +1,31 @@ +/* + * 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.connector; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target({FIELD, PARAMETER, METHOD}) +@BindingAnnotation +public @interface ForCatalogPrune +{ +} diff --git a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaConnector.java b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaConnector.java index a629b35e8c36..66393acadc11 100644 --- a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaConnector.java +++ b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaConnector.java @@ -13,8 +13,8 @@ */ package io.trino.connector.informationschema; -import io.trino.metadata.InternalNodeManager; import io.trino.metadata.Metadata; +import io.trino.node.InternalNode; import io.trino.security.AccessControl; import io.trino.spi.connector.ConnectorMetadata; import io.trino.spi.connector.ConnectorPageSourceProvider; @@ -34,13 +34,13 @@ public class InformationSchemaConnector private final ConnectorSplitManager splitManager; private final ConnectorPageSourceProvider pageSourceProvider; - public InformationSchemaConnector(String catalogName, InternalNodeManager nodeManager, Metadata metadata, AccessControl accessControl, int maxPrefetchedInformationSchemaPrefixes) + public InformationSchemaConnector(String catalogName, InternalNode currentNode, Metadata metadata, AccessControl accessControl, int maxPrefetchedInformationSchemaPrefixes) { requireNonNull(catalogName, "catalogName is null"); requireNonNull(metadata, "metadata is null"); this.metadata = new InformationSchemaMetadata(catalogName, metadata, maxPrefetchedInformationSchemaPrefixes); - this.splitManager = new InformationSchemaSplitManager(nodeManager); + this.splitManager = new InformationSchemaSplitManager(currentNode.getHostAndPort()); this.pageSourceProvider = new InformationSchemaPageSourceProvider(metadata, accessControl); } diff --git a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaSplit.java b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaSplit.java index 4aff81d24096..a48ffcc51b3f 100644 --- a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaSplit.java +++ b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaSplit.java @@ -13,37 +13,19 @@ */ package io.trino.connector.informationschema; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import io.trino.spi.HostAddress; import io.trino.spi.connector.ConnectorSplit; import java.util.List; -import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkArgument; -import static io.airlift.slice.SizeOf.estimatedSizeOf; import static io.airlift.slice.SizeOf.instanceSize; -import static java.util.Objects.requireNonNull; -import static java.util.stream.Collectors.joining; -public class InformationSchemaSplit +public record InformationSchemaSplit(HostAddress address) implements ConnectorSplit { private static final int INSTANCE_SIZE = instanceSize(InformationSchemaSplit.class); - private final List addresses; - - @JsonCreator - public InformationSchemaSplit( - @JsonProperty("addresses") List addresses) - { - requireNonNull(addresses, "addresses is null"); - checkArgument(!addresses.isEmpty(), "addresses is empty"); - this.addresses = ImmutableList.copyOf(addresses); - } - @Override public boolean isRemotelyAccessible() { @@ -51,24 +33,14 @@ public boolean isRemotelyAccessible() } @Override - @JsonProperty public List getAddresses() { - return addresses; - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("addresses", addresses.stream().map(HostAddress::toString).collect(joining(","))) - .toString(); + return ImmutableList.of(address); } @Override public long getRetainedSizeInBytes() { - return INSTANCE_SIZE - + estimatedSizeOf(addresses, HostAddress::getRetainedSizeInBytes); + return INSTANCE_SIZE + address.getRetainedSizeInBytes(); } } diff --git a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaSplitManager.java b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaSplitManager.java index 0924eff62137..c8d7aeb6a10a 100644 --- a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaSplitManager.java +++ b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaSplitManager.java @@ -13,11 +13,8 @@ */ package io.trino.connector.informationschema; -import com.google.common.collect.ImmutableList; -import io.trino.metadata.InternalNodeManager; import io.trino.spi.HostAddress; import io.trino.spi.connector.ConnectorSession; -import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorSplitSource; import io.trino.spi.connector.ConnectorTableHandle; @@ -26,18 +23,14 @@ import io.trino.spi.connector.DynamicFilter; import io.trino.spi.connector.FixedSplitSource; -import java.util.List; - -import static java.util.Objects.requireNonNull; - public class InformationSchemaSplitManager implements ConnectorSplitManager { - private final InternalNodeManager nodeManager; + private final InformationSchemaSplit split; - public InformationSchemaSplitManager(InternalNodeManager nodeManager) + public InformationSchemaSplitManager(HostAddress hostAndPort) { - this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); + split = new InformationSchemaSplit(hostAndPort); } @Override @@ -48,8 +41,6 @@ public ConnectorSplitSource getSplits( DynamicFilter dynamicFilter, Constraint constraint) { - List localAddress = ImmutableList.of(nodeManager.getCurrentNode().getHostAndPort()); - ConnectorSplit split = new InformationSchemaSplit(localAddress); - return new FixedSplitSource(ImmutableList.of(split)); + return new FixedSplitSource(split); } } diff --git a/core/trino-main/src/main/java/io/trino/connector/system/NodeSystemTable.java b/core/trino-main/src/main/java/io/trino/connector/system/NodeSystemTable.java index 4f573a063779..2af9407cdb50 100644 --- a/core/trino-main/src/main/java/io/trino/connector/system/NodeSystemTable.java +++ b/core/trino-main/src/main/java/io/trino/connector/system/NodeSystemTable.java @@ -14,10 +14,10 @@ package io.trino.connector.system; import com.google.inject.Inject; -import io.trino.metadata.AllNodes; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; -import io.trino.metadata.NodeState; +import io.trino.node.AllNodes; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; +import io.trino.node.NodeState; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorTableMetadata; import io.trino.spi.connector.ConnectorTransactionHandle; @@ -32,11 +32,11 @@ import java.util.Set; import static io.trino.metadata.MetadataUtil.TableMetadataBuilder.tableMetadataBuilder; -import static io.trino.metadata.NodeState.ACTIVE; -import static io.trino.metadata.NodeState.DRAINED; -import static io.trino.metadata.NodeState.DRAINING; -import static io.trino.metadata.NodeState.INACTIVE; -import static io.trino.metadata.NodeState.SHUTTING_DOWN; +import static io.trino.node.NodeState.ACTIVE; +import static io.trino.node.NodeState.DRAINED; +import static io.trino.node.NodeState.DRAINING; +import static io.trino.node.NodeState.INACTIVE; +import static io.trino.node.NodeState.SHUTTING_DOWN; import static io.trino.spi.connector.SystemTable.Distribution.SINGLE_COORDINATOR; import static io.trino.spi.type.BooleanType.BOOLEAN; import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; @@ -80,11 +80,11 @@ public RecordCursor cursor(ConnectorTransactionHandle transactionHandle, Connect { Builder table = InMemoryRecordSet.builder(NODES_TABLE); AllNodes allNodes = nodeManager.getAllNodes(); - addRows(table, allNodes.getActiveNodes(), ACTIVE); - addRows(table, allNodes.getInactiveNodes(), INACTIVE); - addRows(table, allNodes.getShuttingDownNodes(), SHUTTING_DOWN); - addRows(table, allNodes.getDrainingNodes(), DRAINING); - addRows(table, allNodes.getDrainedNodes(), DRAINED); + addRows(table, allNodes.activeNodes(), ACTIVE); + addRows(table, allNodes.inactiveNodes(), INACTIVE); + addRows(table, allNodes.shuttingDownNodes(), SHUTTING_DOWN); + addRows(table, allNodes.drainingNodes(), DRAINING); + addRows(table, allNodes.drainedNodes(), DRAINED); return table.build().cursor(); } diff --git a/core/trino-main/src/main/java/io/trino/connector/system/SystemConnector.java b/core/trino-main/src/main/java/io/trino/connector/system/SystemConnector.java index a77bc4c7beba..7775fcf920bf 100644 --- a/core/trino-main/src/main/java/io/trino/connector/system/SystemConnector.java +++ b/core/trino-main/src/main/java/io/trino/connector/system/SystemConnector.java @@ -13,7 +13,8 @@ */ package io.trino.connector.system; -import io.trino.metadata.InternalNodeManager; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.security.AccessControl; import io.trino.spi.connector.ConnectorMetadata; import io.trino.spi.connector.ConnectorPageSourceProvider; @@ -37,12 +38,14 @@ public class SystemConnector private final Function transactionHandleFunction; public SystemConnector( + InternalNode currentNode, InternalNodeManager nodeManager, SystemTablesProvider tables, Function transactionHandleFunction, AccessControl accessControl, String catalogName) { + requireNonNull(currentNode, "currentNode is null"); requireNonNull(nodeManager, "nodeManager is null"); requireNonNull(tables, "tables is null"); requireNonNull(transactionHandleFunction, "transactionHandleFunction is null"); @@ -50,7 +53,7 @@ public SystemConnector( requireNonNull(catalogName, "catalogName is null"); this.metadata = new SystemTablesMetadata(tables); - this.splitManager = new SystemSplitManager(nodeManager, tables); + this.splitManager = new SystemSplitManager(currentNode, nodeManager, tables); this.pageSourceProvider = new SystemPageSourceProvider(tables, accessControl, catalogName); this.transactionHandleFunction = transactionHandleFunction; } diff --git a/core/trino-main/src/main/java/io/trino/connector/system/SystemSplitManager.java b/core/trino-main/src/main/java/io/trino/connector/system/SystemSplitManager.java index 57b02743cdca..814eea7ddbf6 100644 --- a/core/trino-main/src/main/java/io/trino/connector/system/SystemSplitManager.java +++ b/core/trino-main/src/main/java/io/trino/connector/system/SystemSplitManager.java @@ -15,8 +15,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.spi.HostAddress; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorSession; @@ -36,7 +36,7 @@ import java.util.Optional; import java.util.Set; -import static io.trino.metadata.NodeState.ACTIVE; +import static io.trino.node.NodeState.ACTIVE; import static io.trino.spi.connector.SystemTable.Distribution.ALL_COORDINATORS; import static io.trino.spi.connector.SystemTable.Distribution.ALL_NODES; import static io.trino.spi.connector.SystemTable.Distribution.SINGLE_COORDINATOR; @@ -45,11 +45,13 @@ public class SystemSplitManager implements ConnectorSplitManager { + private final InternalNode currentNode; private final InternalNodeManager nodeManager; private final SystemTablesProvider tables; - public SystemSplitManager(InternalNodeManager nodeManager, SystemTablesProvider tables) + public SystemSplitManager(InternalNode currentNode, InternalNodeManager nodeManager, SystemTablesProvider tables) { + this.currentNode = requireNonNull(currentNode, "currentNode is null"); this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.tables = requireNonNull(tables, "tables is null"); } @@ -76,7 +78,7 @@ public ConnectorSplitSource getSplits( Distribution tableDistributionMode = systemTable.getDistribution(); if (tableDistributionMode == SINGLE_COORDINATOR) { - HostAddress address = nodeManager.getCurrentNode().getHostAndPort(); + HostAddress address = currentNode.getHostAndPort(); ConnectorSplit split = new SystemSplit(address, tableConstraint, Optional.empty()); return new FixedSplitSource(ImmutableList.of(split)); } diff --git a/core/trino-main/src/main/java/io/trino/connector/system/jdbc/TableJdbcTable.java b/core/trino-main/src/main/java/io/trino/connector/system/jdbc/TableJdbcTable.java index 4acd8d9d7fa9..1f2a4d1dd99d 100644 --- a/core/trino-main/src/main/java/io/trino/connector/system/jdbc/TableJdbcTable.java +++ b/core/trino-main/src/main/java/io/trino/connector/system/jdbc/TableJdbcTable.java @@ -19,9 +19,9 @@ import io.trino.Session; import io.trino.connector.system.SystemColumnHandle; import io.trino.connector.system.SystemSplit; -import io.trino.metadata.InternalNodeManager; import io.trino.metadata.Metadata; import io.trino.metadata.QualifiedTablePrefix; +import io.trino.node.InternalNode; import io.trino.security.AccessControl; import io.trino.spi.HostAddress; import io.trino.spi.connector.ColumnHandle; @@ -76,14 +76,14 @@ public class TableJdbcTable private final Metadata metadata; private final AccessControl accessControl; - private final InternalNodeManager nodeManager; + private final InternalNode currentNode; @Inject - public TableJdbcTable(Metadata metadata, AccessControl accessControl, InternalNodeManager nodeManager) + public TableJdbcTable(Metadata metadata, AccessControl accessControl, InternalNode currentNode) { this.metadata = requireNonNull(metadata, "metadata is null"); this.accessControl = requireNonNull(accessControl, "accessControl is null"); - this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); + this.currentNode = requireNonNull(currentNode, "currentNode is null"); } @Override @@ -143,7 +143,7 @@ public Optional splitSource(ConnectorSession connectorSess Session session = ((FullConnectorSession) connectorSession).getSession(); // This is an implementation of SINGLE_COORDINATOR distribution for this table. - HostAddress address = nodeManager.getCurrentNode().getHostAndPort(); + HostAddress address = currentNode.getHostAndPort(); List splits = listCatalogNames(session, metadata, accessControl, catalogDomain).stream() .map(catalog -> new SystemSplit(address, constraint, Optional.of(catalog))) .collect(toImmutableList()); diff --git a/core/trino-main/src/main/java/io/trino/cost/TaskCountEstimator.java b/core/trino-main/src/main/java/io/trino/cost/TaskCountEstimator.java index c78a11de5bb3..99caa3e18a4a 100644 --- a/core/trino-main/src/main/java/io/trino/cost/TaskCountEstimator.java +++ b/core/trino-main/src/main/java/io/trino/cost/TaskCountEstimator.java @@ -16,8 +16,8 @@ import com.google.inject.Inject; import io.trino.Session; import io.trino.execution.scheduler.NodeSchedulerConfig; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.operator.RetryPolicy; import java.util.Set; @@ -42,7 +42,7 @@ public TaskCountEstimator(NodeSchedulerConfig nodeSchedulerConfig, InternalNodeM boolean schedulerIncludeCoordinator = nodeSchedulerConfig.isIncludeCoordinator(); requireNonNull(nodeManager, "nodeManager is null"); this.numberOfNodes = () -> { - Set activeNodes = nodeManager.getAllNodes().getActiveNodes(); + Set activeNodes = nodeManager.getAllNodes().activeNodes(); int count; if (schedulerIncludeCoordinator) { count = activeNodes.size(); diff --git a/core/trino-main/src/main/java/io/trino/execution/ClusterSizeMonitor.java b/core/trino-main/src/main/java/io/trino/execution/ClusterSizeMonitor.java index 62d355ba81af..220818b93ccb 100644 --- a/core/trino-main/src/main/java/io/trino/execution/ClusterSizeMonitor.java +++ b/core/trino-main/src/main/java/io/trino/execution/ClusterSizeMonitor.java @@ -21,8 +21,8 @@ import com.google.inject.Inject; import io.airlift.units.Duration; import io.trino.execution.scheduler.NodeSchedulerConfig; -import io.trino.metadata.AllNodes; -import io.trino.metadata.InternalNodeManager; +import io.trino.node.AllNodes; +import io.trino.node.InternalNodeManager; import io.trino.spi.TrinoException; import jakarta.annotation.PostConstruct; import jakarta.annotation.PreDestroy; @@ -137,10 +137,10 @@ private synchronized void removeFuture(MinNodesFuture minNodesFuture) private synchronized void updateAllNodes(AllNodes allNodes) { if (includeCoordinator) { - currentCount = allNodes.getActiveNodes().size(); + currentCount = allNodes.activeNodes().size(); } else { - currentCount = Sets.difference(allNodes.getActiveNodes(), allNodes.getActiveCoordinators()).size(); + currentCount = Sets.difference(allNodes.activeNodes(), allNodes.activeCoordinators()).size(); } ImmutableList.Builder> listenersBuilder = ImmutableList.builder(); diff --git a/core/trino-main/src/main/java/io/trino/execution/LocationFactory.java b/core/trino-main/src/main/java/io/trino/execution/LocationFactory.java index 5a9de6e1f53f..e95c0f17a16b 100644 --- a/core/trino-main/src/main/java/io/trino/execution/LocationFactory.java +++ b/core/trino-main/src/main/java/io/trino/execution/LocationFactory.java @@ -13,7 +13,7 @@ */ package io.trino.execution; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import io.trino.spi.QueryId; import java.net.URI; diff --git a/core/trino-main/src/main/java/io/trino/execution/MemoryTrackingRemoteTaskFactory.java b/core/trino-main/src/main/java/io/trino/execution/MemoryTrackingRemoteTaskFactory.java index 8c645315a9b3..37b4374f5427 100644 --- a/core/trino-main/src/main/java/io/trino/execution/MemoryTrackingRemoteTaskFactory.java +++ b/core/trino-main/src/main/java/io/trino/execution/MemoryTrackingRemoteTaskFactory.java @@ -20,8 +20,8 @@ import io.trino.execution.NodeTaskMap.PartitionedSplitCountTracker; import io.trino.execution.StateMachine.StateChangeListener; import io.trino.execution.buffer.OutputBuffers; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.sql.planner.PlanFragment; import io.trino.sql.planner.plan.DynamicFilterId; import io.trino.sql.planner.plan.PlanNodeId; diff --git a/core/trino-main/src/main/java/io/trino/execution/NodeTaskMap.java b/core/trino-main/src/main/java/io/trino/execution/NodeTaskMap.java index aefe550d4543..42aa2df465e7 100644 --- a/core/trino-main/src/main/java/io/trino/execution/NodeTaskMap.java +++ b/core/trino-main/src/main/java/io/trino/execution/NodeTaskMap.java @@ -17,7 +17,7 @@ import com.google.errorprone.annotations.ThreadSafe; import com.google.inject.Inject; import io.airlift.log.Logger; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import io.trino.util.FinalizerService; import java.util.Set; diff --git a/core/trino-main/src/main/java/io/trino/execution/RemoteTaskFactory.java b/core/trino-main/src/main/java/io/trino/execution/RemoteTaskFactory.java index 4b232c9e377d..22a07966b9cc 100644 --- a/core/trino-main/src/main/java/io/trino/execution/RemoteTaskFactory.java +++ b/core/trino-main/src/main/java/io/trino/execution/RemoteTaskFactory.java @@ -19,8 +19,8 @@ import io.trino.Session; import io.trino.execution.NodeTaskMap.PartitionedSplitCountTracker; import io.trino.execution.buffer.OutputBuffers; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.sql.planner.PlanFragment; import io.trino.sql.planner.plan.DynamicFilterId; import io.trino.sql.planner.plan.PlanNodeId; diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlQueryExecution.java b/core/trino-main/src/main/java/io/trino/execution/SqlQueryExecution.java index b96e8d916b55..da4ba13e0964 100644 --- a/core/trino-main/src/main/java/io/trino/execution/SqlQueryExecution.java +++ b/core/trino-main/src/main/java/io/trino/execution/SqlQueryExecution.java @@ -46,8 +46,8 @@ import io.trino.execution.scheduler.faulttolerant.TaskDescriptorStorage; import io.trino.execution.scheduler.policy.ExecutionPolicy; import io.trino.execution.warnings.WarningCollector; -import io.trino.failuredetector.FailureDetector; import io.trino.metadata.TableHandle; +import io.trino.node.InternalNodeManager; import io.trino.operator.ForScheduler; import io.trino.operator.RetryPolicy; import io.trino.server.BasicQueryInfo; @@ -129,7 +129,7 @@ public class SqlQueryExecution private final int scheduleSplitBatchSize; private final ExecutorService queryExecutor; private final ScheduledExecutorService schedulerExecutor; - private final FailureDetector failureDetector; + private final InternalNodeManager nodeManager; private final AtomicReference queryScheduler = new AtomicReference<>(); private final AtomicReference queryPlan = new AtomicReference<>(); @@ -169,7 +169,7 @@ private SqlQueryExecution( int scheduleSplitBatchSize, ExecutorService queryExecutor, ScheduledExecutorService schedulerExecutor, - FailureDetector failureDetector, + InternalNodeManager nodeManager, NodeTaskMap nodeTaskMap, ExecutionPolicy executionPolicy, SplitSchedulerStats schedulerStats, @@ -200,7 +200,7 @@ private SqlQueryExecution( this.planFragmenter = requireNonNull(planFragmenter, "planFragmenter is null"); this.queryExecutor = requireNonNull(queryExecutor, "queryExecutor is null"); this.schedulerExecutor = requireNonNull(schedulerExecutor, "schedulerExecutor is null"); - this.failureDetector = requireNonNull(failureDetector, "failureDetector is null"); + this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.nodeTaskMap = requireNonNull(nodeTaskMap, "nodeTaskMap is null"); this.executionPolicy = requireNonNull(executionPolicy, "executionPolicy is null"); this.schedulerStats = requireNonNull(schedulerStats, "schedulerStats is null"); @@ -538,7 +538,7 @@ private void planDistribution(PlanRoot plan, CachingTableStatsProvider tableStat scheduleSplitBatchSize, queryExecutor, schedulerExecutor, - failureDetector, + nodeManager, nodeTaskMap, executionPolicy, tracer, @@ -565,7 +565,7 @@ private void planDistribution(PlanRoot plan, CachingTableStatsProvider tableStat nodePartitioningManager, exchangeManagerRegistry.getExchangeManager(), nodeAllocatorService, - failureDetector, + nodeManager, dynamicFilterService, taskExecutionStats, new AdaptivePlanner( @@ -790,7 +790,7 @@ public static class SqlQueryExecutionFactory private final RemoteTaskFactory remoteTaskFactory; private final ExecutorService queryExecutor; private final ScheduledExecutorService schedulerExecutor; - private final FailureDetector failureDetector; + private final InternalNodeManager nodeManager; private final NodeTaskMap nodeTaskMap; private final Map executionPolicies; private final StatsCalculator statsCalculator; @@ -821,7 +821,7 @@ public static class SqlQueryExecutionFactory RemoteTaskFactory remoteTaskFactory, @ForQueryExecution ExecutorService queryExecutor, @ForScheduler ScheduledExecutorService schedulerExecutor, - FailureDetector failureDetector, + InternalNodeManager nodeManager, NodeTaskMap nodeTaskMap, Map executionPolicies, SplitSchedulerStats schedulerStats, @@ -851,7 +851,7 @@ public static class SqlQueryExecutionFactory this.remoteTaskFactory = requireNonNull(remoteTaskFactory, "remoteTaskFactory is null"); this.queryExecutor = requireNonNull(queryExecutor, "queryExecutor is null"); this.schedulerExecutor = requireNonNull(schedulerExecutor, "schedulerExecutor is null"); - this.failureDetector = requireNonNull(failureDetector, "failureDetector is null"); + this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.nodeTaskMap = requireNonNull(nodeTaskMap, "nodeTaskMap is null"); this.executionPolicies = requireNonNull(executionPolicies, "executionPolicies is null"); requireNonNull(planOptimizersFactory, "planOptimizersFactory is null"); @@ -901,7 +901,7 @@ public QueryExecution createQueryExecution( scheduleSplitBatchSize, queryExecutor, schedulerExecutor, - failureDetector, + nodeManager, nodeTaskMap, executionPolicy, schedulerStats, diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlStage.java b/core/trino-main/src/main/java/io/trino/execution/SqlStage.java index fda974055a00..6d05849e4b7e 100644 --- a/core/trino-main/src/main/java/io/trino/execution/SqlStage.java +++ b/core/trino-main/src/main/java/io/trino/execution/SqlStage.java @@ -24,8 +24,8 @@ import io.trino.execution.StateMachine.StateChangeListener; import io.trino.execution.buffer.OutputBuffers; import io.trino.execution.scheduler.SplitSchedulerStats; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.spi.metrics.Metrics; import io.trino.sql.planner.PartitioningHandle; import io.trino.sql.planner.PlanFragment; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/BucketNodeMap.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/BucketNodeMap.java index 1c2375826ed1..5f1ed08f86a0 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/BucketNodeMap.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/BucketNodeMap.java @@ -14,8 +14,8 @@ package io.trino.execution.scheduler; import com.google.common.collect.ImmutableList; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import java.util.List; import java.util.function.ToIntFunction; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/DynamicSplitPlacementPolicy.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/DynamicSplitPlacementPolicy.java index 3652fe293b79..089510122138 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/DynamicSplitPlacementPolicy.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/DynamicSplitPlacementPolicy.java @@ -14,8 +14,8 @@ package io.trino.execution.scheduler; import io.trino.execution.RemoteTask; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import java.util.List; import java.util.Set; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedCountScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedCountScheduler.java index 84f6aa3e714a..d90e4cffc6d2 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedCountScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedCountScheduler.java @@ -16,7 +16,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMultimap; import io.trino.execution.RemoteTask; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import java.util.List; import java.util.Optional; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java index 39522e92cc13..1034bcf66d93 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java @@ -20,8 +20,8 @@ import io.airlift.log.Logger; import io.trino.execution.RemoteTask; import io.trino.execution.TableExecuteContextManager; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.server.DynamicFilterService; import io.trino.split.SplitSource; import io.trino.sql.planner.plan.PlanNodeId; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/MultiSourcePartitionedScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/MultiSourcePartitionedScheduler.java index 6652708d05e0..1a20aa19e6fb 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/MultiSourcePartitionedScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/MultiSourcePartitionedScheduler.java @@ -21,7 +21,7 @@ import io.trino.annotation.NotThreadSafe; import io.trino.execution.RemoteTask; import io.trino.execution.TableExecuteContextManager; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import io.trino.server.DynamicFilterService; import io.trino.split.SplitSource; import io.trino.sql.planner.plan.PlanNodeId; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeAssignmentStats.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeAssignmentStats.java index a86cf677fe93..33b052e66f3d 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeAssignmentStats.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeAssignmentStats.java @@ -16,7 +16,7 @@ import io.trino.execution.NodeTaskMap; import io.trino.execution.PartitionedSplitsInfo; import io.trino.execution.RemoteTask; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import io.trino.spi.SplitWeight; import java.util.HashMap; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeMap.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeMap.java index c1f99332e280..177aca9e792e 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeMap.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeMap.java @@ -14,7 +14,7 @@ package io.trino.execution.scheduler; import com.google.common.collect.SetMultimap; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import io.trino.spi.HostAddress; import java.net.InetAddress; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeScheduler.java index 38fa02bc8563..b8c8f16c9610 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeScheduler.java @@ -23,8 +23,8 @@ import io.trino.Session; import io.trino.execution.NodeTaskMap; import io.trino.execution.RemoteTask; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.spi.HostAddress; import io.trino.spi.SplitWeight; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeSelector.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeSelector.java index 217715d2754a..05838f641391 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeSelector.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/NodeSelector.java @@ -15,8 +15,8 @@ import com.google.common.collect.ImmutableSet; import io.trino.execution.RemoteTask; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import java.util.List; import java.util.Set; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedQueryScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedQueryScheduler.java index f1564f285a44..b66df07775e7 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedQueryScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedQueryScheduler.java @@ -52,9 +52,9 @@ import io.trino.execution.scheduler.policy.ExecutionPolicy; import io.trino.execution.scheduler.policy.ExecutionSchedule; import io.trino.execution.scheduler.policy.StagesScheduleResult; -import io.trino.failuredetector.FailureDetector; -import io.trino.metadata.InternalNode; import io.trino.metadata.Metadata; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.operator.RetryPolicy; import io.trino.server.DynamicFilterService; import io.trino.spi.ErrorCode; @@ -172,7 +172,7 @@ public class PipelinedQueryScheduler private final int splitBatchSize; private final ExecutorService executor; private final ScheduledExecutorService schedulerExecutor; - private final FailureDetector failureDetector; + private final InternalNodeManager nodeManager; private final ExecutionPolicy executionPolicy; private final SplitSchedulerStats schedulerStats; private final DynamicFilterService dynamicFilterService; @@ -208,7 +208,7 @@ public PipelinedQueryScheduler( int splitBatchSize, ExecutorService queryExecutor, ScheduledExecutorService schedulerExecutor, - FailureDetector failureDetector, + InternalNodeManager nodeManager, NodeTaskMap nodeTaskMap, ExecutionPolicy executionPolicy, Tracer tracer, @@ -225,7 +225,7 @@ public PipelinedQueryScheduler( this.splitBatchSize = splitBatchSize; this.executor = requireNonNull(queryExecutor, "queryExecutor is null"); this.schedulerExecutor = requireNonNull(schedulerExecutor, "schedulerExecutor is null"); - this.failureDetector = requireNonNull(failureDetector, "failureDetector is null"); + this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.executionPolicy = requireNonNull(executionPolicy, "executionPolicy is null"); this.schedulerStats = requireNonNull(schedulerStats, "schedulerStats is null"); this.dynamicFilterService = requireNonNull(dynamicFilterService, "dynamicFilterService is null"); @@ -252,7 +252,7 @@ public PipelinedQueryScheduler( queryStateMachine, nodeScheduler, stageManager, - failureDetector, + nodeManager, schedulerExecutor, distributedStagesScheduler, coordinatorTaskManager); @@ -333,7 +333,7 @@ private synchronized Optional createDistributedStage stageManager, coordinatorStagesScheduler, executionPolicy, - failureDetector, + nodeManager, schedulerExecutor, splitSourceFactory, splitBatchSize, @@ -552,7 +552,7 @@ public static CoordinatorStagesScheduler create( QueryStateMachine queryStateMachine, NodeScheduler nodeScheduler, StageManager stageManager, - FailureDetector failureDetector, + InternalNodeManager nodeManager, Executor executor, AtomicReference distributedStagesScheduler, SqlTaskManager coordinatorTaskManager) @@ -568,7 +568,7 @@ public static CoordinatorStagesScheduler create( stage, outputBuffersForStagesConsumedByCoordinator, taskLifecycleListener, - failureDetector, + nodeManager, executor, bucketToPartitionForStagesConsumedByCoordinator.get(stage.getFragment().getId()).map(BucketToPartition::bucketToPartition), OptionalInt.empty(), @@ -864,7 +864,7 @@ public static DistributedStagesScheduler create( StageManager stageManager, CoordinatorStagesScheduler coordinatorStagesScheduler, ExecutionPolicy executionPolicy, - FailureDetector failureDetector, + InternalNodeManager nodeManager, ScheduledExecutorService executor, SplitSourceFactory splitSourceFactory, int splitBatchSize, @@ -933,7 +933,7 @@ public static DistributedStagesScheduler create( stageManager.get(fragment.getId()), outputBufferManagers, taskLifecycleListener, - failureDetector, + nodeManager, executor, bucketToPartition.map(BucketToPartition::bucketToPartition), skewedBucketCount, diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedStageExecution.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedStageExecution.java index 0d44da600b76..dbf2f31bfc7b 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedStageExecution.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedStageExecution.java @@ -35,9 +35,9 @@ import io.trino.execution.buffer.OutputBufferStatus; import io.trino.execution.buffer.OutputBuffers; import io.trino.execution.buffer.PipelinedOutputBuffers.OutputBufferId; -import io.trino.failuredetector.FailureDetector; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.spi.TrinoException; import io.trino.spi.metrics.Metrics; import io.trino.split.RemoteSplit; @@ -76,7 +76,6 @@ import static io.trino.execution.scheduler.StageExecution.State.SCHEDULED; import static io.trino.execution.scheduler.StageExecution.State.SCHEDULING; import static io.trino.execution.scheduler.StageExecution.State.SCHEDULING_SPLITS; -import static io.trino.failuredetector.FailureDetector.State.GONE; import static io.trino.operator.ExchangeOperator.REMOTE_CATALOG_HANDLE; import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static io.trino.spi.StandardErrorCode.REMOTE_HOST_GONE; @@ -112,7 +111,7 @@ public class PipelinedStageExecution private final SqlStage stage; private final Map outputBufferManagers; private final TaskLifecycleListener taskLifecycleListener; - private final FailureDetector failureDetector; + private final InternalNodeManager nodeManager; private final Optional bucketToPartition; private final OptionalInt skewedBucketCount; private final Map exchangeSources; @@ -138,7 +137,7 @@ public static PipelinedStageExecution createPipelinedStageExecution( SqlStage stage, Map outputBufferManagers, TaskLifecycleListener taskLifecycleListener, - FailureDetector failureDetector, + InternalNodeManager nodeManager, Executor executor, Optional bucketToPartition, OptionalInt skewedBucketCount, @@ -156,7 +155,7 @@ public static PipelinedStageExecution createPipelinedStageExecution( stage, outputBufferManagers, taskLifecycleListener, - failureDetector, + nodeManager, bucketToPartition, skewedBucketCount, exchangeSources.buildOrThrow(), @@ -170,7 +169,7 @@ private PipelinedStageExecution( SqlStage stage, Map outputBufferManagers, TaskLifecycleListener taskLifecycleListener, - FailureDetector failureDetector, + InternalNodeManager nodeManager, Optional bucketToPartition, OptionalInt skewedBucketCount, Map exchangeSources, @@ -180,7 +179,7 @@ private PipelinedStageExecution( this.stage = requireNonNull(stage, "stage is null"); this.outputBufferManagers = ImmutableMap.copyOf(requireNonNull(outputBufferManagers, "outputBufferManagers is null")); this.taskLifecycleListener = requireNonNull(taskLifecycleListener, "taskLifecycleListener is null"); - this.failureDetector = requireNonNull(failureDetector, "failureDetector is null"); + this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.bucketToPartition = requireNonNull(bucketToPartition, "bucketToPartition is null"); this.skewedBucketCount = requireNonNull(skewedBucketCount, "skewedBucketCount is null"); this.exchangeSources = ImmutableMap.copyOf(requireNonNull(exchangeSources, "exchangeSources is null")); @@ -443,7 +442,7 @@ private boolean addFinishedTask(TaskId taskId) private ExecutionFailureInfo rewriteTransportFailure(ExecutionFailureInfo executionFailureInfo) { - if (executionFailureInfo.getRemoteHost() == null || failureDetector.getState(executionFailureInfo.getRemoteHost()) != GONE) { + if (executionFailureInfo.getRemoteHost() == null || !nodeManager.isGone(executionFailureInfo.getRemoteHost())) { return executionFailureInfo; } diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/ScaledWriterScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/ScaledWriterScheduler.java index 64d07852f68c..0c2d530b3480 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/ScaledWriterScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/ScaledWriterScheduler.java @@ -20,7 +20,7 @@ import io.trino.execution.RemoteTask; import io.trino.execution.TaskStatus; import io.trino.execution.buffer.OutputBufferStatus; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import java.util.Collection; import java.util.HashSet; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/SourcePartitionedScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/SourcePartitionedScheduler.java index b8cb2ae3d96b..7ebd561e881c 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/SourcePartitionedScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/SourcePartitionedScheduler.java @@ -22,8 +22,8 @@ import io.trino.execution.RemoteTask; import io.trino.execution.TableExecuteContext; import io.trino.execution.TableExecuteContextManager; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.server.DynamicFilterService; import io.trino.split.EmptySplit; import io.trino.split.SplitSource; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/SplitPlacementPolicy.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/SplitPlacementPolicy.java index b49224923365..f804f58570a8 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/SplitPlacementPolicy.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/SplitPlacementPolicy.java @@ -13,8 +13,8 @@ */ package io.trino.execution.scheduler; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import java.util.List; import java.util.Set; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/SplitPlacementResult.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/SplitPlacementResult.java index 12dd1e3c521f..be907ad0864a 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/SplitPlacementResult.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/SplitPlacementResult.java @@ -15,8 +15,8 @@ import com.google.common.collect.Multimap; import com.google.common.util.concurrent.ListenableFuture; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import static java.util.Objects.requireNonNull; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/StageExecution.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/StageExecution.java index 1003826015f2..c9df866782bb 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/StageExecution.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/StageExecution.java @@ -21,8 +21,8 @@ import io.trino.execution.StateMachine.StateChangeListener; import io.trino.execution.TaskId; import io.trino.execution.TaskStatus; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.spi.metrics.Metrics; import io.trino.sql.planner.PlanFragment; import io.trino.sql.planner.plan.PlanNodeId; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/TopologyAwareNodeSelector.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/TopologyAwareNodeSelector.java index b24bb6738738..09a4532b2a5a 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/TopologyAwareNodeSelector.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/TopologyAwareNodeSelector.java @@ -21,9 +21,8 @@ import io.airlift.stats.CounterStat; import io.trino.execution.NodeTaskMap; import io.trino.execution.RemoteTask; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.spi.HostAddress; import io.trino.spi.SplitWeight; import io.trino.spi.TrinoException; @@ -54,7 +53,7 @@ public class TopologyAwareNodeSelector { private static final Logger log = Logger.get(TopologyAwareNodeSelector.class); - private final InternalNodeManager nodeManager; + private final InternalNode currentNode; private final NodeTaskMap nodeTaskMap; private final boolean includeCoordinator; private final AtomicReference> nodeMap; @@ -66,7 +65,7 @@ public class TopologyAwareNodeSelector private final NetworkTopology networkTopology; public TopologyAwareNodeSelector( - InternalNodeManager nodeManager, + InternalNode currentNode, NodeTaskMap nodeTaskMap, boolean includeCoordinator, Supplier nodeMap, @@ -77,7 +76,7 @@ public TopologyAwareNodeSelector( List topologicalSplitCounters, NetworkTopology networkTopology) { - this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); + this.currentNode = requireNonNull(currentNode, "currentNode is null"); this.nodeTaskMap = requireNonNull(nodeTaskMap, "nodeTaskMap is null"); this.includeCoordinator = includeCoordinator; this.nodeMap = new AtomicReference<>(nodeMap); @@ -106,7 +105,7 @@ public List allNodes() public InternalNode selectCurrentNode() { // TODO: this is a hack to force scheduling on the coordinator - return nodeManager.getCurrentNode(); + return currentNode; } @Override diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/TopologyAwareNodeSelectorFactory.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/TopologyAwareNodeSelectorFactory.java index 3ba83c9cc320..1b8a771c50ee 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/TopologyAwareNodeSelectorFactory.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/TopologyAwareNodeSelectorFactory.java @@ -25,8 +25,8 @@ import io.trino.Session; import io.trino.cache.NonEvictableCache; import io.trino.execution.NodeTaskMap; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.spi.HostAddress; import io.trino.spi.SplitWeight; @@ -43,7 +43,7 @@ import static io.trino.SystemSessionProperties.getMaxUnacknowledgedSplitsPerTask; import static io.trino.cache.CacheUtils.uncheckedCacheGet; import static io.trino.cache.SafeCaches.buildNonEvictableCache; -import static io.trino.metadata.NodeState.ACTIVE; +import static io.trino.node.NodeState.ACTIVE; import static java.util.Objects.requireNonNull; public class TopologyAwareNodeSelectorFactory @@ -56,6 +56,7 @@ public class TopologyAwareNodeSelectorFactory .expireAfterWrite(30, TimeUnit.SECONDS)); private final NetworkTopology networkTopology; + private final InternalNode currentNode; private final InternalNodeManager nodeManager; private final int minCandidates; private final boolean includeCoordinator; @@ -69,16 +70,19 @@ public class TopologyAwareNodeSelectorFactory @Inject public TopologyAwareNodeSelectorFactory( NetworkTopology networkTopology, + InternalNode currentNode, InternalNodeManager nodeManager, NodeSchedulerConfig schedulerConfig, NodeTaskMap nodeTaskMap, TopologyAwareNodeSelectorConfig topologyConfig) { requireNonNull(networkTopology, "networkTopology is null"); + requireNonNull(currentNode, "currentNode is null"); requireNonNull(nodeManager, "nodeManager is null"); requireNonNull(nodeTaskMap, "nodeTaskMap is null"); this.networkTopology = networkTopology; + this.currentNode = currentNode; this.nodeManager = nodeManager; this.minCandidates = schedulerConfig.getMinCandidates(); this.includeCoordinator = schedulerConfig.isIncludeCoordinator(); @@ -122,7 +126,7 @@ public NodeSelector createNodeSelector(Session session) 5, TimeUnit.SECONDS); return new TopologyAwareNodeSelector( - nodeManager, + currentNode, nodeTaskMap, includeCoordinator, nodeMap, diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/UniformNodeSelector.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/UniformNodeSelector.java index 612ec675839d..661032a220b6 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/UniformNodeSelector.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/UniformNodeSelector.java @@ -25,9 +25,8 @@ import io.trino.execution.NodeTaskMap; import io.trino.execution.RemoteTask; import io.trino.execution.scheduler.NodeSchedulerConfig.SplitsBalancingPolicy; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.spi.TrinoException; import jakarta.annotation.Nullable; @@ -58,7 +57,7 @@ public class UniformNodeSelector { private static final Logger log = Logger.get(UniformNodeSelector.class); - private final InternalNodeManager nodeManager; + private final InternalNode currentNode; private final NodeTaskMap nodeTaskMap; private final boolean includeCoordinator; private final AtomicReference> nodeMap; @@ -71,7 +70,7 @@ public class UniformNodeSelector private final QueueSizeAdjuster queueSizeAdjuster; public UniformNodeSelector( - InternalNodeManager nodeManager, + InternalNode currentNode, NodeTaskMap nodeTaskMap, boolean includeCoordinator, Supplier nodeMap, @@ -83,7 +82,7 @@ public UniformNodeSelector( SplitsBalancingPolicy splitsBalancingPolicy, boolean optimizedLocalScheduling) { - this(nodeManager, + this(currentNode, nodeTaskMap, includeCoordinator, nodeMap, @@ -98,7 +97,7 @@ public UniformNodeSelector( @VisibleForTesting UniformNodeSelector( - InternalNodeManager nodeManager, + InternalNode currentNode, NodeTaskMap nodeTaskMap, boolean includeCoordinator, Supplier nodeMap, @@ -110,7 +109,7 @@ public UniformNodeSelector( boolean optimizedLocalScheduling, QueueSizeAdjuster queueSizeAdjuster) { - this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); + this.currentNode = requireNonNull(currentNode, "currentNode is null"); this.nodeTaskMap = requireNonNull(nodeTaskMap, "nodeTaskMap is null"); this.includeCoordinator = includeCoordinator; this.nodeMap = new AtomicReference<>(nodeMap); @@ -140,7 +139,7 @@ public List allNodes() public InternalNode selectCurrentNode() { // TODO: this is a hack to force scheduling on the coordinator - return nodeManager.getCurrentNode(); + return currentNode; } @Override diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/UniformNodeSelectorFactory.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/UniformNodeSelectorFactory.java index e800b5e6529b..e493df773155 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/UniformNodeSelectorFactory.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/UniformNodeSelectorFactory.java @@ -24,8 +24,8 @@ import io.trino.cache.NonEvictableCache; import io.trino.execution.NodeTaskMap; import io.trino.execution.scheduler.NodeSchedulerConfig.SplitsBalancingPolicy; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.spi.HostAddress; import io.trino.spi.SplitWeight; @@ -40,7 +40,7 @@ import static io.trino.SystemSessionProperties.getMaxUnacknowledgedSplitsPerTask; import static io.trino.cache.CacheUtils.uncheckedCacheGet; import static io.trino.cache.SafeCaches.buildNonEvictableCache; -import static io.trino.metadata.NodeState.ACTIVE; +import static io.trino.node.NodeState.ACTIVE; import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; @@ -54,6 +54,7 @@ public class UniformNodeSelectorFactory CacheBuilder.newBuilder() .expireAfterWrite(30, TimeUnit.SECONDS)); + private final InternalNode currentNode; private final InternalNodeManager nodeManager; private final int minCandidates; private final boolean includeCoordinator; @@ -67,20 +68,23 @@ public class UniformNodeSelectorFactory @Inject public UniformNodeSelectorFactory( + InternalNode currentNode, InternalNodeManager nodeManager, NodeSchedulerConfig config, NodeTaskMap nodeTaskMap) { - this(nodeManager, config, nodeTaskMap, new Duration(5, SECONDS)); + this(currentNode, nodeManager, config, nodeTaskMap, new Duration(5, SECONDS)); } @VisibleForTesting UniformNodeSelectorFactory( + InternalNode currentNode, InternalNodeManager nodeManager, NodeSchedulerConfig config, NodeTaskMap nodeTaskMap, Duration nodeMapMemoizationDuration) { + this.currentNode = requireNonNull(currentNode, "currentNode is null"); this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.minCandidates = config.getMinCandidates(); this.includeCoordinator = config.isIncludeCoordinator(); @@ -114,7 +118,7 @@ public NodeSelector createNodeSelector(Session session) } return new UniformNodeSelector( - nodeManager, + currentNode, nodeTaskMap, includeCoordinator, nodeMap, diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/BinPackingNodeAllocatorService.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/BinPackingNodeAllocatorService.java index 2bc46424aba2..0381620ab38a 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/BinPackingNodeAllocatorService.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/BinPackingNodeAllocatorService.java @@ -37,9 +37,9 @@ import io.trino.memory.ClusterMemoryManager; import io.trino.memory.MemoryInfo; import io.trino.memory.MemoryManagerConfig; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; -import io.trino.metadata.InternalNodeManager.NodesSnapshot; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; +import io.trino.node.InternalNodeManager.NodesSnapshot; import io.trino.spi.HostAddress; import io.trino.spi.QueryId; import io.trino.spi.TrinoException; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java index 9e9f9e9bcd58..d05c9038f5e1 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java @@ -78,10 +78,10 @@ import io.trino.execution.scheduler.faulttolerant.SplitAssigner.AssignmentResult; import io.trino.execution.scheduler.faulttolerant.SplitAssigner.Partition; import io.trino.execution.scheduler.faulttolerant.SplitAssigner.PartitionUpdate; -import io.trino.failuredetector.FailureDetector; -import io.trino.metadata.InternalNode; import io.trino.metadata.Metadata; import io.trino.metadata.Split; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.operator.RetryPolicy; import io.trino.server.DynamicFilterService; import io.trino.spi.ErrorCode; @@ -172,7 +172,6 @@ import static io.trino.execution.scheduler.faulttolerant.TaskExecutionClass.EAGER_SPECULATIVE; import static io.trino.execution.scheduler.faulttolerant.TaskExecutionClass.SPECULATIVE; import static io.trino.execution.scheduler.faulttolerant.TaskExecutionClass.STANDARD; -import static io.trino.failuredetector.FailureDetector.State.GONE; import static io.trino.operator.ExchangeOperator.REMOTE_CATALOG_HANDLE; import static io.trino.operator.RetryPolicy.TASK; import static io.trino.spi.ErrorType.EXTERNAL; @@ -226,7 +225,7 @@ public class EventDrivenFaultTolerantQueryScheduler private final NodePartitioningManager nodePartitioningManager; private final ExchangeManager exchangeManager; private final NodeAllocatorService nodeAllocatorService; - private final FailureDetector failureDetector; + private final InternalNodeManager nodeManager; private final DynamicFilterService dynamicFilterService; private final TaskExecutionStats taskExecutionStats; private final Optional adaptivePlanner; @@ -258,7 +257,7 @@ public EventDrivenFaultTolerantQueryScheduler( NodePartitioningManager nodePartitioningManager, ExchangeManager exchangeManager, NodeAllocatorService nodeAllocatorService, - FailureDetector failureDetector, + InternalNodeManager nodeManager, DynamicFilterService dynamicFilterService, TaskExecutionStats taskExecutionStats, AdaptivePlanner adaptivePlanner, @@ -283,7 +282,7 @@ public EventDrivenFaultTolerantQueryScheduler( this.nodePartitioningManager = requireNonNull(nodePartitioningManager, "partitioningSchemeFactory is null"); this.exchangeManager = requireNonNull(exchangeManager, "exchangeManager is null"); this.nodeAllocatorService = requireNonNull(nodeAllocatorService, "nodeAllocatorService is null"); - this.failureDetector = requireNonNull(failureDetector, "failureDetector is null"); + this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.dynamicFilterService = requireNonNull(dynamicFilterService, "dynamicFilterService is null"); this.taskExecutionStats = requireNonNull(taskExecutionStats, "taskExecutionStats is null"); this.adaptivePlanner = isFaultTolerantExecutionAdaptiveQueryPlanningEnabled(queryStateMachine.getSession()) ? @@ -359,7 +358,7 @@ public synchronized void start() getMaxTasksWaitingForNodePerQuery(session), getMaxTasksWaitingForExecutionPerQuery(session), nodeAllocator, - failureDetector, + nodeManager, stageRegistry, taskExecutionStats, stageExecutionStats, @@ -705,7 +704,7 @@ private static class Scheduler private final int maxTasksWaitingForNode; private final int maxTasksWaitingForExecution; private final NodeAllocator nodeAllocator; - private final FailureDetector failureDetector; + private final InternalNodeManager nodeManager; private final StageRegistry stageRegistry; private final TaskExecutionStats taskExecutionStats; private final StageExecutionStats stageExecutionStats; @@ -760,7 +759,7 @@ public Scheduler( int maxTasksWaitingForNode, int maxTasksWaitingForExecution, NodeAllocator nodeAllocator, - FailureDetector failureDetector, + InternalNodeManager nodeManager, StageRegistry stageRegistry, TaskExecutionStats taskExecutionStats, StageExecutionStats stageExecutionStats, @@ -792,7 +791,7 @@ public Scheduler( this.maxTasksWaitingForNode = maxTasksWaitingForNode; this.maxTasksWaitingForExecution = maxTasksWaitingForExecution; this.nodeAllocator = requireNonNull(nodeAllocator, "nodeAllocator is null"); - this.failureDetector = requireNonNull(failureDetector, "failureDetector is null"); + this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.stageRegistry = requireNonNull(stageRegistry, "stageRegistry is null"); this.taskExecutionStats = requireNonNull(taskExecutionStats, "taskExecutionStats is null"); this.stageExecutionStats = requireNonNull(stageExecutionStats, "stageExecutionStats is null"); @@ -1887,7 +1886,7 @@ private boolean shouldDelayScheduling(@Nullable ErrorCode errorCode) private ExecutionFailureInfo rewriteTransportFailure(ExecutionFailureInfo executionFailureInfo) { - if (executionFailureInfo.getRemoteHost() == null || failureDetector.getState(executionFailureInfo.getRemoteHost()) != GONE) { + if (executionFailureInfo.getRemoteHost() == null || !nodeManager.isGone(executionFailureInfo.getRemoteHost())) { return executionFailureInfo; } diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenTaskSourceFactory.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenTaskSourceFactory.java index a3b409e254bb..0d255b840004 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenTaskSourceFactory.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenTaskSourceFactory.java @@ -22,9 +22,9 @@ import io.trino.execution.QueryManagerConfig; import io.trino.execution.TableExecuteContextManager; import io.trino.execution.scheduler.OutputDataSizeEstimate; -import io.trino.metadata.InternalNodeManager; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.spi.HostAddress; -import io.trino.spi.Node; import io.trino.spi.exchange.Exchange; import io.trino.sql.planner.MergePartitioningHandle; import io.trino.sql.planner.PartitioningHandle; @@ -75,6 +75,7 @@ public class EventDrivenTaskSourceFactory { private final SplitSourceFactory splitSourceFactory; private final Executor executor; + private final InternalNode currentNode; private final InternalNodeManager nodeManager; private final TableExecuteContextManager tableExecuteContextManager; private final int splitBatchSize; @@ -83,6 +84,7 @@ public class EventDrivenTaskSourceFactory public EventDrivenTaskSourceFactory( SplitSourceFactory splitSourceFactory, @ForQueryExecution ExecutorService executor, + InternalNode currentNode, InternalNodeManager nodeManager, TableExecuteContextManager tableExecuteContextManager, QueryManagerConfig queryManagerConfig) @@ -90,6 +92,7 @@ public EventDrivenTaskSourceFactory( this( splitSourceFactory, executor, + currentNode, nodeManager, tableExecuteContextManager, requireNonNull(queryManagerConfig, "queryManagerConfig is null").getScheduleSplitBatchSize()); @@ -98,12 +101,14 @@ public EventDrivenTaskSourceFactory( public EventDrivenTaskSourceFactory( SplitSourceFactory splitSourceFactory, Executor executor, + InternalNode currentNode, InternalNodeManager nodeManager, TableExecuteContextManager tableExecuteContextManager, int splitBatchSize) { this.splitSourceFactory = requireNonNull(splitSourceFactory, "splitSourceFactory is null"); this.executor = requireNonNull(executor, "executor is null"); + this.currentNode = requireNonNull(currentNode, "currentNode is null"); this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.tableExecuteContextManager = requireNonNull(tableExecuteContextManager, "tableExecuteContextManager is null"); this.splitBatchSize = splitBatchSize; @@ -173,7 +178,6 @@ private SplitAssigner createSplitAssigner( if (partitioning.equals(SINGLE_DISTRIBUTION) || coordinatorOnly) { Optional hostRequirement = Optional.empty(); if (coordinatorOnly) { - Node currentNode = nodeManager.getCurrentNode(); verify(currentNode.isCoordinator(), "current node is expected to be a coordinator"); hostRequirement = Optional.of(currentNode.getHostAndPort()); } @@ -235,7 +239,7 @@ private SplitAssigner createSplitAssigner( outputDataSizeEstimates, fragment, getFaultTolerantExecutionHashDistributionComputeTaskTargetSize(session).toBytes(), - toIntExact(round(getFaultTolerantExecutionHashDistributionComputeTasksToNodesMinRatio(session) * nodeManager.getAllNodes().getActiveNodes().size())), + toIntExact(round(getFaultTolerantExecutionHashDistributionComputeTasksToNodesMinRatio(session) * nodeManager.getAllNodes().activeNodes().size())), Integer.MAX_VALUE); // compute tasks are bounded by the number of partitions anyways } if (partitioning.equals(SCALED_WRITER_HASH_DISTRIBUTION) @@ -249,7 +253,7 @@ private SplitAssigner createSplitAssigner( outputDataSizeEstimates, fragment, getFaultTolerantExecutionHashDistributionWriteTaskTargetSize(session).toBytes(), - toIntExact(round(getFaultTolerantExecutionHashDistributionWriteTasksToNodesMinRatio(session) * nodeManager.getAllNodes().getActiveNodes().size())), + toIntExact(round(getFaultTolerantExecutionHashDistributionWriteTasksToNodesMinRatio(session) * nodeManager.getAllNodes().activeNodes().size())), getFaultTolerantExecutionHashDistributionWriteTaskTargetMaxCount(session)); } diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/FaultTolerantPartitioningScheme.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/FaultTolerantPartitioningScheme.java index 04bc2a290071..c85cfa30c7a6 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/FaultTolerantPartitioningScheme.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/FaultTolerantPartitioningScheme.java @@ -15,8 +15,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import java.util.List; import java.util.Optional; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/FaultTolerantPartitioningSchemeFactory.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/FaultTolerantPartitioningSchemeFactory.java index e0ae5e378881..1f44bab12542 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/FaultTolerantPartitioningSchemeFactory.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/FaultTolerantPartitioningSchemeFactory.java @@ -16,8 +16,8 @@ import com.google.common.collect.ImmutableList; import io.trino.Session; import io.trino.annotation.NotThreadSafe; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.spi.Node; import io.trino.spi.connector.ConnectorBucketNodeMap; import io.trino.sql.planner.MergePartitioningHandle; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/HashDistributionSplitAssigner.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/HashDistributionSplitAssigner.java index 05455b2786b6..a91c5e6c540b 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/HashDistributionSplitAssigner.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/HashDistributionSplitAssigner.java @@ -21,8 +21,8 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.ListMultimap; import io.trino.execution.scheduler.OutputDataSizeEstimate; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.spi.HostAddress; import io.trino.spi.connector.CatalogHandle; import io.trino.sql.planner.PlanFragment; diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/NodeAllocator.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/NodeAllocator.java index 3b6f2db14696..227e4129d31c 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/NodeAllocator.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/NodeAllocator.java @@ -16,7 +16,7 @@ import com.google.common.util.concurrent.ListenableFuture; import io.airlift.units.DataSize; import io.trino.execution.TaskId; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import java.io.Closeable; diff --git a/core/trino-main/src/main/java/io/trino/memory/ClusterMemoryManager.java b/core/trino-main/src/main/java/io/trino/memory/ClusterMemoryManager.java index 586d8dcda949..08177e7e6367 100644 --- a/core/trino-main/src/main/java/io/trino/memory/ClusterMemoryManager.java +++ b/core/trino-main/src/main/java/io/trino/memory/ClusterMemoryManager.java @@ -37,8 +37,8 @@ import io.trino.memory.LowMemoryKiller.ForQueryLowMemoryKiller; import io.trino.memory.LowMemoryKiller.ForTaskLowMemoryKiller; import io.trino.memory.LowMemoryKiller.RunningQueryInfo; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.operator.RetryPolicy; import io.trino.server.BasicQueryInfo; import io.trino.server.ServerConfig; @@ -79,10 +79,10 @@ import static io.trino.SystemSessionProperties.getQueryMaxTotalMemory; import static io.trino.SystemSessionProperties.getRetryPolicy; import static io.trino.SystemSessionProperties.resourceOvercommit; -import static io.trino.metadata.NodeState.ACTIVE; -import static io.trino.metadata.NodeState.DRAINED; -import static io.trino.metadata.NodeState.DRAINING; -import static io.trino.metadata.NodeState.SHUTTING_DOWN; +import static io.trino.node.NodeState.ACTIVE; +import static io.trino.node.NodeState.DRAINED; +import static io.trino.node.NodeState.DRAINING; +import static io.trino.node.NodeState.SHUTTING_DOWN; import static io.trino.spi.StandardErrorCode.CLUSTER_OUT_OF_MEMORY; import static java.lang.Math.min; import static java.lang.String.format; diff --git a/core/trino-main/src/main/java/io/trino/memory/RemoteNodeMemory.java b/core/trino-main/src/main/java/io/trino/memory/RemoteNodeMemory.java index fa3885ce5334..2563370d14c2 100644 --- a/core/trino-main/src/main/java/io/trino/memory/RemoteNodeMemory.java +++ b/core/trino-main/src/main/java/io/trino/memory/RemoteNodeMemory.java @@ -23,7 +23,7 @@ import io.airlift.json.JsonCodec; import io.airlift.log.Logger; import io.airlift.units.Duration; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import java.net.URI; import java.util.Optional; diff --git a/core/trino-main/src/main/java/io/trino/metadata/AllNodes.java b/core/trino-main/src/main/java/io/trino/metadata/AllNodes.java deleted file mode 100644 index 75d34118a95e..000000000000 --- a/core/trino-main/src/main/java/io/trino/metadata/AllNodes.java +++ /dev/null @@ -1,100 +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.metadata; - -import com.google.common.collect.ImmutableSet; - -import java.util.Objects; -import java.util.Set; - -import static java.util.Objects.requireNonNull; - -public class AllNodes -{ - private final Set activeNodes; - private final Set inactiveNodes; - private final Set drainingNodes; - private final Set drainedNodes; - private final Set shuttingDownNodes; - private final Set activeCoordinators; - - public AllNodes(Set activeNodes, - Set inactiveNodes, - Set drainingNodes, - Set drainedNodes, - Set shuttingDownNodes, - Set activeCoordinators) - { - this.activeNodes = ImmutableSet.copyOf(requireNonNull(activeNodes, "activeNodes is null")); - this.inactiveNodes = ImmutableSet.copyOf(requireNonNull(inactiveNodes, "inactiveNodes is null")); - this.drainedNodes = ImmutableSet.copyOf(requireNonNull(drainedNodes, "drainedNodes is null")); - this.drainingNodes = ImmutableSet.copyOf(requireNonNull(drainingNodes, "drainingNodes is null")); - this.shuttingDownNodes = ImmutableSet.copyOf(requireNonNull(shuttingDownNodes, "shuttingDownNodes is null")); - this.activeCoordinators = ImmutableSet.copyOf(requireNonNull(activeCoordinators, "activeCoordinators is null")); - } - - public Set getActiveNodes() - { - return activeNodes; - } - - public Set getInactiveNodes() - { - return inactiveNodes; - } - - public Set getShuttingDownNodes() - { - return shuttingDownNodes; - } - - public Set getDrainedNodes() - { - return drainedNodes; - } - - public Set getDrainingNodes() - { - return drainingNodes; - } - - public Set getActiveCoordinators() - { - return activeCoordinators; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - AllNodes allNodes = (AllNodes) o; - return Objects.equals(activeNodes, allNodes.activeNodes) && - Objects.equals(inactiveNodes, allNodes.inactiveNodes) && - Objects.equals(drainedNodes, allNodes.drainedNodes) && - Objects.equals(drainingNodes, allNodes.drainingNodes) && - Objects.equals(shuttingDownNodes, allNodes.shuttingDownNodes) && - Objects.equals(activeCoordinators, allNodes.activeCoordinators); - } - - @Override - public int hashCode() - { - return Objects.hash(activeNodes, inactiveNodes, drainingNodes, drainedNodes, shuttingDownNodes, activeCoordinators); - } -} diff --git a/core/trino-main/src/main/java/io/trino/metadata/DiscoveryNodeManager.java b/core/trino-main/src/main/java/io/trino/metadata/DiscoveryNodeManager.java deleted file mode 100644 index ce858cdbe731..000000000000 --- a/core/trino-main/src/main/java/io/trino/metadata/DiscoveryNodeManager.java +++ /dev/null @@ -1,387 +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.metadata; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Sets.SetView; -import com.google.errorprone.annotations.ThreadSafe; -import com.google.errorprone.annotations.concurrent.GuardedBy; -import com.google.inject.Inject; -import io.airlift.discovery.client.ServiceDescriptor; -import io.airlift.discovery.client.ServiceSelector; -import io.airlift.discovery.client.ServiceType; -import io.airlift.http.client.HttpClient; -import io.airlift.log.Logger; -import io.airlift.node.NodeInfo; -import io.trino.client.NodeVersion; -import io.trino.failuredetector.FailureDetector; -import io.trino.server.InternalCommunicationConfig; -import jakarta.annotation.PostConstruct; -import jakarta.annotation.PreDestroy; -import org.weakref.jmx.Managed; - -import java.net.URI; -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; - -import static com.google.common.base.Preconditions.checkState; -import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static com.google.common.collect.Sets.difference; -import static io.airlift.concurrent.Threads.daemonThreadsNamed; -import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; -import static io.trino.metadata.NodeState.INACTIVE; -import static java.util.Objects.requireNonNull; -import static java.util.concurrent.Executors.newCachedThreadPool; -import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor; - -@ThreadSafe -public final class DiscoveryNodeManager - implements InternalNodeManager -{ - private static final Logger log = Logger.get(DiscoveryNodeManager.class); - - private final ServiceSelector serviceSelector; - private final FailureDetector failureDetector; - private final NodeVersion expectedNodeVersion; - private final ConcurrentHashMap nodeStates = new ConcurrentHashMap<>(); - private final HttpClient httpClient; - private final ScheduledExecutorService nodeStateUpdateExecutor; - private final ExecutorService nodeStateEventExecutor; - private final boolean httpsRequired; - private final InternalNode currentNode; - - @GuardedBy("this") - private AllNodes allNodes; - - @GuardedBy("this") - private Set coordinators; - - @GuardedBy("this") - private final List> listeners = new ArrayList<>(); - - @Inject - public DiscoveryNodeManager( - @ServiceType("trino") ServiceSelector serviceSelector, - NodeInfo nodeInfo, - FailureDetector failureDetector, - NodeVersion expectedNodeVersion, - @ForNodeManager HttpClient httpClient, - InternalCommunicationConfig internalCommunicationConfig) - { - this.serviceSelector = requireNonNull(serviceSelector, "serviceSelector is null"); - this.failureDetector = requireNonNull(failureDetector, "failureDetector is null"); - this.expectedNodeVersion = requireNonNull(expectedNodeVersion, "expectedNodeVersion is null"); - this.httpClient = requireNonNull(httpClient, "httpClient is null"); - this.nodeStateUpdateExecutor = newSingleThreadScheduledExecutor(daemonThreadsNamed("node-state-poller-%s")); - this.nodeStateEventExecutor = newCachedThreadPool(daemonThreadsNamed("node-state-events-%s")); - this.httpsRequired = internalCommunicationConfig.isHttpsRequired(); - - this.currentNode = findCurrentNode( - serviceSelector.selectAllServices(), - nodeInfo.getNodeId(), - expectedNodeVersion, - httpsRequired); - - refreshNodesInternal(); - } - - private static InternalNode findCurrentNode(List allServices, String currentNodeId, NodeVersion expectedNodeVersion, boolean httpsRequired) - { - for (ServiceDescriptor service : allServices) { - URI uri = getHttpUri(service, httpsRequired); - NodeVersion nodeVersion = getNodeVersion(service); - if (uri != null && nodeVersion != null) { - InternalNode node = new InternalNode(service.getNodeId(), uri, nodeVersion, isCoordinator(service)); - - if (node.getNodeIdentifier().equals(currentNodeId)) { - checkState( - node.getNodeVersion().equals(expectedNodeVersion), - "INVARIANT: current node version (%s) should be equal to %s", - node.getNodeVersion(), - expectedNodeVersion); - return node; - } - } - } - throw new IllegalStateException("INVARIANT: current node not returned from service selector"); - } - - @PostConstruct - public void startPollingNodeStates() - { - nodeStateUpdateExecutor.scheduleWithFixedDelay(() -> { - try { - pollWorkers(); - } - catch (Exception e) { - log.error(e, "Error polling state of nodes"); - } - }, 5, 5, TimeUnit.SECONDS); - pollWorkers(); - } - - @PreDestroy - public void destroy() - { - nodeStateUpdateExecutor.shutdown(); - nodeStateEventExecutor.shutdown(); - } - - private void pollWorkers() - { - AllNodes allNodes = getAllNodes(); - Set aliveNodes = ImmutableSet.builder() - .addAll(allNodes.getActiveNodes()) - .addAll(allNodes.getDrainingNodes()) - .addAll(allNodes.getDrainedNodes()) - .addAll(allNodes.getShuttingDownNodes()) - .build(); - - Set aliveNodeIds = aliveNodes.stream() - .map(InternalNode::getNodeIdentifier) - .collect(toImmutableSet()); - - // Remove nodes that don't exist anymore - // Make a copy to materialize the set difference - Set deadNodes = difference(nodeStates.keySet(), aliveNodeIds).immutableCopy(); - nodeStates.keySet().removeAll(deadNodes); - - // Add new nodes - for (InternalNode node : aliveNodes) { - nodeStates.putIfAbsent(node.getNodeIdentifier(), - new RemoteNodeState(httpClient, uriBuilderFrom(node.getInternalUri()).appendPath("/v1/info/state").build())); - } - - // Schedule refresh - nodeStates.values().forEach(RemoteNodeState::asyncRefresh); - - // update indexes - refreshNodesInternal(); - } - - @PreDestroy - public void stop() - { - nodeStateUpdateExecutor.shutdownNow(); - } - - @Override - public void refreshNodes() - { - refreshNodesInternal(); - } - - private synchronized void refreshNodesInternal() - { - // This is a deny-list. - Set failed = failureDetector.getFailed(); - Set services = serviceSelector.selectAllServices().stream() - .filter(service -> !failed.contains(service)) - .collect(toImmutableSet()); - - ImmutableSet.Builder activeNodesBuilder = ImmutableSet.builder(); - ImmutableSet.Builder inactiveNodesBuilder = ImmutableSet.builder(); - ImmutableSet.Builder drainingNodesBuilder = ImmutableSet.builder(); - ImmutableSet.Builder drainedNodesBuilder = ImmutableSet.builder(); - ImmutableSet.Builder shuttingDownNodesBuilder = ImmutableSet.builder(); - ImmutableSet.Builder coordinatorsBuilder = ImmutableSet.builder(); - - for (ServiceDescriptor service : services) { - URI uri = getHttpUri(service, httpsRequired); - NodeVersion nodeVersion = getNodeVersion(service); - boolean coordinator = isCoordinator(service); - if (uri != null && nodeVersion != null) { - InternalNode node = new InternalNode(service.getNodeId(), uri, nodeVersion, coordinator); - NodeState nodeState = getNodeState(node); - - switch (nodeState) { - case ACTIVE: - activeNodesBuilder.add(node); - if (coordinator) { - coordinatorsBuilder.add(node); - } - break; - case INACTIVE: - inactiveNodesBuilder.add(node); - break; - case DRAINING: - drainingNodesBuilder.add(node); - break; - case DRAINED: - drainedNodesBuilder.add(node); - break; - case SHUTTING_DOWN: - shuttingDownNodesBuilder.add(node); - break; - default: - log.error("Unknown state %s for node %s", nodeState, node); - } - } - } - - Set activeNodes = activeNodesBuilder.build(); - Set drainingNodes = drainingNodesBuilder.build(); - Set drainedNodes = drainedNodesBuilder.build(); - Set inactiveNodes = inactiveNodesBuilder.build(); - Set coordinators = coordinatorsBuilder.build(); - Set shuttingDownNodes = shuttingDownNodesBuilder.build(); - if (allNodes != null) { - // log node that are no longer active (but not shutting down) - Set aliveNodes = ImmutableSet.builder() - .addAll(activeNodes) - .addAll(drainingNodes) - .addAll(drainedNodes) - .addAll(shuttingDownNodes) - .build(); - SetView missingNodes = difference(allNodes.getActiveNodes(), aliveNodes); - for (InternalNode missingNode : missingNodes) { - log.info("Previously active node is missing: %s (last seen at %s)", missingNode.getNodeIdentifier(), missingNode.getHost()); - } - } - - AllNodes allNodes = new AllNodes(activeNodes, inactiveNodes, drainingNodes, drainedNodes, shuttingDownNodes, coordinators); - // only update if all nodes actually changed (note: this does not include the connectors registered with the nodes) - if (!allNodes.equals(this.allNodes)) { - // assign allNodes to a local variable for use in the callback below - this.allNodes = allNodes; - this.coordinators = coordinators; - - // notify listeners - List> listeners = ImmutableList.copyOf(this.listeners); - nodeStateEventExecutor.submit(() -> listeners.forEach(listener -> listener.accept(allNodes))); - } - } - - private NodeState getNodeState(InternalNode node) - { - if (expectedNodeVersion.equals(node.getNodeVersion())) { - String nodeId = node.getNodeIdentifier(); - // The empty case that is being set to a default value of ACTIVE is limited to the case where a node - // has announced itself but no state has yet been successfully retrieved. RemoteNodeState will retain - // the previously known state if any has been reported. - return Optional.ofNullable(nodeStates.get(nodeId)) - .flatMap(RemoteNodeState::getNodeState) - .orElse(NodeState.ACTIVE); - } - return INACTIVE; - } - - @Override - public synchronized AllNodes getAllNodes() - { - return allNodes; - } - - @Managed - public int getActiveNodeCount() - { - return getAllNodes().getActiveNodes().size(); - } - - @Managed - public int getInactiveNodeCount() - { - return getAllNodes().getInactiveNodes().size(); - } - - @Managed - public int getDrainingNodeCount() - { - return getAllNodes().getDrainingNodes().size(); - } - - @Managed - public int getDrainedNodeCount() - { - return getAllNodes().getDrainedNodes().size(); - } - - @Managed - public int getShuttingDownNodeCount() - { - return getAllNodes().getShuttingDownNodes().size(); - } - - @Override - public Set getNodes(NodeState state) - { - return switch (state) { - case ACTIVE -> getAllNodes().getActiveNodes(); - case INACTIVE -> getAllNodes().getInactiveNodes(); - case DRAINING -> getAllNodes().getDrainingNodes(); - case DRAINED -> getAllNodes().getDrainedNodes(); - case SHUTTING_DOWN -> getAllNodes().getShuttingDownNodes(); - }; - } - - @Override - public synchronized NodesSnapshot getActiveNodesSnapshot() - { - return new NodesSnapshot(allNodes.getActiveNodes()); - } - - @Override - public InternalNode getCurrentNode() - { - return currentNode; - } - - @Override - public synchronized Set getCoordinators() - { - return coordinators; - } - - @Override - public synchronized void addNodeChangeListener(Consumer listener) - { - listeners.add(requireNonNull(listener, "listener is null")); - AllNodes allNodes = this.allNodes; - nodeStateEventExecutor.submit(() -> listener.accept(allNodes)); - } - - @Override - public synchronized void removeNodeChangeListener(Consumer listener) - { - listeners.remove(requireNonNull(listener, "listener is null")); - } - - private static URI getHttpUri(ServiceDescriptor descriptor, boolean httpsRequired) - { - String url = descriptor.getProperties().get(httpsRequired ? "https" : "http"); - if (url != null) { - return URI.create(url); - } - return null; - } - - private static NodeVersion getNodeVersion(ServiceDescriptor descriptor) - { - String nodeVersion = descriptor.getProperties().get("node_version"); - return nodeVersion == null ? null : new NodeVersion(nodeVersion); - } - - private static boolean isCoordinator(ServiceDescriptor service) - { - return Boolean.parseBoolean(service.getProperties().get("coordinator")); - } -} diff --git a/core/trino-main/src/main/java/io/trino/metadata/InMemoryNodeManager.java b/core/trino-main/src/main/java/io/trino/metadata/InMemoryNodeManager.java deleted file mode 100644 index 38ae98e6d7e5..000000000000 --- a/core/trino-main/src/main/java/io/trino/metadata/InMemoryNodeManager.java +++ /dev/null @@ -1,105 +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.metadata; - -import com.google.common.collect.ImmutableSet; -import com.google.errorprone.annotations.ThreadSafe; -import io.trino.client.NodeVersion; - -import java.net.URI; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.function.Consumer; - -import static java.util.Objects.requireNonNull; - -@ThreadSafe -public class InMemoryNodeManager - implements InternalNodeManager -{ - private static final InternalNode CURRENT_NODE = new InternalNode("local", URI.create("local://127.0.0.1:8080"), NodeVersion.UNKNOWN, true); - private final Set allNodes = ConcurrentHashMap.newKeySet(); - - public InMemoryNodeManager(InternalNode... remoteNodes) - { - this(ImmutableSet.copyOf(remoteNodes)); - } - - public InMemoryNodeManager(Set remoteNodes) - { - allNodes.add(CURRENT_NODE); - allNodes.addAll(remoteNodes); - } - - public void addNodes(InternalNode... internalNodes) - { - for (InternalNode internalNode : internalNodes) { - allNodes.add(requireNonNull(internalNode, "internalNode is null")); - } - } - - public void removeNode(InternalNode internalNode) - { - allNodes.remove(internalNode); - } - - @Override - public Set getNodes(NodeState state) - { - return switch (state) { - case ACTIVE -> ImmutableSet.copyOf(allNodes); - case DRAINING, DRAINED, INACTIVE, SHUTTING_DOWN -> ImmutableSet.of(); - }; - } - - @Override - public NodesSnapshot getActiveNodesSnapshot() - { - return new NodesSnapshot(ImmutableSet.copyOf(allNodes)); - } - - @Override - public AllNodes getAllNodes() - { - return new AllNodes( - ImmutableSet.copyOf(allNodes), - ImmutableSet.of(), - ImmutableSet.of(), - ImmutableSet.of(), - ImmutableSet.of(), - ImmutableSet.of(CURRENT_NODE)); - } - - @Override - public InternalNode getCurrentNode() - { - return CURRENT_NODE; - } - - @Override - public Set getCoordinators() - { - // always use localNode as coordinator - return ImmutableSet.of(CURRENT_NODE); - } - - @Override - public void refreshNodes() {} - - @Override - public void addNodeChangeListener(Consumer listener) {} - - @Override - public void removeNodeChangeListener(Consumer listener) {} -} diff --git a/core/trino-main/src/main/java/io/trino/metadata/RemoteNodeState.java b/core/trino-main/src/main/java/io/trino/metadata/RemoteNodeState.java deleted file mode 100644 index 0a1756ec8802..000000000000 --- a/core/trino-main/src/main/java/io/trino/metadata/RemoteNodeState.java +++ /dev/null @@ -1,114 +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.metadata; - -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.errorprone.annotations.ThreadSafe; -import io.airlift.http.client.FullJsonResponseHandler.JsonResponse; -import io.airlift.http.client.HttpClient; -import io.airlift.http.client.HttpClient.HttpResponseFuture; -import io.airlift.http.client.Request; -import io.airlift.json.JsonCodec; -import io.airlift.log.Logger; -import io.airlift.units.Duration; -import jakarta.annotation.Nullable; - -import java.net.URI; -import java.util.Optional; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; - -import static com.google.common.net.MediaType.JSON_UTF_8; -import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static io.airlift.http.client.FullJsonResponseHandler.createFullJsonResponseHandler; -import static io.airlift.http.client.HttpStatus.OK; -import static io.airlift.http.client.Request.Builder.prepareGet; -import static io.airlift.json.JsonCodec.jsonCodec; -import static io.airlift.units.Duration.nanosSince; -import static jakarta.ws.rs.core.HttpHeaders.CONTENT_TYPE; -import static java.util.Objects.requireNonNull; -import static java.util.concurrent.TimeUnit.SECONDS; - -@ThreadSafe -public class RemoteNodeState -{ - private static final Logger log = Logger.get(RemoteNodeState.class); - private static final JsonCodec NODE_STATE_CODEC = jsonCodec(NodeState.class); - - private final HttpClient httpClient; - private final URI stateInfoUri; - private final AtomicReference> nodeState = new AtomicReference<>(Optional.empty()); - private final AtomicReference> future = new AtomicReference<>(); - private final AtomicLong lastUpdateNanos = new AtomicLong(); - private final AtomicLong lastWarningLogged = new AtomicLong(); - - public RemoteNodeState(HttpClient httpClient, URI stateInfoUri) - { - this.httpClient = requireNonNull(httpClient, "httpClient is null"); - this.stateInfoUri = requireNonNull(stateInfoUri, "stateInfoUri is null"); - } - - public Optional getNodeState() - { - return nodeState.get(); - } - - public synchronized void asyncRefresh() - { - Duration sinceUpdate = nanosSince(lastUpdateNanos.get()); - if (nanosSince(lastWarningLogged.get()).toMillis() > 1_000 && - sinceUpdate.toMillis() > 10_000 && - future.get() != null) { - log.warn("Node state update request to %s has not returned in %s", stateInfoUri, sinceUpdate.toString(SECONDS)); - lastWarningLogged.set(System.nanoTime()); - } - if (sinceUpdate.toMillis() > 1_000 && future.get() == null) { - Request request = prepareGet() - .setUri(stateInfoUri) - .setHeader(CONTENT_TYPE, JSON_UTF_8.toString()) - .build(); - HttpResponseFuture> responseFuture = httpClient.executeAsync(request, createFullJsonResponseHandler(NODE_STATE_CODEC)); - future.compareAndSet(null, responseFuture); - - Futures.addCallback(responseFuture, new FutureCallback<>() - { - @Override - public void onSuccess(@Nullable JsonResponse result) - { - lastUpdateNanos.set(System.nanoTime()); - future.compareAndSet(responseFuture, null); - if (result != null) { - if (result.hasValue()) { - nodeState.set(Optional.ofNullable(result.getValue())); - } - if (result.getStatusCode() != OK.code()) { - log.warn("Error fetching node state from %s returned status %d", stateInfoUri, result.getStatusCode()); - return; - } - } - } - - @Override - public void onFailure(Throwable t) - { - log.warn("Error fetching node state from %s: %s", stateInfoUri, t.getMessage()); - lastUpdateNanos.set(System.nanoTime()); - future.compareAndSet(responseFuture, null); - } - }, directExecutor()); - } - } -} diff --git a/core/trino-main/src/main/java/io/trino/node/AirliftAnnouncer.java b/core/trino-main/src/main/java/io/trino/node/AirliftAnnouncer.java new file mode 100644 index 000000000000..076f98dd8a93 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/node/AirliftAnnouncer.java @@ -0,0 +1,47 @@ +/* + * 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.node; + +import com.google.common.util.concurrent.ListenableFuture; +import com.google.inject.Inject; + +public class AirliftAnnouncer + implements Announcer +{ + private final io.airlift.discovery.client.Announcer airliftAnnouncer; + + @Inject + public AirliftAnnouncer(io.airlift.discovery.client.Announcer airliftAnnouncer) + { + this.airliftAnnouncer = airliftAnnouncer; + } + + @Override + public void start() + { + airliftAnnouncer.start(); + } + + @Override + public ListenableFuture forceAnnounce() + { + return airliftAnnouncer.forceAnnounce(); + } + + @Override + public void stop() + { + airliftAnnouncer.destroy(); + } +} diff --git a/core/trino-main/src/main/java/io/trino/node/AirliftNodeInventory.java b/core/trino-main/src/main/java/io/trino/node/AirliftNodeInventory.java new file mode 100644 index 000000000000..c9d31eed16a2 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/node/AirliftNodeInventory.java @@ -0,0 +1,69 @@ +/* + * 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.node; + +import com.google.inject.Inject; +import io.airlift.discovery.client.ServiceDescriptor; +import io.airlift.discovery.client.ServiceSelector; +import io.airlift.discovery.client.ServiceType; +import io.trino.failuredetector.FailureDetector; +import io.trino.server.InternalCommunicationConfig; + +import java.net.URI; +import java.util.Objects; +import java.util.Set; + +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static java.util.Objects.requireNonNull; +import static java.util.function.Predicate.not; + +public class AirliftNodeInventory + implements NodeInventory +{ + private final ServiceSelector serviceSelector; + private final FailureDetector failureDetector; + private final boolean httpsRequired; + + @Inject + public AirliftNodeInventory( + @ServiceType("trino") ServiceSelector serviceSelector, + FailureDetector failureDetector, + InternalCommunicationConfig internalCommunicationConfig) + { + this.serviceSelector = requireNonNull(serviceSelector, "serviceSelector is null"); + this.failureDetector = requireNonNull(failureDetector, "failureDetector is null"); + this.httpsRequired = internalCommunicationConfig.isHttpsRequired(); + } + + @Override + public Set getNodes() + { + // This is a deny-list. + Set failed = failureDetector.getFailed(); + return serviceSelector.selectAllServices().stream() + .filter(not(failed::contains)) + .map(this::getHttpUri) + .filter(Objects::nonNull) + .collect(toImmutableSet()); + } + + private URI getHttpUri(ServiceDescriptor descriptor) + { + String url = descriptor.getProperties().get(httpsRequired ? "https" : "http"); + if (url != null) { + return URI.create(url); + } + return null; + } +} diff --git a/core/trino-main/src/main/java/io/trino/node/AirliftNodeInventoryModule.java b/core/trino-main/src/main/java/io/trino/node/AirliftNodeInventoryModule.java new file mode 100644 index 000000000000..3e87485dccaa --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/node/AirliftNodeInventoryModule.java @@ -0,0 +1,134 @@ +/* + * 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.node; + +import com.google.inject.Binder; +import com.google.inject.Scopes; +import com.google.inject.multibindings.Multibinder; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.airlift.discovery.client.DiscoveryModule; +import io.airlift.discovery.client.ForDiscoveryClient; +import io.airlift.discovery.server.DynamicAnnouncementResource; +import io.airlift.discovery.server.EmbeddedDiscoveryModule; +import io.airlift.discovery.server.ServiceResource; +import io.airlift.discovery.store.StoreResource; +import io.airlift.http.client.HttpClientConfig; +import io.airlift.http.client.HttpRequestFilter; +import io.airlift.http.client.Request; +import io.trino.failuredetector.FailureDetectorModule; +import io.trino.server.InternalAuthenticationManager; +import io.trino.server.InternalCommunicationConfig; +import io.trino.server.NodeResource; +import io.trino.server.ServerConfig; + +import java.io.UncheckedIOException; +import java.net.InetAddress; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.UnknownHostException; + +import static com.google.inject.multibindings.Multibinder.newSetBinder; +import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.airlift.discovery.client.DiscoveryBinder.discoveryBinder; +import static io.airlift.jaxrs.JaxrsBinder.jaxrsBinder; +import static io.airlift.node.AddressToHostname.encodeAddressAsHostname; +import static io.trino.server.InternalCommunicationHttpClientModule.configureClient; +import static io.trino.server.security.ResourceSecurityBinder.resourceSecurityBinder; + +public class AirliftNodeInventoryModule + extends AbstractConfigurationAwareModule +{ + private final String nodeVersion; + + public AirliftNodeInventoryModule(String nodeVersion) + { + this.nodeVersion = nodeVersion; + } + + @Override + protected void setup(Binder binder) + { + boolean coordinator = buildConfigObject(ServerConfig.class).isCoordinator(); + if (coordinator) { + if (buildConfigObject(EmbeddedDiscoveryConfig.class).isEnabled()) { + install(new EmbeddedDiscoveryModule()); + } + + binder.bind(NodeInventory.class).to(AirliftNodeInventory.class).in(Scopes.SINGLETON); + + // selector + discoveryBinder(binder).bindSelector("trino"); + + // coordinator announcement + discoveryBinder(binder).bindHttpAnnouncement("trino-coordinator"); + + // failure detector + install(new FailureDetectorModule()); + jaxrsBinder(binder).bind(NodeResource.class); + + // server security configuration + resourceSecurityBinder(binder) + .managementReadResource(ServiceResource.class) + .internalOnlyResource(DynamicAnnouncementResource.class) + .internalOnlyResource(StoreResource.class); + } + + // both coordinator and worker must announce + install(new DiscoveryModule()); + binder.bind(Announcer.class).to(AirliftAnnouncer.class).in(Scopes.SINGLETON); + discoveryBinder(binder).bindHttpAnnouncement("trino") + .addProperty("node_version", nodeVersion) + .addProperty("coordinator", String.valueOf(coordinator)); + + // internal communication setup for discovery http client + InternalCommunicationConfig internalCommunicationConfig = buildConfigObject(InternalCommunicationConfig.class); + Multibinder discoveryFilterBinder = newSetBinder(binder, HttpRequestFilter.class, ForDiscoveryClient.class); + if (internalCommunicationConfig.isHttpsRequired() && internalCommunicationConfig.getKeyStorePath() == null && internalCommunicationConfig.getTrustStorePath() == null) { + discoveryFilterBinder.addBinding().to(DiscoveryEncodeAddressAsHostname.class); + } + configBinder(binder).bindConfigDefaults(HttpClientConfig.class, ForDiscoveryClient.class, config -> configureClient(config, internalCommunicationConfig)); + discoveryFilterBinder.addBinding().to(InternalAuthenticationManager.class); + } + + private static class DiscoveryEncodeAddressAsHostname + implements HttpRequestFilter + { + @Override + public Request filterRequest(Request request) + { + return Request.Builder.fromRequest(request) + .setUri(toIpEncodedAsHostnameUri(request.getUri())) + .build(); + } + + private static URI toIpEncodedAsHostnameUri(URI uri) + { + if (!uri.getScheme().equals("https")) { + return uri; + } + try { + String host = uri.getHost(); + InetAddress inetAddress = InetAddress.getByName(host); + String addressAsHostname = encodeAddressAsHostname(inetAddress); + return new URI(uri.getScheme(), uri.getUserInfo(), addressAsHostname, uri.getPort(), uri.getPath(), uri.getQuery(), uri.getFragment()); + } + catch (UnknownHostException e) { + throw new UncheckedIOException(e); + } + catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + } +} diff --git a/core/trino-main/src/main/java/io/trino/node/AllNodes.java b/core/trino-main/src/main/java/io/trino/node/AllNodes.java new file mode 100644 index 000000000000..392a5ee14cf2 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/node/AllNodes.java @@ -0,0 +1,39 @@ +/* + * 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.node; + +import com.google.common.collect.ImmutableSet; + +import java.util.Set; + +import static java.util.Objects.requireNonNull; + +public record AllNodes( + Set activeNodes, + Set inactiveNodes, + Set drainingNodes, + Set drainedNodes, + Set shuttingDownNodes, + Set activeCoordinators) +{ + public AllNodes + { + activeNodes = ImmutableSet.copyOf(requireNonNull(activeNodes, "activeNodes is null")); + inactiveNodes = ImmutableSet.copyOf(requireNonNull(inactiveNodes, "inactiveNodes is null")); + drainedNodes = ImmutableSet.copyOf(requireNonNull(drainedNodes, "drainedNodes is null")); + drainingNodes = ImmutableSet.copyOf(requireNonNull(drainingNodes, "drainingNodes is null")); + shuttingDownNodes = ImmutableSet.copyOf(requireNonNull(shuttingDownNodes, "shuttingDownNodes is null")); + activeCoordinators = ImmutableSet.copyOf(requireNonNull(activeCoordinators, "activeCoordinators is null")); + } +} diff --git a/core/trino-main/src/main/java/io/trino/node/Announcer.java b/core/trino-main/src/main/java/io/trino/node/Announcer.java new file mode 100644 index 000000000000..4bad12b4314c --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/node/Announcer.java @@ -0,0 +1,39 @@ +/* + * 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.node; + +import com.google.common.util.concurrent.ListenableFuture; + +public interface Announcer +{ + /** + * Announce the existence of this node to the cluster. + * This method should be called after the node is fully initialized. + */ + void start(); + + /** + * Force an announcement of this node to the cluster. + * This method can be used to refresh the node's presence in the cluster. + * + * @return a Future that completes when the announcement is done + */ + ListenableFuture forceAnnounce(); + + /** + * Stop announcing the existence of this node to the cluster. + * This method should be called before the node is shut down. + */ + void stop(); +} diff --git a/core/trino-main/src/main/java/io/trino/node/CoordinatorNodeManager.java b/core/trino-main/src/main/java/io/trino/node/CoordinatorNodeManager.java new file mode 100644 index 000000000000..a7631a73122c --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/node/CoordinatorNodeManager.java @@ -0,0 +1,302 @@ +/* + * 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.node; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Ticker; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.errorprone.annotations.ThreadSafe; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import com.google.inject.Inject; +import io.airlift.http.client.HttpClient; +import io.airlift.log.Logger; +import io.airlift.node.NodeInfo; +import io.trino.server.NodeStateManager.CurrentNodeState; +import io.trino.spi.HostAddress; +import jakarta.annotation.PostConstruct; +import jakarta.annotation.PreDestroy; +import org.weakref.jmx.Managed; + +import java.net.URI; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.function.Supplier; + +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.airlift.concurrent.Threads.daemonThreadsNamed; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.Executors.newCachedThreadPool; +import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor; +import static java.util.function.Function.identity; + +@ThreadSafe +public final class CoordinatorNodeManager + implements InternalNodeManager +{ + private static final Logger log = Logger.get(CoordinatorNodeManager.class); + + private final Supplier currentNodeState; + private final NodeInventory nodeInventory; + private final String expectedNodeEnvironment; + private final ConcurrentHashMap nodeStates = new ConcurrentHashMap<>(); + private final HttpClient httpClient; + private final ScheduledExecutorService nodeStateUpdateExecutor; + private final ExecutorService nodeStateEventExecutor; + private final InternalNode currentNode; + private final Ticker ticker; + + @GuardedBy("this") + private AllNodes allNodes; + + @GuardedBy("this") + private Set invalidNodes; + + @GuardedBy("this") + private Map goneNodes; + + @GuardedBy("this") + private final List> listeners = new ArrayList<>(); + + @Inject + public CoordinatorNodeManager( + NodeInventory nodeInventory, + NodeInfo nodeInfo, + InternalNode currentNode, + CurrentNodeState currentNodeState, + @ForNodeManager HttpClient httpClient) + { + this( + nodeInventory, + currentNode, + currentNodeState, + nodeInfo.getEnvironment(), + httpClient, + Ticker.systemTicker()); + } + + @VisibleForTesting + CoordinatorNodeManager( + NodeInventory nodeInventory, + InternalNode currentNode, + Supplier currentNodeState, + String expectedNodeEnvironment, + HttpClient httpClient, + Ticker ticker) + { + this.nodeInventory = requireNonNull(nodeInventory, "nodeInventory is null"); + this.currentNode = requireNonNull(currentNode, "currentNode is null"); + this.currentNodeState = requireNonNull(currentNodeState, "currentNodeState is null"); + this.expectedNodeEnvironment = requireNonNull(expectedNodeEnvironment, "expectedNodeEnvironment is null"); + this.httpClient = requireNonNull(httpClient, "httpClient is null"); + this.nodeStateUpdateExecutor = newSingleThreadScheduledExecutor(daemonThreadsNamed("node-state-poller-%s")); + this.nodeStateEventExecutor = newCachedThreadPool(daemonThreadsNamed("node-state-events-%s")); + this.ticker = requireNonNull(ticker, "ticker is null"); + + refreshNodes(); + } + + @PostConstruct + public void startPollingNodeStates() + { + nodeStateUpdateExecutor.scheduleWithFixedDelay(() -> { + try { + refreshNodes(); + } + catch (Exception e) { + log.error(e, "Error polling state of nodes"); + } + }, 5, 5, TimeUnit.SECONDS); + refreshNodes(); + } + + @PreDestroy + public void stop() + { + nodeStateUpdateExecutor.shutdown(); + nodeStateEventExecutor.shutdown(); + } + + @Override + public void refreshNodes() + { + // Add new nodes + for (URI uri : nodeInventory.getNodes()) { + if (uri.equals(currentNode.getInternalUri())) { + // Skip the current node + continue; + } + + // Mark the node as seen, and get the current state + RemoteNodeState remoteNodeState = nodeStates.computeIfAbsent( + uri, + _ -> new RemoteNodeState( + uri, + expectedNodeEnvironment, + currentNode.getNodeVersion(), + httpClient, + ticker)); + remoteNodeState.setSeen(); + } + + // Remove nodes that are no longer present + for (var entry : nodeStates.entrySet()) { + RemoteNodeState remoteNodeState = entry.getValue(); + if (remoteNodeState.isMissing()) { + if (remoteNodeState.hasBeenActive() && remoteNodeState.getState() != NodeState.SHUTTING_DOWN) { + log.info("Previously active node is missing: %s", entry.getKey()); + } + nodeStates.remove(entry.getKey()); + } + } + + // Schedule refresh + nodeStates.values().forEach(RemoteNodeState::asyncRefresh); + + // update indexes + refreshNodesInternal(); + } + + private synchronized void refreshNodesInternal() + { + ImmutableSet.Builder activeNodesBuilder = ImmutableSet.builder(); + ImmutableSet.Builder inactiveNodesBuilder = ImmutableSet.builder(); + ImmutableSet.Builder drainingNodesBuilder = ImmutableSet.builder(); + ImmutableSet.Builder drainedNodesBuilder = ImmutableSet.builder(); + ImmutableSet.Builder shuttingDownNodesBuilder = ImmutableSet.builder(); + ImmutableSet.Builder invalidNodesBuilder = ImmutableSet.builder(); + ImmutableSet.Builder goneNodesBuilder = ImmutableSet.builder(); + + switch (currentNodeState.get()) { + case ACTIVE -> activeNodesBuilder.add(currentNode); + // INVALID or GONE should never happen, but if it does, treat as INACTIVE to avoid exceptions + case INACTIVE, INVALID, GONE -> inactiveNodesBuilder.add(currentNode); + case DRAINING -> drainingNodesBuilder.add(currentNode); + case DRAINED -> drainedNodesBuilder.add(currentNode); + case SHUTTING_DOWN -> shuttingDownNodesBuilder.add(currentNode); + } + + for (RemoteNodeState remoteNodeState : nodeStates.values()) { + InternalNode node = remoteNodeState.getInternalNode().orElse(null); + if (node == null) { + continue; + } + switch (remoteNodeState.getState()) { + case ACTIVE -> activeNodesBuilder.add(node); + case INACTIVE -> inactiveNodesBuilder.add(node); + case DRAINING -> drainingNodesBuilder.add(node); + case DRAINED -> drainedNodesBuilder.add(node); + case SHUTTING_DOWN -> shuttingDownNodesBuilder.add(node); + case INVALID -> invalidNodesBuilder.add(node); + case GONE -> goneNodesBuilder.add(node); + } + } + + this.invalidNodes = invalidNodesBuilder.build(); + this.goneNodes = goneNodesBuilder.build().stream() + .collect(toImmutableMap(InternalNode::getHostAndPort, identity())); + + Set activeNodes = activeNodesBuilder.build(); + Set drainingNodes = drainingNodesBuilder.build(); + Set drainedNodes = drainedNodesBuilder.build(); + Set inactiveNodes = inactiveNodesBuilder.build(); + Set shuttingDownNodes = shuttingDownNodesBuilder.build(); + + Set coordinators = activeNodes.stream() + .filter(InternalNode::isCoordinator) + .collect(toImmutableSet()); + + AllNodes allNodes = new AllNodes(activeNodes, inactiveNodes, drainingNodes, drainedNodes, shuttingDownNodes, coordinators); + // only update if all nodes actually changed (note: this does not include the connectors registered with the nodes) + if (!allNodes.equals(this.allNodes)) { + // assign allNodes to a local variable for use in the callback below + this.allNodes = allNodes; + + // notify listeners + List> listeners = ImmutableList.copyOf(this.listeners); + nodeStateEventExecutor.submit(() -> listeners.forEach(listener -> listener.accept(allNodes))); + } + } + + @Override + public synchronized AllNodes getAllNodes() + { + return allNodes; + } + + @Managed + public int getActiveNodeCount() + { + return getAllNodes().activeNodes().size(); + } + + @Managed + public int getInactiveNodeCount() + { + return getAllNodes().inactiveNodes().size(); + } + + @Managed + public int getDrainingNodeCount() + { + return getAllNodes().drainingNodes().size(); + } + + @Managed + public int getDrainedNodeCount() + { + return getAllNodes().drainedNodes().size(); + } + + @Managed + public int getShuttingDownNodeCount() + { + return getAllNodes().shuttingDownNodes().size(); + } + + @VisibleForTesting + synchronized Set getInvalidNodes() + { + return invalidNodes; + } + + @Override + public synchronized boolean isGone(HostAddress hostAddress) + { + requireNonNull(hostAddress, "hostAddress is null"); + return goneNodes.containsKey(hostAddress); + } + + @Override + public synchronized void addNodeChangeListener(Consumer listener) + { + listeners.add(requireNonNull(listener, "listener is null")); + AllNodes allNodes = this.allNodes; + nodeStateEventExecutor.submit(() -> listener.accept(allNodes)); + } + + @Override + public synchronized void removeNodeChangeListener(Consumer listener) + { + listeners.remove(requireNonNull(listener, "listener is null")); + } +} diff --git a/core/trino-main/src/main/java/io/trino/server/EmbeddedDiscoveryConfig.java b/core/trino-main/src/main/java/io/trino/node/EmbeddedDiscoveryConfig.java similarity index 97% rename from core/trino-main/src/main/java/io/trino/server/EmbeddedDiscoveryConfig.java rename to core/trino-main/src/main/java/io/trino/node/EmbeddedDiscoveryConfig.java index d18051b20ac4..30349510271d 100644 --- a/core/trino-main/src/main/java/io/trino/server/EmbeddedDiscoveryConfig.java +++ b/core/trino-main/src/main/java/io/trino/node/EmbeddedDiscoveryConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.server; +package io.trino.node; import io.airlift.configuration.Config; diff --git a/core/trino-main/src/main/java/io/trino/metadata/ForNodeManager.java b/core/trino-main/src/main/java/io/trino/node/ForNodeManager.java similarity index 97% rename from core/trino-main/src/main/java/io/trino/metadata/ForNodeManager.java rename to core/trino-main/src/main/java/io/trino/node/ForNodeManager.java index 0a7bbfb19519..017a44dc1fae 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/ForNodeManager.java +++ b/core/trino-main/src/main/java/io/trino/node/ForNodeManager.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.metadata; +package io.trino.node; import com.google.inject.BindingAnnotation; diff --git a/core/trino-main/src/main/java/io/trino/metadata/InternalNode.java b/core/trino-main/src/main/java/io/trino/node/InternalNode.java similarity index 99% rename from core/trino-main/src/main/java/io/trino/metadata/InternalNode.java rename to core/trino-main/src/main/java/io/trino/node/InternalNode.java index 054ebf6e0c15..954d27edd6b7 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/InternalNode.java +++ b/core/trino-main/src/main/java/io/trino/node/InternalNode.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.metadata; +package io.trino.node; import io.airlift.slice.XxHash64; import io.trino.client.NodeVersion; diff --git a/core/trino-main/src/main/java/io/trino/metadata/InternalNodeManager.java b/core/trino-main/src/main/java/io/trino/node/InternalNodeManager.java similarity index 66% rename from core/trino-main/src/main/java/io/trino/metadata/InternalNodeManager.java rename to core/trino-main/src/main/java/io/trino/node/InternalNodeManager.java index ed5e94f8df8b..2495e40489c5 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/InternalNodeManager.java +++ b/core/trino-main/src/main/java/io/trino/node/InternalNodeManager.java @@ -11,9 +11,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.metadata; +package io.trino.node; import com.google.common.collect.ImmutableSet; +import io.trino.spi.HostAddress; import java.util.Set; import java.util.function.Consumer; @@ -23,16 +24,32 @@ public interface InternalNodeManager { - Set getNodes(NodeState state); - - NodesSnapshot getActiveNodesSnapshot(); + default Set getNodes(NodeState state) + { + return switch (state) { + case ACTIVE -> getAllNodes().activeNodes(); + case INACTIVE -> getAllNodes().inactiveNodes(); + case DRAINING -> getAllNodes().drainingNodes(); + case DRAINED -> getAllNodes().drainedNodes(); + case SHUTTING_DOWN -> getAllNodes().shuttingDownNodes(); + case INVALID, GONE -> ImmutableSet.of(); + }; + } - InternalNode getCurrentNode(); + default NodesSnapshot getActiveNodesSnapshot() + { + return new NodesSnapshot(getAllNodes().activeNodes()); + } - Set getCoordinators(); + default Set getCoordinators() + { + return getAllNodes().activeCoordinators(); + } AllNodes getAllNodes(); + boolean isGone(HostAddress hostAddress); + void refreshNodes(); void addNodeChangeListener(Consumer listener); diff --git a/core/trino-main/src/main/java/io/trino/node/NodeInventory.java b/core/trino-main/src/main/java/io/trino/node/NodeInventory.java new file mode 100644 index 000000000000..1084599dcb5e --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/node/NodeInventory.java @@ -0,0 +1,22 @@ +/* + * 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.node; + +import java.net.URI; +import java.util.Set; + +public interface NodeInventory +{ + Set getNodes(); +} diff --git a/core/trino-main/src/main/java/io/trino/node/NodeManagerModule.java b/core/trino-main/src/main/java/io/trino/node/NodeManagerModule.java new file mode 100644 index 000000000000..ba7399ab7aa1 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/node/NodeManagerModule.java @@ -0,0 +1,56 @@ +/* + * 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.node; + +import com.google.inject.Binder; +import com.google.inject.Scopes; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.airlift.units.Duration; +import io.trino.server.ServerConfig; + +import static io.trino.server.InternalCommunicationHttpClientModule.internalHttpClientModule; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.weakref.jmx.guice.ExportBinder.newExporter; + +public class NodeManagerModule + extends AbstractConfigurationAwareModule +{ + private final String nodeVersion; + + public NodeManagerModule(String nodeVersion) + { + this.nodeVersion = nodeVersion; + } + + @Override + protected void setup(Binder binder) + { + ServerConfig serverConfig = buildConfigObject(ServerConfig.class); + if (serverConfig.isCoordinator()) { + binder.bind(CoordinatorNodeManager.class).in(Scopes.SINGLETON); + binder.bind(InternalNodeManager.class).to(CoordinatorNodeManager.class).in(Scopes.SINGLETON); + newExporter(binder).export(CoordinatorNodeManager.class).withGeneratedName(); + install(internalHttpClientModule("node-manager", ForNodeManager.class) + .withConfigDefaults(config -> { + config.setIdleTimeout(new Duration(30, SECONDS)); + config.setRequestTimeout(new Duration(10, SECONDS)); + }).build()); + } + else { + binder.bind(InternalNodeManager.class).to(WorkerInternalNodeManager.class).in(Scopes.SINGLETON); + } + + install(new AirliftNodeInventoryModule(nodeVersion)); + } +} diff --git a/core/trino-main/src/main/java/io/trino/metadata/NodeState.java b/core/trino-main/src/main/java/io/trino/node/NodeState.java similarity index 51% rename from core/trino-main/src/main/java/io/trino/metadata/NodeState.java rename to core/trino-main/src/main/java/io/trino/node/NodeState.java index e3370da594d6..53d1a13d024f 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/NodeState.java +++ b/core/trino-main/src/main/java/io/trino/node/NodeState.java @@ -11,20 +11,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.metadata; +package io.trino.node; public enum NodeState { /** - * Server is up and running ready to handle tasks + * The node is up and ready to handle tasks. */ ACTIVE, /** - * Never used internally, might be used by discoveryNodeManager when communication error occurs + * The node is currently not handling tasks, but it is still part of the cluster. + * This is an internal state used by node manager when communication errors occur. */ INACTIVE, /** - * A reversible graceful shutdown, can go to forward to DRAINED or back to ACTIVE. + * A reversible graceful shutdown can go to forward to DRAINED or back to ACTIVE. */ DRAINING, /** @@ -34,5 +35,16 @@ public enum NodeState /** * Graceful shutdown, non-reversible, when observed will drain and terminate */ - SHUTTING_DOWN + SHUTTING_DOWN, + /** + * The node is not valid for this cluster. Nodes in this state are not visible to the node manager. + * This is an internal state used by node manager when the environment or version of the node is + * not valid for the cluster. + */ + INVALID, + /** + * Connections to the node have been refused. Nodes in this state are not visible to the node manager. + * This is an internal state used to by execution engine to produce better error messages. + */ + GONE, } diff --git a/core/trino-main/src/main/java/io/trino/node/RemoteNodeState.java b/core/trino-main/src/main/java/io/trino/node/RemoteNodeState.java new file mode 100644 index 000000000000..77b991ee0eec --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/node/RemoteNodeState.java @@ -0,0 +1,211 @@ +/* + * 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.node; + +import com.google.common.base.Ticker; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.errorprone.annotations.FormatMethod; +import com.google.errorprone.annotations.ThreadSafe; +import io.airlift.http.client.FullJsonResponseHandler.JsonResponse; +import io.airlift.http.client.HttpClient; +import io.airlift.http.client.HttpClient.HttpResponseFuture; +import io.airlift.http.client.Request; +import io.airlift.json.JsonCodec; +import io.airlift.log.Logger; +import io.trino.client.NodeVersion; +import io.trino.server.ServerInfo; +import jakarta.annotation.Nullable; + +import java.net.ConnectException; +import java.net.URI; +import java.util.Optional; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import static com.google.common.net.MediaType.JSON_UTF_8; +import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static io.airlift.http.client.FullJsonResponseHandler.createFullJsonResponseHandler; +import static io.airlift.http.client.HttpStatus.OK; +import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; +import static io.airlift.http.client.Request.Builder.prepareGet; +import static io.airlift.json.JsonCodec.jsonCodec; +import static io.trino.node.NodeState.ACTIVE; +import static io.trino.node.NodeState.GONE; +import static io.trino.node.NodeState.INACTIVE; +import static io.trino.node.NodeState.INVALID; +import static jakarta.ws.rs.core.HttpHeaders.CONTENT_TYPE; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; + +@ThreadSafe +public class RemoteNodeState +{ + private static final Logger log = Logger.get(RemoteNodeState.class); + private static final JsonCodec SERVER_INFO_CODEC = jsonCodec(ServerInfo.class); + + private final URI serverUri; + private final String expectedNodeEnvironment; + private final NodeVersion expectedNodeVersion; + private final HttpClient httpClient; + private final URI infoUri; + private final Ticker ticker; + private final AtomicReference internalNode = new AtomicReference<>(); + private final AtomicReference nodeState = new AtomicReference<>(INACTIVE); + private final AtomicBoolean hasBeenActive = new AtomicBoolean(false); + private final AtomicReference> future = new AtomicReference<>(); + private final AtomicLong lastSeenNanos; + private final AtomicLong lastUpdateNanos; + private final AtomicLong lastWarningLogged; + + public RemoteNodeState(URI serverUri, String expectedNodeEnvironment, NodeVersion expectedNodeVersion, HttpClient httpClient, Ticker ticker) + { + this.serverUri = requireNonNull(serverUri, "serverUri is null"); + this.expectedNodeEnvironment = requireNonNull(expectedNodeEnvironment, "expectedNodeEnvironment is null"); + this.expectedNodeVersion = requireNonNull(expectedNodeVersion, "expectedNodeVersion is null"); + this.httpClient = requireNonNull(httpClient, "httpClient is null"); + this.infoUri = uriBuilderFrom(serverUri).appendPath("/v1/info").build(); + this.ticker = requireNonNull(ticker, "ticker is null"); + this.lastSeenNanos = new AtomicLong(ticker.read()); + // Initialize to 30 seconds in the past to force immediate refresh + this.lastUpdateNanos = new AtomicLong(ticker.read() - SECONDS.toNanos(30)); + this.lastWarningLogged = new AtomicLong(ticker.read() - SECONDS.toNanos(30)); + } + + public NodeState getState() + { + return nodeState.get(); + } + + public Optional getInternalNode() + { + return Optional.ofNullable(internalNode.get()); + } + + public boolean hasBeenActive() + { + return hasBeenActive.get(); + } + + public void setSeen() + { + lastSeenNanos.set(ticker.read()); + } + + public boolean isMissing() + { + // If the node has not been seen for 30 seconds, it is considered missing. + return lastSeenNanos.get() + SECONDS.toNanos(30) <= ticker.read(); + } + + public synchronized void asyncRefresh() + { + long millisSinceUpdate = millisSince(lastUpdateNanos.get()); + if (millisSinceUpdate > 10_000 && future.get() != null) { + logWarning("Node state update request to %s has not returned in %sms", infoUri, millisSinceUpdate); + } + if (millisSinceUpdate >= 1_000 && future.get() == null) { + Request request = prepareGet() + .setUri(infoUri) + .setHeader(CONTENT_TYPE, JSON_UTF_8.toString()) + .build(); + HttpResponseFuture> responseFuture = httpClient.executeAsync(request, createFullJsonResponseHandler(SERVER_INFO_CODEC)); + future.compareAndSet(null, responseFuture); + + Futures.addCallback(responseFuture, new FutureCallback<>() + { + @Override + public void onSuccess(@Nullable JsonResponse result) + { + try { + // If the result is null, mark the node as INACTIVE to prevent work from being scheduled on it + NodeState nodeState; + if (result == null || !result.hasValue()) { + nodeState = INACTIVE; + } + else { + ServerInfo serverInfo = result.getValue(); + + // Set state to INVALID if the node is not in the expected environment or version + // This prevents the node from being visible outside the node manager + if (!serverInfo.environment().equals(expectedNodeEnvironment)) { + logWarning("Node environment mismatch: expected %s, got %s", expectedNodeEnvironment, serverInfo.environment()); + nodeState = INVALID; + } + else if (!serverInfo.nodeVersion().equals(expectedNodeVersion)) { + logWarning("Node version mismatch: expected %s, got %s", expectedNodeVersion, serverInfo.nodeVersion()); + nodeState = INVALID; + } + else { + nodeState = serverInfo.state(); + } + + internalNode.set(new InternalNode( + serverInfo.nodeId(), + serverUri, + serverInfo.nodeVersion(), + serverInfo.coordinator())); + } + + RemoteNodeState.this.nodeState.set(nodeState); + if (nodeState == ACTIVE) { + hasBeenActive.set(true); + } + if (result == null || result.getStatusCode() != OK.code()) { + logWarning("Error fetching node state from %s", infoUri); + } + } + catch (Throwable e) { + // Any failure results in the node being marked as INACTIVE to prevent work from being scheduled on it + nodeState.set(INACTIVE); + logWarning("Error processing node state from %s: %s", infoUri, e.getMessage()); + throw e; + } + finally { + lastUpdateNanos.set(ticker.read()); + future.compareAndSet(responseFuture, null); + } + } + + @Override + public void onFailure(Throwable t) + { + // Any failure results in the node being marked a GONE or INACTIVE to prevent work from being scheduled on it + nodeState.set(t instanceof ConnectException ? GONE : INACTIVE); + logWarning("Error fetching node state from %s: %s", infoUri, t.getMessage()); + lastUpdateNanos.set(ticker.read()); + future.compareAndSet(responseFuture, null); + } + }, directExecutor()); + } + } + + private long millisSince(long startNanos) + { + return (ticker.read() - startNanos) / MILLISECONDS.toNanos(1); + } + + @FormatMethod + private void logWarning(String format, Object... args) + { + // log at most once per second per node + if (ticker.read() - lastWarningLogged.get() >= SECONDS.toNanos(1)) { + log.warn(format, args); + lastWarningLogged.set(ticker.read()); + } + } +} diff --git a/core/trino-main/src/main/java/io/trino/node/TestingAnnouncer.java b/core/trino-main/src/main/java/io/trino/node/TestingAnnouncer.java new file mode 100644 index 000000000000..a7616b9bbfb5 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/node/TestingAnnouncer.java @@ -0,0 +1,33 @@ +/* + * 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.node; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; + +public class TestingAnnouncer + implements Announcer +{ + @Override + public void start() {} + + @Override + public ListenableFuture forceAnnounce() + { + return Futures.immediateFuture(null); + } + + @Override + public void stop() {} +} diff --git a/core/trino-main/src/main/java/io/trino/node/TestingInternalNodeManager.java b/core/trino-main/src/main/java/io/trino/node/TestingInternalNodeManager.java new file mode 100644 index 000000000000..5b7e067ff0ce --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/node/TestingInternalNodeManager.java @@ -0,0 +1,167 @@ +/* + * 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.node; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.errorprone.annotations.ThreadSafe; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import com.google.inject.Inject; +import io.trino.client.NodeVersion; +import io.trino.spi.HostAddress; + +import java.net.URI; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.airlift.concurrent.Threads.daemonThreadsNamed; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.Executors.newSingleThreadExecutor; + +@ThreadSafe +public class TestingInternalNodeManager + implements InternalNodeManager +{ + public static final InternalNode CURRENT_NODE = new InternalNode("local", URI.create("local://127.0.0.1:8080"), NodeVersion.UNKNOWN, true); + + private final InternalNode currentNode; + private final ExecutorService nodeStateEventExecutor; + + @GuardedBy("this") + private AllNodes allNodes; + + @GuardedBy("this") + private final List> listeners = new ArrayList<>(); + + @Inject + public TestingInternalNodeManager(InternalNode currentNode) + { + this.currentNode = requireNonNull(currentNode, "currentNode is null"); + checkArgument(currentNode.isCoordinator(), "currentNode must be a coordinator: %s", currentNode); + + this.allNodes = new AllNodes( + ImmutableSet.of(currentNode), + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(currentNode)); + this.nodeStateEventExecutor = newSingleThreadExecutor(daemonThreadsNamed("node-state-events-%s")); + } + + public static TestingInternalNodeManager createDefault(InternalNode... remoteNodes) + { + return createDefault(ImmutableSet.copyOf(remoteNodes)); + } + + public static TestingInternalNodeManager createDefault(Set remoteNodes) + { + TestingInternalNodeManager nodeManager = new TestingInternalNodeManager(CURRENT_NODE); + nodeManager.addNodes(remoteNodes); + return nodeManager; + } + + public void addNodes(InternalNode... internalNodes) + { + addNodes(ImmutableSet.copyOf(internalNodes)); + } + + public synchronized void addNodes(Collection internalNodes) + { + checkArgument(internalNodes.stream().noneMatch(currentNode::equals), "Cannot add current node"); + Set newActiveNodes = ImmutableSet.builder() + .addAll(allNodes.activeNodes()) + .addAll(internalNodes) + .build(); + + setAllNodes(new AllNodes( + newActiveNodes, + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(), + newActiveNodes.stream() + .filter(InternalNode::isCoordinator) + .collect(toImmutableSet()))); + } + + public synchronized void removeNode(InternalNode internalNode) + { + requireNonNull(internalNode, "internalNode is null"); + checkArgument(!currentNode.equals(internalNode), "Cannot remove current node"); + + Set newActiveNodes = new HashSet<>(allNodes.activeNodes()); + newActiveNodes.remove(internalNode); + + setAllNodes(new AllNodes( + newActiveNodes, + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(), + ImmutableSet.of(), + newActiveNodes.stream() + .filter(InternalNode::isCoordinator) + .collect(toImmutableSet()))); + } + + @GuardedBy("this") + private void setAllNodes(AllNodes newAllNodes) + { + // did the node set change? + if (newAllNodes.equals(allNodes)) { + return; + } + allNodes = newAllNodes; + + // notify listeners + List> listeners = ImmutableList.copyOf(this.listeners); + nodeStateEventExecutor.submit(() -> listeners.forEach(listener -> listener.accept(newAllNodes))); + } + + @Override + public synchronized AllNodes getAllNodes() + { + return allNodes; + } + + @Override + public boolean isGone(HostAddress hostAddress) + { + return false; + } + + @Override + public void refreshNodes() {} + + @Override + public synchronized void addNodeChangeListener(Consumer listener) + { + listeners.add(requireNonNull(listener, "listener is null")); + AllNodes allNodes = this.allNodes; + nodeStateEventExecutor.submit(() -> listener.accept(allNodes)); + } + + @Override + public synchronized void removeNodeChangeListener(Consumer listener) + { + listeners.remove(requireNonNull(listener, "listener is null")); + } +} diff --git a/core/trino-main/src/main/java/io/trino/server/CoordinatorDiscoveryModule.java b/core/trino-main/src/main/java/io/trino/node/TestingNodeManagerModule.java similarity index 54% rename from core/trino-main/src/main/java/io/trino/server/CoordinatorDiscoveryModule.java rename to core/trino-main/src/main/java/io/trino/node/TestingNodeManagerModule.java index eeaf39e9d1f1..b128333ac583 100644 --- a/core/trino-main/src/main/java/io/trino/server/CoordinatorDiscoveryModule.java +++ b/core/trino-main/src/main/java/io/trino/node/TestingNodeManagerModule.java @@ -11,21 +11,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.server; +package io.trino.node; import com.google.inject.Binder; +import com.google.inject.Scopes; import io.airlift.configuration.AbstractConfigurationAwareModule; -import io.airlift.discovery.server.EmbeddedDiscoveryModule; +import io.trino.server.ServerConfig; -public class CoordinatorDiscoveryModule +public class TestingNodeManagerModule extends AbstractConfigurationAwareModule { @Override protected void setup(Binder binder) { - if (buildConfigObject(ServerConfig.class).isCoordinator() && - buildConfigObject(EmbeddedDiscoveryConfig.class).isEnabled()) { - install(new EmbeddedDiscoveryModule()); + ServerConfig serverConfig = buildConfigObject(ServerConfig.class); + if (serverConfig.isCoordinator()) { + binder.bind(TestingInternalNodeManager.class).in(Scopes.SINGLETON); + binder.bind(InternalNodeManager.class).to(TestingInternalNodeManager.class).in(Scopes.SINGLETON); } + else { + binder.bind(InternalNodeManager.class).to(WorkerInternalNodeManager.class).in(Scopes.SINGLETON); + } + binder.bind(Announcer.class).to(TestingAnnouncer.class).in(Scopes.SINGLETON); } } diff --git a/core/trino-main/src/main/java/io/trino/node/WorkerInternalNodeManager.java b/core/trino-main/src/main/java/io/trino/node/WorkerInternalNodeManager.java new file mode 100644 index 000000000000..0696a87e3cb9 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/node/WorkerInternalNodeManager.java @@ -0,0 +1,57 @@ +/* + * 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.node; + +import io.trino.spi.HostAddress; + +import java.util.function.Consumer; + +public class WorkerInternalNodeManager + implements InternalNodeManager +{ + @Override + public AllNodes getAllNodes() + { + throw new UnsupportedOperationException("Nodes cannot be listed on a worker node"); + } + + @Override + public boolean isGone(HostAddress hostAddress) + { + throw new UnsupportedOperationException("Nodes cannot be listed on a worker node"); + } + + @Override + public void refreshNodes() + { + throw new UnsupportedOperationException("Nodes cannot be listed on a worker node"); + } + + @Override + public void addNodeChangeListener(Consumer listener) + { + throw disabledException(); + } + + @Override + public void removeNodeChangeListener(Consumer listener) + { + throw disabledException(); + } + + private static UnsupportedOperationException disabledException() + { + return new UnsupportedOperationException("Nodes cannot be retrieved on a worker node"); + } +} diff --git a/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java b/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java index 91997f8fe96d..0c1079edabfa 100644 --- a/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java +++ b/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java @@ -49,6 +49,7 @@ import io.trino.execution.ExecutionFailureInfo; import io.trino.execution.ExplainAnalyzeContext; import io.trino.execution.ForQueryExecution; +import io.trino.execution.NodeTaskMap; import io.trino.execution.QueryExecution; import io.trino.execution.QueryExecutionMBean; import io.trino.execution.QueryExecutorInternal; @@ -68,8 +69,12 @@ import io.trino.execution.resourcegroups.LegacyResourceGroupConfigurationManager; import io.trino.execution.resourcegroups.ResourceGroupInfoProvider; import io.trino.execution.resourcegroups.ResourceGroupManager; +import io.trino.execution.scheduler.NodeScheduler; +import io.trino.execution.scheduler.NodeSchedulerConfig; import io.trino.execution.scheduler.SplitSchedulerStats; import io.trino.execution.scheduler.TaskExecutionStats; +import io.trino.execution.scheduler.TopologyAwareNodeSelectorModule; +import io.trino.execution.scheduler.UniformNodeSelectorModule; import io.trino.execution.scheduler.faulttolerant.BinPackingNodeAllocatorService; import io.trino.execution.scheduler.faulttolerant.ByEagerParentOutputStatsEstimator; import io.trino.execution.scheduler.faulttolerant.BySmallStageOutputStatsEstimator; @@ -88,7 +93,6 @@ import io.trino.execution.scheduler.policy.AllAtOnceExecutionPolicy; import io.trino.execution.scheduler.policy.ExecutionPolicy; import io.trino.execution.scheduler.policy.PhasedExecutionPolicy; -import io.trino.failuredetector.FailureDetectorModule; import io.trino.memory.ClusterMemoryManager; import io.trino.memory.ForMemoryManager; import io.trino.memory.LeastWastedEffortTaskLowMemoryKiller; @@ -144,10 +148,11 @@ import static io.airlift.concurrent.Threads.threadsNamed; import static io.airlift.configuration.ConditionalModule.conditionalModule; import static io.airlift.configuration.ConfigBinder.configBinder; -import static io.airlift.discovery.client.DiscoveryBinder.discoveryBinder; import static io.airlift.jaxrs.JaxrsBinder.jaxrsBinder; import static io.airlift.json.JsonCodecBinder.jsonCodecBinder; import static io.airlift.units.DataSize.Unit.MEGABYTE; +import static io.trino.execution.scheduler.NodeSchedulerConfig.NodeSchedulerPolicy.TOPOLOGY; +import static io.trino.execution.scheduler.NodeSchedulerConfig.NodeSchedulerPolicy.UNIFORM; import static io.trino.plugin.base.ClosingBinder.closingBinder; import static io.trino.server.InternalCommunicationHttpClientModule.internalHttpClientModule; import static io.trino.util.Executors.decorateWithVersion; @@ -165,9 +170,6 @@ protected void setup(Binder binder) { install(new WebUiModule()); - // coordinator announcement - discoveryBinder(binder).bindHttpAnnouncement("trino-coordinator"); - // statement resource jsonCodecBinder(binder).bindJsonCodec(TaskInfo.class); jaxrsBinder(binder).bind(QueuedStatementResource.class); @@ -182,9 +184,7 @@ protected void setup(Binder binder) config.setMaxResponseHeaderSize(DataSize.of(2, MEGABYTE)); }); - // failure detector - install(new FailureDetectorModule()); - jaxrsBinder(binder).bind(NodeResource.class); + // worker resource jaxrsBinder(binder).bind(WorkerResource.class); install(internalHttpClientModule("worker-info", ForWorkerInfo.class).build()); @@ -244,6 +244,21 @@ protected void setup(Binder binder) // node partitioning manager binder.bind(NodePartitioningManager.class).in(Scopes.SINGLETON); + // node scheduler + binder.bind(NodeScheduler.class).in(Scopes.SINGLETON); + binder.bind(NodeTaskMap.class).in(Scopes.SINGLETON); + newExporter(binder).export(NodeScheduler.class).withGeneratedName(); + + // network topology + install(conditionalModule( + NodeSchedulerConfig.class, + config -> UNIFORM == config.getNodeSchedulerPolicy(), + new UniformNodeSelectorModule())); + install(conditionalModule( + NodeSchedulerConfig.class, + config -> TOPOLOGY == config.getNodeSchedulerPolicy(), + new TopologyAwareNodeSelectorModule())); + // node allocator binder.bind(BinPackingNodeAllocatorService.class).in(Scopes.SINGLETON); newExporter(binder).export(BinPackingNodeAllocatorService.class).withGeneratedName(); diff --git a/core/trino-main/src/main/java/io/trino/server/HttpRemoteTaskFactory.java b/core/trino-main/src/main/java/io/trino/server/HttpRemoteTaskFactory.java index f7d4deb214f9..0fb4acb3e290 100644 --- a/core/trino-main/src/main/java/io/trino/server/HttpRemoteTaskFactory.java +++ b/core/trino-main/src/main/java/io/trino/server/HttpRemoteTaskFactory.java @@ -34,8 +34,8 @@ import io.trino.execution.TaskManagerConfig; import io.trino.execution.TaskStatus; import io.trino.execution.buffer.OutputBuffers; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.operator.ForScheduler; import io.trino.server.remotetask.HttpRemoteTask; import io.trino.server.remotetask.RemoteTaskStats; diff --git a/core/trino-main/src/main/java/io/trino/server/InternalCommunicationHttpClientModule.java b/core/trino-main/src/main/java/io/trino/server/InternalCommunicationHttpClientModule.java index d69c66f622ba..6d4c5b683b14 100644 --- a/core/trino-main/src/main/java/io/trino/server/InternalCommunicationHttpClientModule.java +++ b/core/trino-main/src/main/java/io/trino/server/InternalCommunicationHttpClientModule.java @@ -69,7 +69,7 @@ protected void setup(Binder binder) } } - static void configureClient(HttpClientConfig httpConfig, InternalCommunicationConfig internalCommunicationConfig) + public static void configureClient(HttpClientConfig httpConfig, InternalCommunicationConfig internalCommunicationConfig) { httpConfig.setHttp2Enabled(internalCommunicationConfig.isHttp2Enabled()); diff --git a/core/trino-main/src/main/java/io/trino/server/InternalCommunicationModule.java b/core/trino-main/src/main/java/io/trino/server/InternalCommunicationModule.java index 9290a2895cb9..9ef0c39767d4 100644 --- a/core/trino-main/src/main/java/io/trino/server/InternalCommunicationModule.java +++ b/core/trino-main/src/main/java/io/trino/server/InternalCommunicationModule.java @@ -15,26 +15,12 @@ import com.google.inject.Binder; import com.google.inject.Scopes; -import com.google.inject.multibindings.Multibinder; import io.airlift.configuration.AbstractConfigurationAwareModule; -import io.airlift.discovery.client.ForDiscoveryClient; -import io.airlift.http.client.HttpClientConfig; -import io.airlift.http.client.HttpRequestFilter; -import io.airlift.http.client.Request; import io.airlift.http.server.HttpsConfig; import io.airlift.node.NodeConfig; -import java.io.UncheckedIOException; -import java.net.InetAddress; -import java.net.URI; -import java.net.URISyntaxException; -import java.net.UnknownHostException; - -import static com.google.inject.multibindings.Multibinder.newSetBinder; import static io.airlift.configuration.ConfigBinder.configBinder; -import static io.airlift.node.AddressToHostname.encodeAddressAsHostname; import static io.airlift.node.NodeConfig.AddressSource.IP_ENCODED_AS_HOSTNAME; -import static io.trino.server.InternalCommunicationHttpClientModule.configureClient; public class InternalCommunicationModule extends AbstractConfigurationAwareModule @@ -43,47 +29,12 @@ public class InternalCommunicationModule protected void setup(Binder binder) { InternalCommunicationConfig internalCommunicationConfig = buildConfigObject(InternalCommunicationConfig.class); - Multibinder discoveryFilterBinder = newSetBinder(binder, HttpRequestFilter.class, ForDiscoveryClient.class); if (internalCommunicationConfig.isHttpsRequired() && internalCommunicationConfig.getKeyStorePath() == null && internalCommunicationConfig.getTrustStorePath() == null) { String sharedSecret = internalCommunicationConfig.getSharedSecret() .orElseThrow(() -> new IllegalArgumentException("Internal shared secret must be set when internal HTTPS is enabled")); configBinder(binder).bindConfigDefaults(HttpsConfig.class, config -> config.setAutomaticHttpsSharedSecret(sharedSecret)); configBinder(binder).bindConfigGlobalDefaults(NodeConfig.class, config -> config.setInternalAddressSource(IP_ENCODED_AS_HOSTNAME)); - discoveryFilterBinder.addBinding().to(DiscoveryEncodeAddressAsHostname.class); } - discoveryFilterBinder.addBinding().to(InternalAuthenticationManager.class); - configBinder(binder).bindConfigDefaults(HttpClientConfig.class, ForDiscoveryClient.class, config -> configureClient(config, internalCommunicationConfig)); binder.bind(InternalAuthenticationManager.class).in(Scopes.SINGLETON); } - - private static class DiscoveryEncodeAddressAsHostname - implements HttpRequestFilter - { - @Override - public Request filterRequest(Request request) - { - return Request.Builder.fromRequest(request) - .setUri(toIpEncodedAsHostnameUri(request.getUri())) - .build(); - } - - private static URI toIpEncodedAsHostnameUri(URI uri) - { - if (!uri.getScheme().equals("https")) { - return uri; - } - try { - String host = uri.getHost(); - InetAddress inetAddress = InetAddress.getByName(host); - String addressAsHostname = encodeAddressAsHostname(inetAddress); - return new URI(uri.getScheme(), uri.getUserInfo(), addressAsHostname, uri.getPort(), uri.getPath(), uri.getQuery(), uri.getFragment()); - } - catch (UnknownHostException e) { - throw new UncheckedIOException(e); - } - catch (URISyntaxException e) { - throw new RuntimeException(e); - } - } - } } diff --git a/core/trino-main/src/main/java/io/trino/server/NodeStateManager.java b/core/trino-main/src/main/java/io/trino/server/NodeStateManager.java index dee527519587..623f2cc3d4fe 100644 --- a/core/trino-main/src/main/java/io/trino/server/NodeStateManager.java +++ b/core/trino-main/src/main/java/io/trino/server/NodeStateManager.java @@ -22,7 +22,8 @@ import io.trino.execution.TaskId; import io.trino.execution.TaskInfo; import io.trino.execution.TaskState; -import io.trino.metadata.NodeState; +import io.trino.node.NodeState; +import io.trino.server.NodeStateManager.CurrentNodeState.VersionedState; import org.assertj.core.util.VisibleForTesting; import java.util.List; @@ -32,7 +33,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -41,10 +41,10 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; import static io.airlift.concurrent.Threads.threadsNamed; -import static io.trino.metadata.NodeState.ACTIVE; -import static io.trino.metadata.NodeState.DRAINED; -import static io.trino.metadata.NodeState.DRAINING; -import static io.trino.metadata.NodeState.SHUTTING_DOWN; +import static io.trino.node.NodeState.ACTIVE; +import static io.trino.node.NodeState.DRAINED; +import static io.trino.node.NodeState.DRAINING; +import static io.trino.node.NodeState.SHUTTING_DOWN; import static java.lang.String.format; import static java.lang.Thread.currentThread; import static java.util.Objects.requireNonNull; @@ -68,89 +68,23 @@ public class NodeStateManager private final Duration gracePeriod; private final ScheduledExecutorService executor; - private final AtomicReference nodeState = new AtomicReference<>(new VersionedState(ACTIVE, 0)); - private final AtomicLong stateVersionProvider = new AtomicLong(0); + private final CurrentNodeState nodeState; public interface SqlTasksObservable { void addStateChangeListener(TaskId taskId, StateMachine.StateChangeListener stateChangeListener); } - private class VersionedState - { - private final NodeState state; - private final long version; - - private VersionedState(NodeState state, long version) - { - this.state = requireNonNull(state, "state is null"); - this.version = version; - } - - public VersionedState toActive() - { - return new VersionedState(ACTIVE, nextStateVersion()); - } - - public VersionedState toDraining() - { - return new VersionedState(DRAINING, nextStateVersion()); - } - - public VersionedState toDrained() - { - return new VersionedState(DRAINED, nextStateVersion()); - } - - public VersionedState toShuttingDown() - { - return new VersionedState(SHUTTING_DOWN, nextStateVersion()); - } - - public NodeState state() - { - return state; - } - - public long version() - { - return version; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - VersionedState that = (VersionedState) o; - return version == that.version && state == that.state; - } - - @Override - public int hashCode() - { - return Objects.hash(state, version); - } - - @Override - public String toString() - { - return String.format("%s-%s", state.toString(), version); - } - } - @Inject public NodeStateManager( + CurrentNodeState nodeState, SqlTaskManager sqlTaskManager, ServerConfig serverConfig, ShutdownAction shutdownAction, LifeCycleManager lifeCycleManager) { - this(requireNonNull(sqlTaskManager, "sqlTaskManager is null")::addStateChangeListener, + this(nodeState, + requireNonNull(sqlTaskManager, "sqlTaskManager is null")::addStateChangeListener, requireNonNull(sqlTaskManager, "sqlTaskManager is null")::getAllTaskInfo, serverConfig, shutdownAction, @@ -160,6 +94,7 @@ public NodeStateManager( @VisibleForTesting public NodeStateManager( + CurrentNodeState nodeState, SqlTasksObservable sqlTasksObservable, Supplier> taskInfoSupplier, ServerConfig serverConfig, @@ -167,6 +102,7 @@ public NodeStateManager( LifeCycleManager lifeCycleManager, ScheduledExecutorService executor) { + this.nodeState = requireNonNull(nodeState, "nodeState is null"); this.sqlTasksObservable = requireNonNull(sqlTasksObservable, "sqlTasksObservable is null"); this.taskInfoSupplier = requireNonNull(taskInfoSupplier, "taskInfoSupplier is null"); this.shutdownAction = requireNonNull(shutdownAction, "shutdownAction is null"); @@ -178,7 +114,7 @@ public NodeStateManager( public NodeState getServerState() { - return nodeState.get().state(); + return nodeState.getVersion().state(); } /* @@ -201,17 +137,17 @@ public NodeState getServerState() */ public synchronized void transitionState(NodeState state) { - VersionedState currState = nodeState.get(); + VersionedState currState = nodeState.getVersion(); if (currState.state() == state) { return; } switch (state) { case ACTIVE -> { - if (currState.state() == DRAINING && nodeState.compareAndSet(currState, currState.toActive())) { + if (currState.state() == DRAINING && nodeState.compareAndSetVersion(currState, currState.toActive())) { return; } - if (currState.state() == DRAINED && nodeState.compareAndSet(currState, currState.toActive())) { + if (currState.state() == DRAINED && nodeState.compareAndSetVersion(currState, currState.toActive())) { return; } } @@ -220,11 +156,11 @@ public synchronized void transitionState(NodeState state) throw new UnsupportedOperationException("Cannot shutdown coordinator"); } VersionedState shuttingDown = currState.toShuttingDown(); - if (currState.state() == DRAINED && nodeState.compareAndSet(currState, shuttingDown)) { + if (currState.state() == DRAINED && nodeState.compareAndSetVersion(currState, shuttingDown)) { requestTerminate(); return; } - nodeState.set(shuttingDown); + nodeState.setVersion(shuttingDown); requestGracefulShutdown(); return; } @@ -232,31 +168,26 @@ public synchronized void transitionState(NodeState state) if (isCoordinator) { throw new UnsupportedOperationException("Cannot drain coordinator"); } - if (currState.state() == ACTIVE && nodeState.compareAndSet(currState, currState.toDraining())) { + if (currState.state() == ACTIVE && nodeState.compareAndSetVersion(currState, currState.toDraining())) { requestDrain(); return; } } - case DRAINED -> throw new IllegalStateException(format("Invalid state transition from %s to %s, transition to DRAINED is internal only", currState, state)); - case INACTIVE -> throw new IllegalStateException(format("Invalid state transition from %s to %s, INACTIVE is not a valid internal state", currState, state)); + case INACTIVE, DRAINED, INVALID, GONE -> + throw new IllegalArgumentException("Cannot transition state to internal state " + state); } throw new IllegalStateException(format("Invalid state transition from %s to %s", currState, state)); } - private long nextStateVersion() - { - return stateVersionProvider.incrementAndGet(); - } - private synchronized void requestDrain() { log.debug("Drain requested, NodeState: %s", getServerState()); // wait for a grace period (so that draining state is observed by the coordinator) before starting draining // when coordinator observes draining no new tasks are assigned to this worker - VersionedState expectedState = nodeState.get(); + VersionedState expectedState = nodeState.getVersion(); executor.schedule(() -> drain(expectedState), gracePeriod.toMillis(), MILLISECONDS); } @@ -271,7 +202,7 @@ private void requestGracefulShutdown() { log.info("Shutdown requested"); - VersionedState expectedState = nodeState.get(); + VersionedState expectedState = nodeState.getVersion(); // wait for a grace period (so that shutting down state is observed by the coordinator) to start the shutdown sequence shutdownHandler.schedule(() -> shutdown(expectedState), gracePeriod.toMillis(), MILLISECONDS); } @@ -308,7 +239,7 @@ private void terminate() private void drain(VersionedState expectedState) { - if (nodeState.get() == expectedState) { + if (nodeState.getVersion() == expectedState) { waitActiveTasksToFinish(expectedState); } drainingComplete(expectedState); @@ -317,12 +248,12 @@ private void drain(VersionedState expectedState) private synchronized void drainingComplete(VersionedState expectedState) { VersionedState drained = expectedState.toDrained(); - boolean success = nodeState.compareAndSet(expectedState, drained); + boolean success = nodeState.compareAndSetVersion(expectedState, drained); if (success) { log.info("Worker State change: DRAINING -> DRAINED, server can be safely SHUT DOWN."); } else { - log.info("Worker State change: %s, expected: %s, will not transition to DRAINED", nodeState.get(), expectedState); + log.info("Worker State change: %s, expected: %s, will not transition to DRAINED", nodeState.getVersion(), expectedState); } } @@ -330,7 +261,7 @@ private void waitActiveTasksToFinish(VersionedState expectedState) { // At this point no new tasks should be scheduled by coordinator on this worker node. // Wait for all remaining tasks to finish. - while (nodeState.get() == expectedState) { + while (nodeState.getVersion() == expectedState) { List activeTasks = getActiveTasks(); log.info("Waiting for %s active tasks to finish", activeTasks.size()); if (activeTasks.isEmpty()) { @@ -341,7 +272,7 @@ private void waitActiveTasksToFinish(VersionedState expectedState) } // wait for another grace period for all task states to be observed by the coordinator - if (nodeState.get() == expectedState) { + if (nodeState.getVersion() == expectedState) { sleepUninterruptibly(gracePeriod.toMillis(), MILLISECONDS); } } @@ -360,8 +291,8 @@ private void waitTasksToFinish(List activeTasks, VersionedState expect } try { - while (!countDownLatch.await(1, TimeUnit.SECONDS)) { - if (nodeState.get() != expectedState) { + while (!countDownLatch.await(1, SECONDS)) { + if (nodeState.getVersion() != expectedState) { log.info("Wait for tasks interrupted by state change, worker is no longer draining."); break; @@ -381,4 +312,104 @@ private List getActiveTasks() .filter(taskInfo -> !taskInfo.taskStatus().getState().isDone()) .collect(toImmutableList()); } + + public static class CurrentNodeState + implements Supplier + { + private final AtomicReference nodeState = new AtomicReference<>(new VersionedState(ACTIVE, 0)); + private final AtomicLong stateVersionProvider = new AtomicLong(0); + + @Override + public NodeState get() + { + return getVersion().state(); + } + + private VersionedState getVersion() + { + return nodeState.get(); + } + + private void setVersion(VersionedState newValue) + { + nodeState.set(newValue); + } + + private boolean compareAndSetVersion(VersionedState expectedValue, VersionedState newValue) + { + return nodeState.compareAndSet(expectedValue, newValue); + } + + private long nextStateVersion() + { + return stateVersionProvider.incrementAndGet(); + } + + class VersionedState + { + private final NodeState state; + private final long version; + + private VersionedState(NodeState state, long version) + { + this.state = requireNonNull(state, "state is null"); + this.version = version; + } + + public VersionedState toActive() + { + return new VersionedState(ACTIVE, nextStateVersion()); + } + + public VersionedState toDraining() + { + return new VersionedState(DRAINING, nextStateVersion()); + } + + public VersionedState toDrained() + { + return new VersionedState(DRAINED, nextStateVersion()); + } + + public VersionedState toShuttingDown() + { + return new VersionedState(SHUTTING_DOWN, nextStateVersion()); + } + + public NodeState state() + { + return state; + } + + public long version() + { + return version; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + VersionedState that = (VersionedState) o; + return version == that.version && state == that.state; + } + + @Override + public int hashCode() + { + return Objects.hash(state, version); + } + + @Override + public String toString() + { + return "%s-%s".formatted(state.toString(), version); + } + } + } } diff --git a/core/trino-main/src/main/java/io/trino/server/NodeStateManagerModule.java b/core/trino-main/src/main/java/io/trino/server/NodeStateManagerModule.java index 02422c2e47e6..dbbb0b13486c 100644 --- a/core/trino-main/src/main/java/io/trino/server/NodeStateManagerModule.java +++ b/core/trino-main/src/main/java/io/trino/server/NodeStateManagerModule.java @@ -16,6 +16,7 @@ import com.google.inject.Binder; import com.google.inject.Scopes; import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.server.NodeStateManager.CurrentNodeState; public class NodeStateManagerModule extends AbstractConfigurationAwareModule @@ -24,6 +25,7 @@ public class NodeStateManagerModule protected void setup(Binder binder) { binder.bind(ShutdownAction.class).to(DefaultShutdownAction.class).in(Scopes.SINGLETON); + binder.bind(CurrentNodeState.class).in(Scopes.SINGLETON); binder.bind(NodeStateManager.class).in(Scopes.SINGLETON); } } diff --git a/core/trino-main/src/main/java/io/trino/server/Server.java b/core/trino-main/src/main/java/io/trino/server/Server.java index 3fe6a5934aa4..0636adfe04b9 100644 --- a/core/trino-main/src/main/java/io/trino/server/Server.java +++ b/core/trino-main/src/main/java/io/trino/server/Server.java @@ -24,8 +24,6 @@ import io.airlift.compress.v3.lz4.Lz4NativeCompressor; import io.airlift.compress.v3.snappy.SnappyNativeCompressor; import io.airlift.compress.v3.zstd.ZstdNativeCompressor; -import io.airlift.discovery.client.Announcer; -import io.airlift.discovery.client.DiscoveryModule; import io.airlift.http.server.HttpServerModule; import io.airlift.jaxrs.JaxrsModule; import io.airlift.jmx.JmxHttpModule; @@ -47,6 +45,8 @@ import io.trino.exchange.ExchangeManagerRegistry; import io.trino.execution.resourcegroups.ResourceGroupManager; import io.trino.execution.warnings.WarningCollectorModule; +import io.trino.node.Announcer; +import io.trino.node.NodeManagerModule; import io.trino.security.AccessControlManager; import io.trino.security.AccessControlModule; import io.trino.security.GroupProviderManager; @@ -94,7 +94,6 @@ private void doStart(String trinoVersion) ImmutableList.Builder modules = ImmutableList.builder(); modules.add( new NodeModule(), - new DiscoveryModule(), new HttpServerModule(), new JsonModule(), new JaxrsModule(), @@ -109,9 +108,9 @@ private void doStart(String trinoVersion) new AccessControlModule(), new EventListenerModule(), new ExchangeManagerModule(), - new CoordinatorDiscoveryModule(), new CatalogManagerModule(), new TransactionManagerModule(), + new NodeManagerModule(trinoVersion), new ServerMainModule(trinoVersion), new NodeStateManagerModule(), new WarningCollectorModule()); diff --git a/core/trino-main/src/main/java/io/trino/server/ServerInfo.java b/core/trino-main/src/main/java/io/trino/server/ServerInfo.java new file mode 100644 index 000000000000..5359c3f874eb --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/server/ServerInfo.java @@ -0,0 +1,43 @@ +/* + * 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.server; + +import io.airlift.units.Duration; +import io.trino.client.NodeVersion; +import io.trino.node.NodeState; + +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public record ServerInfo( + String nodeId, + NodeState state, + NodeVersion nodeVersion, + String environment, + boolean coordinator, + Optional coordinatorId, + boolean starting, + Duration uptime) +{ + public ServerInfo + { + requireNonNull(nodeId, "nodeId is null"); + requireNonNull(state, "state is null"); + requireNonNull(nodeVersion, "nodeVersion is null"); + requireNonNull(environment, "environment is null"); + requireNonNull(coordinatorId, "coordinatorId is null"); + requireNonNull(uptime, "uptime is null"); + } +} diff --git a/core/trino-main/src/main/java/io/trino/server/ServerInfoResource.java b/core/trino-main/src/main/java/io/trino/server/ServerInfoResource.java index 4d90bcada165..2d7445a523cb 100644 --- a/core/trino-main/src/main/java/io/trino/server/ServerInfoResource.java +++ b/core/trino-main/src/main/java/io/trino/server/ServerInfoResource.java @@ -17,9 +17,8 @@ import io.airlift.log.Logger; import io.airlift.node.NodeInfo; import io.trino.client.NodeVersion; -import io.trino.client.ServerInfo; import io.trino.execution.QueryIdGenerator; -import io.trino.metadata.NodeState; +import io.trino.node.NodeState; import io.trino.server.security.ResourceSecurity; import jakarta.ws.rs.BadRequestException; import jakarta.ws.rs.Consumes; @@ -45,6 +44,7 @@ public class ServerInfoResource { private static final Logger log = Logger.get(ServerInfoResource.class); + private final String nodeId; private final NodeVersion version; private final String environment; private final boolean coordinator; @@ -62,6 +62,7 @@ public ServerInfoResource( StartupStatus startupStatus, Optional queryIdGenerator) { + this.nodeId = nodeInfo.getNodeId(); this.version = requireNonNull(nodeVersion, "nodeVersion is null"); this.environment = nodeInfo.getEnvironment(); this.coordinator = serverConfig.isCoordinator(); @@ -76,7 +77,15 @@ public ServerInfoResource( public ServerInfo getInfo() { boolean starting = !startupStatus.isStartupComplete(); - return new ServerInfo(version, environment, coordinator, starting, Optional.of(nanosSince(startTime)), queryIdGenerator.map(QueryIdGenerator::getCoordinatorId)); + return new ServerInfo( + nodeId, + nodeStateManager.getServerState(), + version, + environment, + coordinator, + queryIdGenerator.map(QueryIdGenerator::getCoordinatorId), + starting, + nanosSince(startTime)); } @ResourceSecurity(MANAGEMENT_WRITE) diff --git a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java index ecbd661fc179..28095585f950 100644 --- a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java +++ b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java @@ -23,6 +23,8 @@ import io.airlift.concurrent.BoundedExecutor; import io.airlift.configuration.AbstractConfigurationAwareModule; import io.airlift.http.server.HttpServerConfig; +import io.airlift.http.server.HttpServerInfo; +import io.airlift.node.NodeInfo; import io.airlift.slice.Slice; import io.airlift.stats.GcMonitor; import io.airlift.stats.JmxGcMonitor; @@ -43,7 +45,6 @@ import io.trino.execution.LocationFactory; import io.trino.execution.MemoryRevokingScheduler; import io.trino.execution.NoOpFailureInjector; -import io.trino.execution.NodeTaskMap; import io.trino.execution.QueryIdGenerator; import io.trino.execution.QueryManagerConfig; import io.trino.execution.SqlTaskManager; @@ -54,10 +55,7 @@ import io.trino.execution.executor.dedicated.ThreadPerDriverTaskExecutor; import io.trino.execution.executor.timesharing.MultilevelSplitQueue; import io.trino.execution.executor.timesharing.TimeSharingTaskExecutor; -import io.trino.execution.scheduler.NodeScheduler; import io.trino.execution.scheduler.NodeSchedulerConfig; -import io.trino.execution.scheduler.TopologyAwareNodeSelectorModule; -import io.trino.execution.scheduler.UniformNodeSelectorModule; import io.trino.memory.LocalMemoryManager; import io.trino.memory.LocalMemoryManagerExporter; import io.trino.memory.MemoryInfo; @@ -66,15 +64,12 @@ import io.trino.memory.NodeMemoryConfig; import io.trino.metadata.BlockEncodingManager; import io.trino.metadata.DisabledSystemSecurityMetadata; -import io.trino.metadata.DiscoveryNodeManager; -import io.trino.metadata.ForNodeManager; import io.trino.metadata.FunctionBundle; import io.trino.metadata.FunctionManager; import io.trino.metadata.GlobalFunctionCatalog; import io.trino.metadata.HandleJsonModule; import io.trino.metadata.InternalBlockEncodingSerde; import io.trino.metadata.InternalFunctionBundle; -import io.trino.metadata.InternalNodeManager; import io.trino.metadata.LanguageFunctionEngineManager; import io.trino.metadata.LanguageFunctionManager; import io.trino.metadata.Metadata; @@ -85,6 +80,7 @@ import io.trino.metadata.TableFunctionRegistry; import io.trino.metadata.TableProceduresRegistry; import io.trino.metadata.TypeRegistry; +import io.trino.node.InternalNode; import io.trino.operator.DirectExchangeClientConfig; import io.trino.operator.DirectExchangeClientFactory; import io.trino.operator.DirectExchangeClientSupplier; @@ -168,15 +164,11 @@ import static com.google.inject.multibindings.Multibinder.newSetBinder; import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; import static io.airlift.concurrent.Threads.daemonThreadsNamed; -import static io.airlift.configuration.ConditionalModule.conditionalModule; import static io.airlift.configuration.ConfigBinder.configBinder; -import static io.airlift.discovery.client.DiscoveryBinder.discoveryBinder; import static io.airlift.jaxrs.JaxrsBinder.jaxrsBinder; import static io.airlift.json.JsonBinder.jsonBinder; import static io.airlift.json.JsonCodecBinder.jsonCodecBinder; import static io.airlift.units.DataSize.Unit.MEGABYTE; -import static io.trino.execution.scheduler.NodeSchedulerConfig.NodeSchedulerPolicy.TOPOLOGY; -import static io.trino.execution.scheduler.NodeSchedulerConfig.NodeSchedulerPolicy.UNIFORM; import static io.trino.operator.RetryPolicy.TASK; import static io.trino.plugin.base.ClosingBinder.closingBinder; import static io.trino.server.InternalCommunicationHttpClientModule.internalHttpClientModule; @@ -251,34 +243,8 @@ protected void setup(Binder binder) binder.bind(SystemSessionProperties.class).in(Scopes.SINGLETON); binder.bind(SessionPropertyDefaults.class).in(Scopes.SINGLETON); - // node manager - discoveryBinder(binder).bindSelector("trino"); - binder.bind(DiscoveryNodeManager.class).in(Scopes.SINGLETON); - binder.bind(InternalNodeManager.class).to(DiscoveryNodeManager.class).in(Scopes.SINGLETON); - newExporter(binder).export(DiscoveryNodeManager.class).withGeneratedName(); - install(internalHttpClientModule("node-manager", ForNodeManager.class) - .withConfigDefaults(config -> { - config.setIdleTimeout(new Duration(30, SECONDS)); - config.setRequestTimeout(new Duration(10, SECONDS)); - }).build()); - - // node scheduler - // TODO: remove from NodePartitioningManager and move to CoordinatorModule + // TODO: move to CoordinatorModule when SystemSessionProperties and DefaultCatalogFactory are moved configBinder(binder).bindConfig(NodeSchedulerConfig.class); - binder.bind(NodeScheduler.class).in(Scopes.SINGLETON); - binder.bind(NodeTaskMap.class).in(Scopes.SINGLETON); - newExporter(binder).export(NodeScheduler.class).withGeneratedName(); - - // network topology - // TODO: move to CoordinatorModule when NodeScheduler is moved - install(conditionalModule( - NodeSchedulerConfig.class, - config -> UNIFORM == config.getNodeSchedulerPolicy(), - new UniformNodeSelectorModule())); - install(conditionalModule( - NodeSchedulerConfig.class, - config -> TOPOLOGY == config.getNodeSchedulerPolicy(), - new TopologyAwareNodeSelectorModule())); // task execution newOptionalBinder(binder, FailureInjector.class).setDefault().to(NoOpFailureInjector.class).in(Scopes.SINGLETON); @@ -450,11 +416,8 @@ protected void setup(Binder binder) // split monitor binder.bind(SplitMonitor.class).in(Scopes.SINGLETON); - // version and announcement + // node version binder.bind(NodeVersion.class).toInstance(new NodeVersion(nodeVersion)); - discoveryBinder(binder).bindHttpAnnouncement("trino") - .addProperty("node_version", nodeVersion) - .addProperty("coordinator", String.valueOf(serverConfig.isCoordinator())); // server info resource jaxrsBinder(binder).bind(ServerInfoResource.class); @@ -514,6 +477,22 @@ protected void setup(Binder binder) closingBinder(binder).registerExecutor(Key.get(ScheduledExecutorService.class, ForAsyncHttp.class)); } + @Provides + @Singleton + public static InternalNode currentInternalNode( + NodeInfo nodeInfo, + HttpServerInfo httpServerInfo, + NodeVersion nodeVersion, + ServerConfig serverConfig, + InternalCommunicationConfig internalCommunicationConfig) + { + return new InternalNode( + nodeInfo.getNodeId(), + internalCommunicationConfig.isHttpsRequired() ? httpServerInfo.getHttpsUri() : httpServerInfo.getHttpUri(), + nodeVersion, + serverConfig.isCoordinator()); + } + private static class RegisterFunctionBundles { @Inject diff --git a/core/trino-main/src/main/java/io/trino/server/protocol/spooling/CoordinatorSegmentResource.java b/core/trino-main/src/main/java/io/trino/server/protocol/spooling/CoordinatorSegmentResource.java index dbbfd855622e..9d938d30e2d1 100644 --- a/core/trino-main/src/main/java/io/trino/server/protocol/spooling/CoordinatorSegmentResource.java +++ b/core/trino-main/src/main/java/io/trino/server/protocol/spooling/CoordinatorSegmentResource.java @@ -14,8 +14,8 @@ package io.trino.server.protocol.spooling; import com.google.inject.Inject; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; import io.trino.server.ExternalUriInfo; import io.trino.server.protocol.spooling.SpoolingConfig.SegmentRetrievalMode; import io.trino.server.security.ResourceSecurity; diff --git a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpLocationFactory.java b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpLocationFactory.java index 6e68a36bc5a3..7d5857be1b9d 100644 --- a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpLocationFactory.java +++ b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpLocationFactory.java @@ -17,8 +17,7 @@ import io.airlift.http.server.HttpServerInfo; import io.trino.execution.LocationFactory; import io.trino.execution.TaskId; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; +import io.trino.node.InternalNode; import io.trino.server.InternalCommunicationConfig; import io.trino.spi.QueryId; @@ -30,18 +29,18 @@ public class HttpLocationFactory implements LocationFactory { - private final InternalNodeManager nodeManager; + private final InternalNode currentNode; private final URI baseUri; @Inject - public HttpLocationFactory(InternalNodeManager nodeManager, HttpServerInfo httpServerInfo, InternalCommunicationConfig config) + public HttpLocationFactory(InternalNode currentNode, HttpServerInfo httpServerInfo, InternalCommunicationConfig config) { - this(nodeManager, config.isHttpsRequired() ? httpServerInfo.getHttpsUri() : httpServerInfo.getHttpUri()); + this(currentNode, config.isHttpsRequired() ? httpServerInfo.getHttpsUri() : httpServerInfo.getHttpUri()); } - public HttpLocationFactory(InternalNodeManager nodeManager, URI baseUri) + public HttpLocationFactory(InternalNode currentNode, URI baseUri) { - this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); + this.currentNode = currentNode; this.baseUri = requireNonNull(baseUri, "baseUri is null"); } @@ -58,7 +57,7 @@ public URI createQueryLocation(QueryId queryId) @Override public URI createLocalTaskLocation(TaskId taskId) { - return createTaskLocation(nodeManager.getCurrentNode(), taskId); + return createTaskLocation(currentNode, taskId); } @Override diff --git a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java index 3c5bd3b6ad09..eec10df3901a 100644 --- a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java +++ b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java @@ -57,8 +57,8 @@ import io.trino.execution.buffer.PipelinedBufferInfo; import io.trino.execution.buffer.PipelinedOutputBuffers; import io.trino.execution.buffer.SpoolingOutputStats; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.operator.RetryPolicy; import io.trino.operator.TaskStats; import io.trino.server.DynamicFilterService; diff --git a/core/trino-main/src/main/java/io/trino/server/security/ServerSecurityModule.java b/core/trino-main/src/main/java/io/trino/server/security/ServerSecurityModule.java index 2225936cc86b..7a12d838a925 100644 --- a/core/trino-main/src/main/java/io/trino/server/security/ServerSecurityModule.java +++ b/core/trino-main/src/main/java/io/trino/server/security/ServerSecurityModule.java @@ -20,9 +20,6 @@ import com.google.inject.Scopes; import com.google.inject.multibindings.MapBinder; import io.airlift.configuration.AbstractConfigurationAwareModule; -import io.airlift.discovery.server.DynamicAnnouncementResource; -import io.airlift.discovery.server.ServiceResource; -import io.airlift.discovery.store.StoreResource; import io.airlift.http.server.HttpServer.ClientCertificate; import io.airlift.http.server.HttpServerConfig; import io.airlift.jmx.MBeanResource; @@ -59,11 +56,8 @@ protected void setup(Binder binder) jaxrsBinder(binder).bind(ResourceSecurityDynamicFeature.class); resourceSecurityBinder(binder) - .managementReadResource(ServiceResource.class) .managementReadResource(MBeanResource.class) - .managementReadResource(MetricsResource.class) - .internalOnlyResource(DynamicAnnouncementResource.class) - .internalOnlyResource(StoreResource.class); + .managementReadResource(MetricsResource.class); newOptionalBinder(binder, PasswordAuthenticatorManager.class); binder.bind(CertificateAuthenticatorManager.class).in(Scopes.SINGLETON); diff --git a/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java b/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java index 54fe75471c24..a703bc161d24 100644 --- a/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java +++ b/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java @@ -26,10 +26,6 @@ import com.google.inject.TypeLiteral; import io.airlift.bootstrap.Bootstrap; import io.airlift.bootstrap.LifeCycleManager; -import io.airlift.discovery.client.Announcer; -import io.airlift.discovery.client.DiscoveryModule; -import io.airlift.discovery.client.ServiceSelectorManager; -import io.airlift.discovery.client.testing.TestingDiscoveryModule; import io.airlift.http.server.testing.TestingHttpServer; import io.airlift.http.server.testing.TestingHttpServerModule; import io.airlift.http.server.tracing.TracingServletFilter; @@ -64,18 +60,21 @@ import io.trino.execution.resourcegroups.InternalResourceGroupManager; import io.trino.memory.ClusterMemoryManager; import io.trino.memory.LocalMemoryManager; -import io.trino.metadata.AllNodes; import io.trino.metadata.CatalogManager; import io.trino.metadata.FunctionBundle; import io.trino.metadata.GlobalFunctionCatalog; -import io.trino.metadata.InternalNodeManager; import io.trino.metadata.SessionPropertyManager; import io.trino.metadata.TablePropertyManager; +import io.trino.node.Announcer; +import io.trino.node.InternalNode; +import io.trino.node.TestingInternalNodeManager; +import io.trino.node.TestingNodeManagerModule; import io.trino.security.AccessControl; import io.trino.security.AccessControlConfig; import io.trino.security.AccessControlManager; import io.trino.security.GroupProviderManager; import io.trino.server.NodeStateManager; +import io.trino.server.NodeStateManager.CurrentNodeState; import io.trino.server.PluginInstaller; import io.trino.server.PrefixObjectNameGeneratorModule; import io.trino.server.QuerySessionSupplier; @@ -184,6 +183,7 @@ public static Builder builder() private final boolean preserveData; private final LifeCycleManager lifeCycleManager; private final PluginInstaller pluginInstaller; + private final InternalNode currentNode; private final Optional catalogManager; private final TestingHttpServer server; private final TransactionManager transactionManager; @@ -203,8 +203,7 @@ public static Builder builder() private final NodePartitioningManager nodePartitioningManager; private final ClusterMemoryManager clusterMemoryManager; private final LocalMemoryManager localMemoryManager; - private final InternalNodeManager nodeManager; - private final ServiceSelectorManager serviceSelectorManager; + private final TestingInternalNodeManager nodeManager; private final DispatchManager dispatchManager; private final SqlQueryManager queryManager; private final SqlTaskManager taskManager; @@ -247,7 +246,6 @@ private TestingTrinoServer( boolean coordinator, Map properties, Optional environment, - Optional discoveryUri, Module additionalModule, Optional baseDataDir, Optional spanProcessor, @@ -286,7 +284,6 @@ private TestingTrinoServer( serverProperties.put("catalog.store", "memory"); } } - serverProperties.put("failure-detector.enabled", "false"); // Reduce memory footprint in tests serverProperties.put("query.min-expire-age", "5s"); @@ -307,6 +304,7 @@ private TestingTrinoServer( .add(new ServerSecurityModule()) .add(new CatalogManagerModule()) .add(new TransactionManagerModule()) + .add(new TestingNodeManagerModule()) .add(new ServerMainModule(VERSION)) .add(new TestingWarningCollectorModule()) .add(binder -> { @@ -326,6 +324,7 @@ private TestingTrinoServer( binder.bind(AccessControl.class).annotatedWith(ForTracing.class).to(AccessControlManager.class).in(Scopes.SINGLETON); binder.bind(AccessControl.class).to(TracingAccessControl.class).in(Scopes.SINGLETON); binder.bind(ShutdownAction.class).to(TestShutdownAction.class).in(Scopes.SINGLETON); + binder.bind(CurrentNodeState.class).in(Scopes.SINGLETON); binder.bind(NodeStateManager.class).in(Scopes.SINGLETON); binder.bind(ProcedureTester.class).in(Scopes.SINGLETON); binder.bind(ExchangeManagerRegistry.class).in(Scopes.SINGLETON); @@ -351,15 +350,6 @@ private TestingTrinoServer( modules.add(new TestingSessionTimeModule()); } - if (discoveryUri.isPresent()) { - requireNonNull(environment, "environment required when discoveryUri is present"); - serverProperties.put("discovery.uri", discoveryUri.get().toString()); - modules.add(new DiscoveryModule()); - } - else { - modules.add(new TestingDiscoveryModule()); - } - modules.add(additionalModule); Bootstrap app = new Bootstrap(modules.build()); @@ -380,6 +370,8 @@ private TestingTrinoServer( pluginInstaller = injector.getInstance(PluginInstaller.class); + currentNode = injector.getInstance(InternalNode.class); + var catalogStoreManager = injector.getInstance(Key.get(new TypeLiteral>() {})); catalogStoreManager.ifPresent(CatalogStoreManager::loadConfiguredCatalogStore); @@ -411,6 +403,7 @@ private TestingTrinoServer( clusterMemoryManager = injector.getInstance(ClusterMemoryManager.class); statsCalculator = injector.getInstance(StatsCalculator.class); injector.getInstance(CertificateAuthenticatorManager.class).useDefaultAuthenticator(); + nodeManager = injector.getInstance(TestingInternalNodeManager.class); } else { dispatchManager = null; @@ -421,10 +414,9 @@ private TestingTrinoServer( nodePartitioningManager = null; clusterMemoryManager = null; statsCalculator = null; + nodeManager = null; } localMemoryManager = injector.getInstance(LocalMemoryManager.class); - nodeManager = injector.getInstance(InternalNodeManager.class); - serviceSelectorManager = injector.getInstance(ServiceSelectorManager.class); nodeStateManager = injector.getInstance(NodeStateManager.class); taskManager = injector.getInstance(SqlTaskManager.class); shutdownAction = injector.getInstance(ShutdownAction.class); @@ -451,8 +443,6 @@ private TestingTrinoServer( // Technically `this` reference might escape here. However, the object is fully constructed. additionalConfiguration.accept(this); injector.getInstance(StartupStatus.class).startupComplete(); - - refreshNodes(); } @Override @@ -479,6 +469,11 @@ public void installPlugin(Plugin plugin) pluginInstaller.installPlugin(plugin); } + public InternalNode getCurrentNode() + { + return currentNode; + } + public DispatchManager getDispatchManager() { return dispatchManager; @@ -682,11 +677,23 @@ public boolean isCoordinator() return coordinator; } - public final AllNodes refreshNodes() + public void registerServer(InternalNode server) { - serviceSelectorManager.forceRefresh(); - nodeManager.refreshNodes(); - return nodeManager.getAllNodes(); + checkState(coordinator, "Current server is not a coordinator"); + nodeManager.addNodes(server); + } + + public void unregisterServer(InternalNode worker) + { + checkState(coordinator, "Current server is not a coordinator"); + nodeManager.removeNode(worker); + } + + public int getWorkerCount() + { + return (int) nodeManager.getAllNodes().activeNodes().stream() + .filter(node -> !currentNode.equals(node)) + .count(); } public T getInstance(Key key) @@ -726,7 +733,6 @@ public static class Builder private boolean coordinator = true; private Map properties = ImmutableMap.of(); private Optional environment = Optional.empty(); - private Optional discoveryUri = Optional.empty(); private Module additionalModule = EMPTY_MODULE; private Optional baseDataDir = Optional.empty(); private Optional spanProcessor = Optional.empty(); @@ -764,12 +770,6 @@ public Builder setEnvironment(String environment) return this; } - public Builder setDiscoveryUri(URI discoveryUri) - { - this.discoveryUri = Optional.of(discoveryUri); - return this; - } - public Builder setAdditionalModule(Module additionalModule) { this.additionalModule = requireNonNull(additionalModule, "additionalModule is null"); @@ -829,7 +829,6 @@ public TestingTrinoServer build() coordinator, properties, environment, - discoveryUri, additionalModule, baseDataDir, spanProcessor, diff --git a/core/trino-main/src/main/java/io/trino/server/ui/ClusterStatsResource.java b/core/trino-main/src/main/java/io/trino/server/ui/ClusterStatsResource.java index 0125486fdf0b..36e8480b71e2 100644 --- a/core/trino-main/src/main/java/io/trino/server/ui/ClusterStatsResource.java +++ b/core/trino-main/src/main/java/io/trino/server/ui/ClusterStatsResource.java @@ -20,9 +20,9 @@ import io.trino.execution.QueryState; import io.trino.execution.scheduler.NodeSchedulerConfig; import io.trino.memory.ClusterMemoryManager; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; -import io.trino.metadata.NodeState; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; +import io.trino.node.NodeState; import io.trino.server.BasicQueryInfo; import io.trino.server.security.ResourceSecurity; import jakarta.ws.rs.GET; diff --git a/core/trino-main/src/main/java/io/trino/server/ui/WorkerResource.java b/core/trino-main/src/main/java/io/trino/server/ui/WorkerResource.java index d1718c281014..aa7bef9696de 100644 --- a/core/trino-main/src/main/java/io/trino/server/ui/WorkerResource.java +++ b/core/trino-main/src/main/java/io/trino/server/ui/WorkerResource.java @@ -22,9 +22,9 @@ import io.trino.dispatcher.DispatchManager; import io.trino.execution.QueryInfo; import io.trino.execution.TaskId; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; -import io.trino.metadata.NodeState; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; +import io.trino.node.NodeState; import io.trino.security.AccessControl; import io.trino.server.ForWorkerInfo; import io.trino.server.GoneException; @@ -52,8 +52,8 @@ import static com.google.common.net.HttpHeaders.CONTENT_TYPE; import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; import static io.airlift.http.client.Request.Builder.prepareGet; -import static io.trino.metadata.NodeState.ACTIVE; -import static io.trino.metadata.NodeState.INACTIVE; +import static io.trino.node.NodeState.ACTIVE; +import static io.trino.node.NodeState.INACTIVE; import static io.trino.security.AccessControlUtil.checkCanViewQueryOwnedBy; import static io.trino.server.security.ResourceSecurity.AccessType.WEB_UI; import static jakarta.ws.rs.core.MediaType.APPLICATION_JSON; @@ -124,8 +124,8 @@ public Response getThreads( @GET public Response getWorkerList() { - Set activeNodes = nodeManager.getAllNodes().getActiveNodes(); - Set inactiveNodes = nodeManager.getAllNodes().getInactiveNodes(); + Set activeNodes = nodeManager.getAllNodes().activeNodes(); + Set inactiveNodes = nodeManager.getAllNodes().inactiveNodes(); Set jsonNodes = new HashSet<>(); for (Node node : activeNodes) { JsonNodeInfo jsonNode = new JsonNodeInfo(node.getNodeIdentifier(), node.getHostAndPort().getHostText(), node.getVersion(), node.isCoordinator(), ACTIVE.toString().toLowerCase(Locale.ENGLISH)); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/NodePartitionMap.java b/core/trino-main/src/main/java/io/trino/sql/planner/NodePartitionMap.java index 5077181782e4..4ebacf429453 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/NodePartitionMap.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/NodePartitionMap.java @@ -15,8 +15,8 @@ import com.google.common.collect.ImmutableList; import io.trino.execution.scheduler.BucketNodeMap; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import java.util.List; import java.util.function.ToIntFunction; diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/NodePartitioningManager.java b/core/trino-main/src/main/java/io/trino/sql/planner/NodePartitioningManager.java index 54e31c945270..4a96c9169352 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/NodePartitioningManager.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/NodePartitioningManager.java @@ -23,8 +23,8 @@ import io.trino.execution.scheduler.BucketNodeMap; import io.trino.execution.scheduler.NodeScheduler; import io.trino.execution.scheduler.NodeSelector; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.operator.RetryPolicy; import io.trino.spi.connector.CatalogHandle; import io.trino.spi.connector.ConnectorBucketNodeMap; diff --git a/core/trino-main/src/main/java/io/trino/testing/PlanTester.java b/core/trino-main/src/main/java/io/trino/testing/PlanTester.java index 654b40d0cf8c..57cbbc0b8a6a 100644 --- a/core/trino-main/src/main/java/io/trino/testing/PlanTester.java +++ b/core/trino-main/src/main/java/io/trino/testing/PlanTester.java @@ -87,10 +87,8 @@ import io.trino.metadata.FunctionManager; import io.trino.metadata.GlobalFunctionCatalog; import io.trino.metadata.HandleResolver; -import io.trino.metadata.InMemoryNodeManager; import io.trino.metadata.InternalBlockEncodingSerde; import io.trino.metadata.InternalFunctionBundle; -import io.trino.metadata.InternalNodeManager; import io.trino.metadata.LanguageFunctionEngineManager; import io.trino.metadata.LanguageFunctionManager; import io.trino.metadata.MaterializedViewPropertyManager; @@ -108,6 +106,8 @@ import io.trino.metadata.TablePropertyManager; import io.trino.metadata.TypeRegistry; import io.trino.metadata.ViewPropertyManager; +import io.trino.node.InternalNodeManager; +import io.trino.node.TestingInternalNodeManager; import io.trino.operator.Driver; import io.trino.operator.DriverContext; import io.trino.operator.DriverFactory; @@ -251,6 +251,7 @@ import static io.trino.execution.ParameterExtractor.bindParameters; import static io.trino.execution.querystats.PlanOptimizersStatsCollector.createPlanOptimizersStatsCollector; import static io.trino.execution.warnings.WarningCollector.NOOP; +import static io.trino.node.TestingInternalNodeManager.CURRENT_NODE; import static io.trino.spi.connector.Constraint.alwaysTrue; import static io.trino.spi.connector.DynamicFilter.EMPTY; import static io.trino.spiller.PartitioningSpillerFactory.unsupportedPartitioningSpillerFactory; @@ -343,7 +344,7 @@ private PlanTester(Session defaultSession, int nodeCountForStats) this.typeOperators = new TypeOperators(); this.blockTypeOperators = new BlockTypeOperators(typeOperators); this.sqlParser = new SqlParser(); - this.nodeManager = new InMemoryNodeManager(); + this.nodeManager = TestingInternalNodeManager.createDefault(); PageSorter pageSorter = new PagesIndexPageSorter(new PagesIndex.TestingFactory(false)); NodeSchedulerConfig nodeSchedulerConfig = new NodeSchedulerConfig().setIncludeCoordinator(true); this.optimizerConfig = new OptimizerConfig(); @@ -366,7 +367,7 @@ private PlanTester(Session defaultSession, int nodeCountForStats) () -> getPlannerContext().getMetadata(), () -> getPlannerContext().getTypeManager(), () -> getPlannerContext().getFunctionManager()); - globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, blockTypeOperators, nodeManager.getCurrentNode().getNodeVersion())); + globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, blockTypeOperators, CURRENT_NODE.getNodeVersion())); TestingGroupProviderManager groupProvider = new TestingGroupProviderManager(); LanguageFunctionManager languageFunctionManager = new LanguageFunctionManager( sqlParser, @@ -387,7 +388,7 @@ private PlanTester(Session defaultSession, int nodeCountForStats) this.joinCompiler = new JoinCompiler(typeOperators); this.hashStrategyCompiler = new FlatHashStrategyCompiler(typeOperators); PageIndexerFactory pageIndexerFactory = new GroupByHashPageIndexerFactory(hashStrategyCompiler); - EventListenerManager eventListenerManager = new EventListenerManager(new EventListenerConfig(), secretsResolver, noop(), tracer, nodeManager.getCurrentNode().getNodeVersion()); + EventListenerManager eventListenerManager = new EventListenerManager(new EventListenerConfig(), secretsResolver, noop(), tracer, CURRENT_NODE.getNodeVersion()); this.accessControl = new TestingAccessControlManager(transactionManager, eventListenerManager, secretsResolver); accessControl.loadSystemAccessControl(AllowAllSystemAccessControl.NAME, ImmutableMap.of()); @@ -395,6 +396,7 @@ private PlanTester(Session defaultSession, int nodeCountForStats) catalogFactory.setCatalogFactory(new DefaultCatalogFactory( metadata, accessControl, + CURRENT_NODE, nodeManager, pageSorter, pageIndexerFactory, @@ -409,7 +411,7 @@ private PlanTester(Session defaultSession, int nodeCountForStats) this.pageSourceManager = new PageSourceManager(createPageSourceProviderFactory(catalogManager)); this.pageSinkManager = new PageSinkManager(createPageSinkProvider(catalogManager)); this.indexManager = new IndexManager(createIndexProvider(catalogManager)); - NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, nodeSchedulerConfig, new NodeTaskMap(finalizerService))); + NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(CURRENT_NODE, nodeManager, nodeSchedulerConfig, new NodeTaskMap(finalizerService))); this.sessionPropertyManager = createSessionPropertyManager(catalogManager, taskManagerConfig, optimizerConfig); this.nodePartitioningManager = new NodePartitioningManager(nodeScheduler, createNodePartitioningProvider(catalogManager)); this.partitionFunctionProvider = new PartitionFunctionProvider(typeOperators, createNodePartitioningProvider(catalogManager)); @@ -782,7 +784,7 @@ private List createDrivers(Session session, Plan plan, OutputFactory out typeOperators, tableExecuteContextManager, exchangeManagerRegistry, - nodeManager.getCurrentNode().getNodeVersion(), + CURRENT_NODE.getNodeVersion(), new CompilerConfig()); // plan query diff --git a/core/trino-main/src/main/java/io/trino/testing/TestingConnectorContext.java b/core/trino-main/src/main/java/io/trino/testing/TestingConnectorContext.java index d26c2da2e2b9..943b15339f6f 100644 --- a/core/trino-main/src/main/java/io/trino/testing/TestingConnectorContext.java +++ b/core/trino-main/src/main/java/io/trino/testing/TestingConnectorContext.java @@ -16,8 +16,7 @@ import io.airlift.tracing.Tracing; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.trace.Tracer; -import io.trino.connector.DefaultNodeManager; -import io.trino.metadata.InMemoryNodeManager; +import io.trino.client.NodeVersion; import io.trino.operator.FlatHashStrategyCompiler; import io.trino.operator.GroupByHashPageIndexerFactory; import io.trino.operator.PagesIndex; @@ -40,16 +39,10 @@ public final class TestingConnectorContext implements ConnectorContext { - private final NodeManager nodeManager; - private final VersionEmbedder versionEmbedder = new EmbedVersion("testversion"); + private final NodeManager nodeManager = TestingNodeManager.create(); + private final VersionEmbedder versionEmbedder = new EmbedVersion(NodeVersion.UNKNOWN); private final PageSorter pageSorter = new PagesIndexPageSorter(new PagesIndex.TestingFactory(false)); - private final PageIndexerFactory pageIndexerFactory; - - public TestingConnectorContext() - { - pageIndexerFactory = new GroupByHashPageIndexerFactory(new FlatHashStrategyCompiler(new TypeOperators())); - nodeManager = new DefaultNodeManager(new InMemoryNodeManager(), true); - } + private final PageIndexerFactory pageIndexerFactory = new GroupByHashPageIndexerFactory(new FlatHashStrategyCompiler(new TypeOperators())); @Override public CatalogHandle getCatalogHandle() diff --git a/core/trino-main/src/main/java/io/trino/testing/TestingNodeManager.java b/core/trino-main/src/main/java/io/trino/testing/TestingNodeManager.java index 69d8c10ef742..3fe20b13df86 100644 --- a/core/trino-main/src/main/java/io/trino/testing/TestingNodeManager.java +++ b/core/trino-main/src/main/java/io/trino/testing/TestingNodeManager.java @@ -13,25 +13,21 @@ */ package io.trino.testing; -import io.trino.client.NodeVersion; -import io.trino.metadata.InternalNode; import io.trino.spi.Node; import io.trino.spi.NodeManager; -import java.net.URI; import java.util.Collection; import java.util.HashSet; import java.util.Set; import java.util.concurrent.CopyOnWriteArraySet; import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.trino.node.TestingInternalNodeManager.CURRENT_NODE; import static java.util.Objects.requireNonNull; public class TestingNodeManager implements NodeManager { - public static final InternalNode DEFAULT_CURRENT_NODE = new InternalNode("local", URI.create("local://127.0.0.1"), NodeVersion.UNKNOWN, true); - private final Node localNode; private final Set nodes = new CopyOnWriteArraySet<>(); private final boolean scheduleOnCoordinator; @@ -89,7 +85,7 @@ public static Builder builder() public static class Builder { - private Node localNode = DEFAULT_CURRENT_NODE; + private Node localNode = CURRENT_NODE; private final Set otherNodes = new HashSet<>(); private boolean scheduleOnCoordinator = true; diff --git a/core/trino-main/src/test/java/io/trino/connector/TestingLocalCatalogPruneTask.java b/core/trino-main/src/test/java/io/trino/connector/TestingLocalCatalogPruneTask.java index ec48bd497d4e..1500204ead33 100644 --- a/core/trino-main/src/test/java/io/trino/connector/TestingLocalCatalogPruneTask.java +++ b/core/trino-main/src/test/java/io/trino/connector/TestingLocalCatalogPruneTask.java @@ -13,23 +13,21 @@ */ package io.trino.connector; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import com.google.common.util.concurrent.ListenableFuture; -import io.airlift.discovery.client.ServiceDescriptor; -import io.airlift.discovery.client.ServiceSelector; import io.airlift.http.client.testing.TestingHttpClient; import io.airlift.node.NodeInfo; +import io.trino.client.NodeVersion; import io.trino.execution.SqlTaskManager; import io.trino.metadata.CatalogManager; -import io.trino.server.InternalCommunicationConfig; +import io.trino.node.InternalNode; +import io.trino.node.TestingInternalNodeManager; import io.trino.spi.connector.CatalogHandle; import io.trino.transaction.TransactionManager; +import java.net.URI; import java.util.List; import java.util.Set; -import static com.google.common.util.concurrent.Futures.immediateFuture; import static java.util.Objects.requireNonNull; public class TestingLocalCatalogPruneTask @@ -49,42 +47,17 @@ public TestingLocalCatalogPruneTask( transactionManager, catalogManager, connectorServicesProvider, - nodeInfo, - new ServiceSelector() { - @Override - public String getType() - { - throw new UnsupportedOperationException("No services to select"); - } - - @Override - public String getPool() - { - throw new UnsupportedOperationException("No pool"); - } - - @Override - public List selectAllServices() - { - return ImmutableList.of(); - } - - @Override - public ListenableFuture> refresh() - { - return immediateFuture(ImmutableList.of()); - } - }, + new InternalNode(nodeInfo.getNodeId(), URI.create("https://example.com"), new NodeVersion("test"), false), + TestingInternalNodeManager.createDefault(), new TestingHttpClient(request -> { - throw new UnsupportedOperationException("Testing Locl Catalog Prune Task does not make http calls"); + throw new UnsupportedOperationException("Testing Local Catalog Prune Task does not make http calls"); }), - catalogPruneTaskConfig, - new InternalCommunicationConfig()); + catalogPruneTaskConfig); this.sqlTaskManagerToPrune = requireNonNull(sqlTaskManagerToPrune, "sqlTaskManagerToPrune is null"); } @Override - void pruneWorkerCatalogs(Set online, List activeCatalogs) + void pruneWorkerCatalogs(Set online, List activeCatalogs) { sqlTaskManagerToPrune.pruneCatalogs(ImmutableSet.copyOf(activeCatalogs)); } diff --git a/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java b/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java index 943708db4462..1d48c7949524 100644 --- a/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java +++ b/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java @@ -15,7 +15,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import io.airlift.configuration.secrets.SecretsResolver; @@ -44,11 +43,11 @@ import io.trino.execution.warnings.WarningCollector; import io.trino.metadata.FunctionManager; import io.trino.metadata.GlobalFunctionCatalog; -import io.trino.metadata.InMemoryNodeManager; -import io.trino.metadata.InternalNodeManager; import io.trino.metadata.LanguageFunctionProvider; import io.trino.metadata.Metadata; import io.trino.metadata.SessionPropertyManager; +import io.trino.node.InternalNodeManager; +import io.trino.node.TestingInternalNodeManager; import io.trino.operator.OperatorStats; import io.trino.plugin.base.security.AllowAllSystemAccessControl; import io.trino.plugin.base.security.DefaultSystemAccessControl; @@ -158,7 +157,7 @@ public void testSubmittedForDispatchedQuery() queryStateMachine, Futures.immediateFuture(dataDefinitionExecution), queryMonitor, - new TestClusterSizeMonitor(new InMemoryNodeManager(ImmutableSet.of()), new NodeSchedulerConfig()), + new TestClusterSizeMonitor(TestingInternalNodeManager.createDefault(), new NodeSchedulerConfig()), executor, queryExecution -> dataDefinitionExecution.start()); queryStateMachine.addStateChangeListener(state -> { diff --git a/core/trino-main/src/test/java/io/trino/execution/BaseTestSqlTaskManager.java b/core/trino-main/src/test/java/io/trino/execution/BaseTestSqlTaskManager.java index fd66efc8dd25..89276e430227 100644 --- a/core/trino-main/src/test/java/io/trino/execution/BaseTestSqlTaskManager.java +++ b/core/trino-main/src/test/java/io/trino/execution/BaseTestSqlTaskManager.java @@ -39,9 +39,9 @@ import io.trino.memory.NodeMemoryConfig; import io.trino.memory.QueryContext; import io.trino.memory.context.LocalMemoryContext; -import io.trino.metadata.InternalNode; import io.trino.metadata.LanguageFunctionEngineManager; import io.trino.metadata.WorkerLanguageFunctionProvider; +import io.trino.node.InternalNode; import io.trino.operator.DirectExchangeClient; import io.trino.operator.DirectExchangeClientSupplier; import io.trino.operator.RetryPolicy; diff --git a/core/trino-main/src/test/java/io/trino/execution/BenchmarkNodeScheduler.java b/core/trino-main/src/test/java/io/trino/execution/BenchmarkNodeScheduler.java index 143c7bc8cf48..1cf90b0b3bd2 100644 --- a/core/trino-main/src/test/java/io/trino/execution/BenchmarkNodeScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/BenchmarkNodeScheduler.java @@ -31,10 +31,10 @@ import io.trino.execution.scheduler.TopologyAwareNodeSelectorFactory; import io.trino.execution.scheduler.UniformNodeSelectorFactory; import io.trino.jmh.Benchmarks; -import io.trino.metadata.InMemoryNodeManager; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; import io.trino.metadata.Split; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; +import io.trino.node.TestingInternalNodeManager; import io.trino.spi.HostAddress; import io.trino.spi.connector.ConnectorSplit; import io.trino.sql.planner.plan.PlanNodeId; @@ -70,6 +70,7 @@ import static io.airlift.slice.SizeOf.estimatedSizeOf; import static io.airlift.slice.SizeOf.instanceSize; import static io.trino.SystemSessionProperties.MAX_UNACKNOWLEDGED_SPLITS_PER_TASK; +import static io.trino.node.TestingInternalNodeManager.CURRENT_NODE; import static io.trino.testing.TestingHandles.TEST_CATALOG_HANDLE; import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.Executors.newScheduledThreadPool; @@ -167,7 +168,7 @@ public void setup() splits.add(new Split(TEST_CATALOG_HANDLE, new TestSplitRemote(ThreadLocalRandom.current().nextInt(DATA_NODES)))); } - NodeScheduler nodeScheduler = new NodeScheduler(getNodeSelectorFactory(new InMemoryNodeManager(), nodeTaskMap)); + NodeScheduler nodeScheduler = new NodeScheduler(getNodeSelectorFactory(nodeTaskMap)); Session session = TestingSession.testSessionBuilder() .setSystemProperty(MAX_UNACKNOWLEDGED_SPLITS_PER_TASK, Integer.toString(Integer.MAX_VALUE)) .build(); @@ -189,16 +190,17 @@ private NodeSchedulerConfig getNodeSchedulerConfig() .setMinPendingSplitsPerTask(MAX_PENDING_SPLITS_PER_TASK_PER_NODE); } - private NodeSelectorFactory getNodeSelectorFactory(InternalNodeManager nodeManager, NodeTaskMap nodeTaskMap) + private NodeSelectorFactory getNodeSelectorFactory(NodeTaskMap nodeTaskMap) { + InternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(); NodeSchedulerConfig nodeSchedulerConfig = getNodeSchedulerConfig(); switch (policy) { case "uniform": - return new UniformNodeSelectorFactory(nodeManager, nodeSchedulerConfig, nodeTaskMap); + return new UniformNodeSelectorFactory(CURRENT_NODE, nodeManager, nodeSchedulerConfig, nodeTaskMap); case "topology": - return new TopologyAwareNodeSelectorFactory(new FlatNetworkTopology(), nodeManager, nodeSchedulerConfig, nodeTaskMap, new TopologyAwareNodeSelectorConfig()); + return new TopologyAwareNodeSelectorFactory(new FlatNetworkTopology(), CURRENT_NODE, nodeManager, nodeSchedulerConfig, nodeTaskMap, new TopologyAwareNodeSelectorConfig()); case "benchmark": - return new TopologyAwareNodeSelectorFactory(new BenchmarkNetworkTopology(), nodeManager, nodeSchedulerConfig, nodeTaskMap, getBenchmarkNetworkTopologyConfig()); + return new TopologyAwareNodeSelectorFactory(new BenchmarkNetworkTopology(), CURRENT_NODE, nodeManager, nodeSchedulerConfig, nodeTaskMap, getBenchmarkNetworkTopologyConfig()); default: throw new IllegalStateException(); } diff --git a/core/trino-main/src/test/java/io/trino/execution/MockRemoteTaskFactory.java b/core/trino-main/src/test/java/io/trino/execution/MockRemoteTaskFactory.java index adc7371241ab..017cccf5bbc6 100644 --- a/core/trino-main/src/test/java/io/trino/execution/MockRemoteTaskFactory.java +++ b/core/trino-main/src/test/java/io/trino/execution/MockRemoteTaskFactory.java @@ -41,8 +41,8 @@ import io.trino.memory.MemoryPool; import io.trino.memory.QueryContext; import io.trino.memory.context.SimpleLocalMemoryContext; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.operator.TaskContext; import io.trino.operator.TaskStats; import io.trino.spi.predicate.TupleDomain; diff --git a/core/trino-main/src/test/java/io/trino/execution/TestNodeScheduler.java b/core/trino-main/src/test/java/io/trino/execution/TestNodeScheduler.java index 82a900821ff8..d354a1b6c16e 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestNodeScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestNodeScheduler.java @@ -33,10 +33,10 @@ import io.trino.execution.scheduler.TopologyAwareNodeSelectorConfig; import io.trino.execution.scheduler.TopologyAwareNodeSelectorFactory; import io.trino.execution.scheduler.UniformNodeSelectorFactory; -import io.trino.metadata.InMemoryNodeManager; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; import io.trino.metadata.Split; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; +import io.trino.node.TestingInternalNodeManager; import io.trino.spi.HostAddress; import io.trino.spi.SplitWeight; import io.trino.spi.connector.ConnectorSplit; @@ -70,7 +70,8 @@ import static io.airlift.concurrent.Threads.daemonThreadsNamed; import static io.airlift.slice.SizeOf.estimatedSizeOf; import static io.airlift.slice.SizeOf.instanceSize; -import static io.trino.metadata.NodeState.ACTIVE; +import static io.trino.node.NodeState.ACTIVE; +import static io.trino.node.TestingInternalNodeManager.CURRENT_NODE; import static io.trino.spi.StandardErrorCode.NO_NODES_AVAILABLE; import static io.trino.testing.TestingHandles.TEST_CATALOG_HANDLE; import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; @@ -90,7 +91,7 @@ public class TestNodeScheduler { private FinalizerService finalizerService; private NodeTaskMap nodeTaskMap; - private InMemoryNodeManager nodeManager; + private TestingInternalNodeManager nodeManager; private NodeSchedulerConfig nodeSchedulerConfig; private NodeScheduler nodeScheduler; private NodeSelector nodeSelector; @@ -105,7 +106,7 @@ public void setUp() session = TestingSession.testSessionBuilder().build(); finalizerService = new FinalizerService(); nodeTaskMap = new NodeTaskMap(finalizerService); - nodeManager = new InMemoryNodeManager(); + nodeManager = TestingInternalNodeManager.createDefault(); nodeSchedulerConfig = new NodeSchedulerConfig() .setMaxSplitsPerNode(20) @@ -113,7 +114,7 @@ public void setUp() .setMaxAdjustedPendingSplitsWeightPerTask(100) .setIncludeCoordinator(false); - nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, nodeSchedulerConfig, nodeTaskMap)); + nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(CURRENT_NODE, nodeManager, nodeSchedulerConfig, nodeTaskMap)); // contents of taskMap indicate the node-task map for the current stage taskMap = new HashMap<>(); nodeSelector = nodeScheduler.createNodeSelector(session); @@ -169,7 +170,7 @@ public void testScheduleLocal() public void testTopologyAwareScheduling() { NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService); - InternalNodeManager nodeManager = new InMemoryNodeManager( + InternalNodeManager nodeManager = TestingInternalNodeManager.createDefault( new InternalNode("node1", URI.create("http://host1.rack1:11"), NodeVersion.UNKNOWN, false), new InternalNode("node2", URI.create("http://host2.rack1:12"), NodeVersion.UNKNOWN, false), new InternalNode("node3", URI.create("http://host3.rack2:13"), NodeVersion.UNKNOWN, false)); @@ -182,7 +183,13 @@ public void testTopologyAwareScheduling() .setMinPendingSplitsPerTask(20); TestNetworkTopology topology = new TestNetworkTopology(); - NodeSelectorFactory nodeSelectorFactory = new TopologyAwareNodeSelectorFactory(topology, nodeManager, nodeSchedulerConfig, nodeTaskMap, getNetworkTopologyConfig()); + NodeSelectorFactory nodeSelectorFactory = new TopologyAwareNodeSelectorFactory( + topology, + CURRENT_NODE, + nodeManager, + nodeSchedulerConfig, + nodeTaskMap, + getNetworkTopologyConfig()); NodeScheduler nodeScheduler = new NodeScheduler(nodeSelectorFactory); NodeSelector nodeSelector = nodeScheduler.createNodeSelector(session); @@ -595,12 +602,17 @@ public void testMaxUnacknowledgedSplitsPerTask() @Timeout(60) public void testTopologyAwareFailover() { - nodeManager = new InMemoryNodeManager( + nodeManager = TestingInternalNodeManager.createDefault( new InternalNode("node1", URI.create("http://host1.rack1:11"), NodeVersion.UNKNOWN, false), new InternalNode("node2", URI.create("http://host2.rack1:12"), NodeVersion.UNKNOWN, false), new InternalNode("node3", URI.create("http://host3.rack2:13"), NodeVersion.UNKNOWN, false)); NodeSelectorFactory nodeSelectorFactory = new TopologyAwareNodeSelectorFactory( - new TestNetworkTopology(), nodeManager, nodeSchedulerConfig, nodeTaskMap, getNetworkTopologyConfig()); + new TestNetworkTopology(), + CURRENT_NODE, + nodeManager, + nodeSchedulerConfig, + nodeTaskMap, + getNetworkTopologyConfig()); NodeScheduler nodeScheduler = new NodeScheduler(nodeSelectorFactory); NodeSelector nodeSelector = nodeScheduler.createNodeSelector(session); diff --git a/core/trino-main/src/test/java/io/trino/execution/TestSqlStage.java b/core/trino-main/src/test/java/io/trino/execution/TestSqlStage.java index ac0517a2e20e..8a9ff1e2a05c 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestSqlStage.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestSqlStage.java @@ -23,8 +23,8 @@ import io.trino.cost.StatsAndCosts; import io.trino.execution.buffer.PipelinedOutputBuffers; import io.trino.execution.scheduler.SplitSchedulerStats; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.operator.RetryPolicy; import io.trino.spi.QueryId; import io.trino.sql.planner.Partitioning; diff --git a/core/trino-main/src/test/java/io/trino/execution/TestingRemoteTaskFactory.java b/core/trino-main/src/test/java/io/trino/execution/TestingRemoteTaskFactory.java index 5244b5d31a01..1662160032be 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestingRemoteTaskFactory.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestingRemoteTaskFactory.java @@ -33,8 +33,8 @@ import io.trino.execution.buffer.OutputBufferStatus; import io.trino.execution.buffer.OutputBuffers; import io.trino.execution.buffer.SpoolingOutputStats; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.operator.TaskStats; import io.trino.plugin.base.metrics.TDigestHistogram; import io.trino.sql.planner.PlanFragment; diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestFixedCountScheduler.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestFixedCountScheduler.java index c2d8d3e14fb0..fdac0e2a854b 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestFixedCountScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestFixedCountScheduler.java @@ -20,7 +20,7 @@ import io.trino.execution.RemoteTask; import io.trino.execution.StageId; import io.trino.execution.TaskId; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestMultiSourcePartitionedScheduler.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestMultiSourcePartitionedScheduler.java index a8771672537a..7ed68ef5190b 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestMultiSourcePartitionedScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestMultiSourcePartitionedScheduler.java @@ -30,14 +30,13 @@ import io.trino.execution.StageId; import io.trino.execution.TableExecuteContextManager; import io.trino.execution.TableInfo; -import io.trino.failuredetector.NoOpFailureDetector; import io.trino.metadata.FunctionManager; -import io.trino.metadata.InMemoryNodeManager; -import io.trino.metadata.InternalNode; -import io.trino.metadata.InternalNodeManager; import io.trino.metadata.Metadata; import io.trino.metadata.QualifiedObjectName; import io.trino.metadata.TableHandle; +import io.trino.node.InternalNode; +import io.trino.node.InternalNodeManager; +import io.trino.node.TestingInternalNodeManager; import io.trino.operator.RetryPolicy; import io.trino.server.DynamicFilterService; import io.trino.spi.QueryId; @@ -100,6 +99,7 @@ import static io.trino.execution.scheduler.StageExecution.State.SCHEDULING; import static io.trino.metadata.FunctionManager.createTestingFunctionManager; import static io.trino.metadata.TestMetadataManager.createTestMetadataManager; +import static io.trino.node.TestingInternalNodeManager.CURRENT_NODE; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.sql.DynamicFilters.createDynamicFilterExpression; @@ -132,7 +132,7 @@ public class TestMultiSourcePartitionedScheduler private final ExecutorService queryExecutor = newCachedThreadPool(daemonThreadsNamed("stageExecutor-%s")); private final ScheduledExecutorService scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed("stageScheduledExecutor-%s")); - private final InMemoryNodeManager nodeManager = new InMemoryNodeManager(); + private final TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(); private final FinalizerService finalizerService = new FinalizerService(); private final Metadata metadata = createTestMetadataManager(); private final FunctionManager functionManager = createTestingFunctionManager(); @@ -275,7 +275,7 @@ public void testScheduleSplitsTasksAreFull() public void testBalancedSplitAssignment() { // use private node manager so we can add a node later - InMemoryNodeManager nodeManager = new InMemoryNodeManager( + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault( new InternalNode("other1", URI.create("http://127.0.0.1:11"), NodeVersion.UNKNOWN, false), new InternalNode("other2", URI.create("http://127.0.0.1:12"), NodeVersion.UNKNOWN, false), new InternalNode("other3", URI.create("http://127.0.0.1:13"), NodeVersion.UNKNOWN, false)); @@ -422,7 +422,7 @@ public void testDynamicFiltersUnblockedOnBlockedBuildSource() public void testNoNewTaskScheduledWhenChildStageBufferIsOverUtilized() { NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService); - InMemoryNodeManager nodeManager = new InMemoryNodeManager( + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault( new InternalNode("other1", URI.create("http://127.0.0.1:11"), NodeVersion.UNKNOWN, false), new InternalNode("other2", URI.create("http://127.0.0.1:12"), NodeVersion.UNKNOWN, false), new InternalNode("other3", URI.create("http://127.0.0.1:13"), NodeVersion.UNKNOWN, false)); @@ -601,7 +601,7 @@ private SplitPlacementPolicy createSplitPlacementPolicies(Session session, Stage .setMaxSplitsPerNode(100) .setMinPendingSplitsPerTask(0) .setSplitsBalancingPolicy(STAGE); - NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, nodeSchedulerConfig, nodeTaskMap, new Duration(0, SECONDS))); + NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(CURRENT_NODE, nodeManager, nodeSchedulerConfig, nodeTaskMap, new Duration(0, SECONDS))); return new DynamicSplitPlacementPolicy(nodeScheduler.createNodeSelector(session), stage::getAllTasks); } @@ -632,7 +632,7 @@ TABLE_SCAN_2_NODE_ID, new TableInfo(Optional.of("test"), new QualifiedObjectName stage, outputBuffers.buildOrThrow(), TaskLifecycleListener.NO_OP, - new NoOpFailureDetector(), + TestingInternalNodeManager.createDefault(), queryExecutor, Optional.of(new int[] {0}), OptionalInt.empty(), diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestScaledWriterScheduler.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestScaledWriterScheduler.java index 1d8c2f50419a..0ca4ce417c79 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestScaledWriterScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestScaledWriterScheduler.java @@ -31,9 +31,9 @@ import io.trino.execution.TaskState; import io.trino.execution.TaskStatus; import io.trino.execution.buffer.OutputBufferStatus; -import io.trino.metadata.InMemoryNodeManager; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; +import io.trino.node.TestingInternalNodeManager; import io.trino.spi.metrics.Metrics; import io.trino.spi.predicate.TupleDomain; import io.trino.sql.planner.Partitioning; @@ -56,6 +56,7 @@ import static io.airlift.concurrent.Threads.threadsNamed; import static io.trino.execution.TestingRemoteTaskFactory.TestingRemoteTask; +import static io.trino.node.TestingInternalNodeManager.CURRENT_NODE; import static io.trino.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION; import static io.trino.testing.TestingHandles.TEST_TABLE_HANDLE; @@ -206,7 +207,8 @@ private ScaledWriterScheduler buildScaledWriterScheduler(AtomicReference { NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService); - InMemoryNodeManager nodeManager = new InMemoryNodeManager(); - NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, new NodeSchedulerConfig().setIncludeCoordinator(false), nodeTaskMap)); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(); + NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(CURRENT_NODE, nodeManager, new NodeSchedulerConfig().setIncludeCoordinator(false), nodeTaskMap)); PlanFragment plan = createFragment(); StageExecution stage = createStageExecution(plan, nodeTaskMap); @@ -386,7 +386,7 @@ public void testNoNodes() public void testWorkerBalancedSplitAssignment() { // use private node manager so we can add a node later - InMemoryNodeManager nodeManager = new InMemoryNodeManager( + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault( new InternalNode("other1", URI.create("http://127.0.0.1:11"), NodeVersion.UNKNOWN, false), new InternalNode("other2", URI.create("http://127.0.0.1:12"), NodeVersion.UNKNOWN, false), new InternalNode("other3", URI.create("http://127.0.0.1:13"), NodeVersion.UNKNOWN, false)); @@ -432,7 +432,7 @@ public void testWorkerBalancedSplitAssignment() public void testStageBalancedSplitAssignment() { // use private node manager so we can add a node later - InMemoryNodeManager nodeManager = new InMemoryNodeManager( + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault( new InternalNode("other1", URI.create("http://127.0.0.1:11"), NodeVersion.UNKNOWN, false), new InternalNode("other2", URI.create("http://127.0.0.1:12"), NodeVersion.UNKNOWN, false), new InternalNode("other3", URI.create("http://127.0.0.1:13"), NodeVersion.UNKNOWN, false)); @@ -497,11 +497,11 @@ public void testNewTaskScheduledWhenChildStageBufferIsUnderutilized() { NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService); // use private node manager so we can add a node later - InMemoryNodeManager nodeManager = new InMemoryNodeManager( + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault( new InternalNode("other1", URI.create("http://127.0.0.1:11"), NodeVersion.UNKNOWN, false), new InternalNode("other2", URI.create("http://127.0.0.1:12"), NodeVersion.UNKNOWN, false), new InternalNode("other3", URI.create("http://127.0.0.1:13"), NodeVersion.UNKNOWN, false)); - NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, new NodeSchedulerConfig().setIncludeCoordinator(false), nodeTaskMap, new Duration(0, SECONDS))); + NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(CURRENT_NODE, nodeManager, new NodeSchedulerConfig().setIncludeCoordinator(false), nodeTaskMap, new Duration(0, SECONDS))); PlanFragment plan = createFragment(); StageExecution stage = createStageExecution(plan, nodeTaskMap); @@ -540,11 +540,11 @@ public void testNoNewTaskScheduledWhenChildStageBufferIsOverutilized() { NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService); // use private node manager so we can add a node later - InMemoryNodeManager nodeManager = new InMemoryNodeManager( + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault( new InternalNode("other1", URI.create("http://127.0.0.1:11"), NodeVersion.UNKNOWN, false), new InternalNode("other2", URI.create("http://127.0.0.1:12"), NodeVersion.UNKNOWN, false), new InternalNode("other3", URI.create("http://127.0.0.1:13"), NodeVersion.UNKNOWN, false)); - NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, new NodeSchedulerConfig().setIncludeCoordinator(false), nodeTaskMap, new Duration(0, SECONDS))); + NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(CURRENT_NODE, nodeManager, new NodeSchedulerConfig().setIncludeCoordinator(false), nodeTaskMap, new Duration(0, SECONDS))); PlanFragment plan = createFragment(); StageExecution stage = createStageExecution(plan, nodeTaskMap); @@ -584,7 +584,7 @@ public void testDynamicFiltersUnblockedOnBlockedBuildSource() PlanFragment plan = createFragment(); NodeTaskMap nodeTaskMap = new NodeTaskMap(finalizerService); StageExecution stage = createStageExecution(plan, nodeTaskMap); - NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, new NodeSchedulerConfig().setIncludeCoordinator(false), nodeTaskMap)); + NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(CURRENT_NODE, nodeManager, new NodeSchedulerConfig().setIncludeCoordinator(false), nodeTaskMap)); DynamicFilterService dynamicFilterService = new DynamicFilterService(metadata, functionManager, typeOperators, new DynamicFilterConfig()); dynamicFilterService.registerQuery( QUERY_ID, @@ -659,7 +659,7 @@ private StageScheduler getSourcePartitionedScheduler( .setMaxSplitsPerNode(20) .setMinPendingSplitsPerTask(0) .setSplitsBalancingPolicy(splitsBalancingPolicy); - NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, nodeSchedulerConfig, nodeTaskMap, new Duration(0, SECONDS))); + NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(CURRENT_NODE, nodeManager, nodeSchedulerConfig, nodeTaskMap, new Duration(0, SECONDS))); SplitPlacementPolicy placementPolicy = new DynamicSplitPlacementPolicy(nodeScheduler.createNodeSelector(session), stage::getAllTasks); return newSourcePartitionedSchedulerAsStageScheduler( @@ -776,7 +776,7 @@ private StageExecution createStageExecution(PlanFragment fragment, NodeTaskMap n stage, outputBuffers.buildOrThrow(), TaskLifecycleListener.NO_OP, - new NoOpFailureDetector(), + TestingInternalNodeManager.createDefault(), queryExecutor, Optional.of(new int[] {0}), OptionalInt.empty(), diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestUniformNodeSelector.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestUniformNodeSelector.java index 1dc6e701ce51..810926f0cf7a 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestUniformNodeSelector.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestUniformNodeSelector.java @@ -28,9 +28,9 @@ import io.trino.execution.RemoteTask; import io.trino.execution.StageId; import io.trino.execution.TaskId; -import io.trino.metadata.InMemoryNodeManager; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; +import io.trino.node.TestingInternalNodeManager; import io.trino.spi.HostAddress; import io.trino.sql.planner.plan.PlanNodeId; import io.trino.testing.TestingSession; @@ -55,7 +55,8 @@ import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.airlift.concurrent.Threads.daemonThreadsNamed; -import static io.trino.metadata.NodeState.ACTIVE; +import static io.trino.node.NodeState.ACTIVE; +import static io.trino.node.TestingInternalNodeManager.CURRENT_NODE; import static io.trino.testing.TestingHandles.TEST_CATALOG_HANDLE; import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.Executors.newScheduledThreadPool; @@ -71,7 +72,7 @@ public class TestUniformNodeSelector private final Set splits = new LinkedHashSet<>(); private FinalizerService finalizerService; private NodeTaskMap nodeTaskMap; - private InMemoryNodeManager nodeManager; + private TestingInternalNodeManager nodeManager; private NodeSchedulerConfig nodeSchedulerConfig; private NodeScheduler nodeScheduler; private NodeSelector nodeSelector; @@ -86,7 +87,7 @@ public void setUp() session = TestingSession.testSessionBuilder().build(); finalizerService = new FinalizerService(); nodeTaskMap = new NodeTaskMap(finalizerService); - nodeManager = new InMemoryNodeManager(); + nodeManager = TestingInternalNodeManager.createDefault(); nodeManager.addNodes(node1); nodeManager.addNodes(node2); @@ -97,7 +98,7 @@ public void setUp() .setIncludeCoordinator(false); // contents of taskMap indicate the node-task map for the current stage - nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, nodeSchedulerConfig, nodeTaskMap)); + nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(CURRENT_NODE, nodeManager, nodeSchedulerConfig, nodeTaskMap)); taskMap = new HashMap<>(); nodeSelector = nodeScheduler.createNodeSelector(session); remoteTaskExecutor = newCachedThreadPool(daemonThreadsNamed("remoteTaskExecutor-%s")); @@ -127,7 +128,7 @@ public void testQueueSizeAdjustmentScaleDown() UniformNodeSelector.QueueSizeAdjuster queueSizeAdjuster = new UniformNodeSelector.QueueSizeAdjuster(10, 100, ticker); nodeSelector = new UniformNodeSelector( - nodeManager, + CURRENT_NODE, nodeTaskMap, false, () -> createNodeMap(), @@ -299,7 +300,7 @@ public void testFailover() { // Node selector without nodeMap memoization, so removing nodes takes effect immediately: nodeSelector = new UniformNodeSelector( - nodeManager, + CURRENT_NODE, nodeTaskMap, false, () -> createNodeMap(), diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/BenchmarkBinPackingNodeAllocator.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/BenchmarkBinPackingNodeAllocator.java index 7b1c4ccc8b36..377d4290ddd0 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/BenchmarkBinPackingNodeAllocator.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/BenchmarkBinPackingNodeAllocator.java @@ -22,8 +22,8 @@ import io.trino.execution.TaskId; import io.trino.jmh.Benchmarks; import io.trino.memory.MemoryInfo; -import io.trino.metadata.InMemoryNodeManager; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; +import io.trino.node.TestingInternalNodeManager; import io.trino.spi.HostAddress; import io.trino.spi.QueryId; import io.trino.spi.connector.CatalogHandle; @@ -116,7 +116,7 @@ public void setup() nodes.add(new InternalNode(nodeIdentifier, URI.create("local://127.0.0.1:" + (8000 + i)), NodeVersion.UNKNOWN, false)); workerMemoryInfos.put(nodeIdentifier, Optional.of(memoryInfo)); } - InMemoryNodeManager nodeManager = new InMemoryNodeManager(ImmutableSet.copyOf(nodes)); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(ImmutableSet.copyOf(nodes)); nodeAllocatorService = new BinPackingNodeAllocatorService( nodeManager, diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/TestBinPackingNodeAllocator.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/TestBinPackingNodeAllocator.java index e8b1c1055347..abb9c7d337ee 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/TestBinPackingNodeAllocator.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/TestBinPackingNodeAllocator.java @@ -22,8 +22,8 @@ import io.trino.execution.StageId; import io.trino.execution.TaskId; import io.trino.memory.MemoryInfo; -import io.trino.metadata.InMemoryNodeManager; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; +import io.trino.node.TestingInternalNodeManager; import io.trino.spi.HostAddress; import io.trino.spi.QueryId; import io.trino.spi.connector.CatalogHandle; @@ -93,12 +93,12 @@ public class TestBinPackingNodeAllocator private ConcurrentHashMap> workerMemoryInfos; private final TestingTicker ticker = new TestingTicker(); - private void setupNodeAllocatorService(InMemoryNodeManager nodeManager) + private void setupNodeAllocatorService(TestingInternalNodeManager nodeManager) { setupNodeAllocatorService(nodeManager, DataSize.ofBytes(0)); } - private void setupNodeAllocatorService(InMemoryNodeManager nodeManager, DataSize taskRuntimeMemoryEstimationOverhead) + private void setupNodeAllocatorService(TestingInternalNodeManager nodeManager, DataSize taskRuntimeMemoryEstimationOverhead) { shutdownNodeAllocatorService(); // just in case @@ -158,7 +158,7 @@ public void shutdownNodeAllocatorService() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateSimple() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); try (NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1)) { @@ -207,7 +207,7 @@ public void testAllocateSimple() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateMultipleRequesters() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); try (NodeAllocator nodeAllocator1 = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1); @@ -262,7 +262,7 @@ public void testAllocateMultipleRequesters() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateDifferentSizes() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); try (NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1)) { @@ -309,7 +309,7 @@ public void testAllocateDifferentSizes() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateDifferentSizesOpportunisticAcquisition() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); try (NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1)) { @@ -350,7 +350,7 @@ public void testAllocateDifferentSizesOpportunisticAcquisition() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateReleaseBeforeAcquired() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1); setupNodeAllocatorService(nodeManager); try (NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1)) { @@ -380,7 +380,7 @@ public void testAllocateReleaseBeforeAcquired() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testNoMatchingNodeAvailable() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(); setupNodeAllocatorService(nodeManager); try (NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1)) { @@ -427,7 +427,7 @@ public void testNoMatchingNodeAvailable() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testNoMatchingNodeAvailableTimeoutReset() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(); setupNodeAllocatorService(nodeManager); try (NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1)) { @@ -476,7 +476,7 @@ public void testNoMatchingNodeAvailableTimeoutReset() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testRemoveAcquiredNode() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1); setupNodeAllocatorService(nodeManager); try (NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1)) { @@ -495,7 +495,7 @@ public void testRemoveAcquiredNode() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateNodeWithAddressRequirements() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); @@ -524,7 +524,7 @@ public void testAllocateNodeWithAddressRequirements() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateNodeWithAddressRequirementsNoResourcesUseDifferentNode() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); @@ -555,7 +555,7 @@ public void testAllocateNodeWithAddressRequirementsNoResourcesUseDifferentNode() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateNodeWithAddressRequirementsNoResourcesWaitIfRemoteNotAvailable() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); @@ -585,7 +585,7 @@ public void testAllocateNodeWithAddressRequirementsNoResourcesWaitIfRemoteNotAva @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateNodeWithAddressRequirementsNoResourcesInitially() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); @@ -612,7 +612,7 @@ public void testAllocateNodeWithAddressRequirementsNoResourcesInitially() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateNotEnoughRuntimeMemory() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); try (NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1)) { @@ -669,7 +669,7 @@ public void testAllocateNotEnoughRuntimeMemory() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateRuntimeMemoryDiscrepancies() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1); setupNodeAllocatorService(nodeManager); // test when global memory usage on node is greater than per task usage @@ -731,7 +731,7 @@ public void testAllocateRuntimeMemoryDiscrepancies() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testSpaceReservedOnPrimaryNodeIfNoNodeWithEnoughRuntimeMemoryAvailable() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); // test when global memory usage on node is greater than per task usage @@ -770,7 +770,7 @@ public void testSpaceReservedOnPrimaryNodeIfNoNodeWithEnoughRuntimeMemoryAvailab @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateWithRuntimeMemoryEstimateOverhead() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1); setupNodeAllocatorService(nodeManager, DataSize.of(4, GIGABYTE)); // test when global memory usage on node is greater than per task usage @@ -806,7 +806,7 @@ public void testAllocateWithRuntimeMemoryEstimateOverhead() @Test public void testStressAcquireRelease() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1); setupNodeAllocatorService(nodeManager, DataSize.of(4, GIGABYTE)); try (NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1)) { @@ -821,7 +821,7 @@ public void testStressAcquireRelease() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateSpeculative() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); try (NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1)) { @@ -877,7 +877,7 @@ public void testAllocateSpeculative() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testSwitchAcquiredSpeculativeToStandard() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1); setupNodeAllocatorService(nodeManager); try (NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1)) { @@ -903,7 +903,7 @@ public void testSwitchAcquiredSpeculativeToStandard() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testAllocateEagerSpeculative() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); try (NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1)) { @@ -951,7 +951,7 @@ public void testAllocateEagerSpeculative() @Timeout(value = TEST_TIMEOUT, unit = MILLISECONDS) public void testChangeMemoryRequirement() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); try (NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(SESSION_QUERY_1)) { @@ -985,7 +985,7 @@ public void testChangeMemoryRequirement() @Timeout(value = TEST_TIMEOUT + 3000, unit = MILLISECONDS) public void testFailover() { - InMemoryNodeManager nodeManager = new InMemoryNodeManager(NODE_1, NODE_2); + TestingInternalNodeManager nodeManager = TestingInternalNodeManager.createDefault(NODE_1, NODE_2); setupNodeAllocatorService(nodeManager); NodeRequirements node2Flexible = new NodeRequirements(Optional.empty(), Optional.of(NODE_2_ADDRESS), true); NodeRequirements node2Rigid = new NodeRequirements(Optional.empty(), Optional.of(NODE_2_ADDRESS), false); diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/TestExponentialGrowthPartitionMemoryEstimator.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/TestExponentialGrowthPartitionMemoryEstimator.java index 476ad1b42ab4..c5201855c081 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/TestExponentialGrowthPartitionMemoryEstimator.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/TestExponentialGrowthPartitionMemoryEstimator.java @@ -22,7 +22,7 @@ import io.trino.cost.StatsAndCosts; import io.trino.execution.scheduler.faulttolerant.PartitionMemoryEstimator.MemoryRequirements; import io.trino.memory.MemoryInfo; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import io.trino.spi.StandardErrorCode; import io.trino.spi.memory.MemoryPoolInfo; import io.trino.sql.planner.Partitioning; diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/TestHashDistributionSplitAssigner.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/TestHashDistributionSplitAssigner.java index 69c6a1a14e8a..00c6b35f77e6 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/TestHashDistributionSplitAssigner.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/faulttolerant/TestHashDistributionSplitAssigner.java @@ -27,8 +27,8 @@ import io.trino.client.NodeVersion; import io.trino.execution.scheduler.OutputDataSizeEstimate; import io.trino.execution.scheduler.faulttolerant.HashDistributionSplitAssigner.TaskPartition; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.sql.planner.plan.PlanFragmentId; import io.trino.sql.planner.plan.PlanNodeId; import org.junit.jupiter.api.Test; diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/TestPhasedExecutionSchedule.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/TestPhasedExecutionSchedule.java index cf241353afef..8a6097917429 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/TestPhasedExecutionSchedule.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/TestPhasedExecutionSchedule.java @@ -28,8 +28,8 @@ import io.trino.execution.TaskStatus; import io.trino.execution.scheduler.StageExecution; import io.trino.execution.scheduler.TaskLifecycleListener; -import io.trino.metadata.InternalNode; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.server.DynamicFilterService; import io.trino.spi.QueryId; import io.trino.spi.metrics.Metrics; diff --git a/core/trino-main/src/test/java/io/trino/memory/LowMemoryKillerTestingUtils.java b/core/trino-main/src/test/java/io/trino/memory/LowMemoryKillerTestingUtils.java index da7c464a8e58..5c13a132f68b 100644 --- a/core/trino-main/src/test/java/io/trino/memory/LowMemoryKillerTestingUtils.java +++ b/core/trino-main/src/test/java/io/trino/memory/LowMemoryKillerTestingUtils.java @@ -21,7 +21,7 @@ import io.trino.execution.StageId; import io.trino.execution.TaskId; import io.trino.execution.TaskInfo; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import io.trino.operator.RetryPolicy; import io.trino.spi.QueryId; import io.trino.spi.memory.MemoryPoolInfo; diff --git a/core/trino-main/src/test/java/io/trino/metadata/TestDiscoveryNodeManager.java b/core/trino-main/src/test/java/io/trino/metadata/TestDiscoveryNodeManager.java deleted file mode 100644 index cd9adcbb9369..000000000000 --- a/core/trino-main/src/test/java/io/trino/metadata/TestDiscoveryNodeManager.java +++ /dev/null @@ -1,276 +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.metadata; - -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.errorprone.annotations.concurrent.GuardedBy; -import io.airlift.discovery.client.ServiceDescriptor; -import io.airlift.discovery.client.ServiceSelector; -import io.airlift.http.client.HttpClient; -import io.airlift.http.client.testing.TestingHttpClient; -import io.airlift.http.client.testing.TestingResponse; -import io.airlift.node.NodeConfig; -import io.airlift.node.NodeInfo; -import io.trino.client.NodeVersion; -import io.trino.failuredetector.NoOpFailureDetector; -import io.trino.server.InternalCommunicationConfig; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.Timeout; - -import java.net.URI; -import java.util.List; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; - -import static io.airlift.discovery.client.ServiceDescriptor.serviceDescriptor; -import static io.airlift.discovery.client.ServiceSelectorConfig.DEFAULT_POOL; -import static io.airlift.http.client.HttpStatus.OK; -import static io.trino.metadata.NodeState.ACTIVE; -import static io.trino.metadata.NodeState.INACTIVE; -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_METHOD; - -@TestInstance(PER_METHOD) -public class TestDiscoveryNodeManager -{ - private final NodeInfo nodeInfo = new NodeInfo("test"); - private final InternalCommunicationConfig internalCommunicationConfig = new InternalCommunicationConfig(); - private NodeVersion expectedVersion; - private Set activeNodes; - private Set inactiveNodes; - private InternalNode coordinator; - private InternalNode currentNode; - private final TrinoNodeServiceSelector selector = new TrinoNodeServiceSelector(); - private HttpClient testHttpClient; - - @BeforeEach - public void setup() - { - testHttpClient = new TestingHttpClient(input -> new TestingResponse(OK, ArrayListMultimap.create(), ACTIVE.name().getBytes(UTF_8))); - - expectedVersion = new NodeVersion("1"); - coordinator = new InternalNode(UUID.randomUUID().toString(), URI.create("https://192.0.2.8"), expectedVersion, true); - currentNode = new InternalNode(nodeInfo.getNodeId(), URI.create("http://192.0.1.1"), expectedVersion, false); - - activeNodes = ImmutableSet.of( - currentNode, - new InternalNode(UUID.randomUUID().toString(), URI.create("http://192.0.2.1:8080"), expectedVersion, false), - new InternalNode(UUID.randomUUID().toString(), URI.create("http://192.0.2.3"), expectedVersion, false), - coordinator); - inactiveNodes = ImmutableSet.of( - new InternalNode(UUID.randomUUID().toString(), URI.create("https://192.0.3.9"), NodeVersion.UNKNOWN, false), - new InternalNode(UUID.randomUUID().toString(), URI.create("https://192.0.4.9"), new NodeVersion("2"), false)); - - selector.announceNodes(activeNodes, inactiveNodes); - } - - @AfterEach - public void tearDown() - { - testHttpClient.close(); - testHttpClient = null; - } - - @Test - public void testGetAllNodes() - { - DiscoveryNodeManager manager = new DiscoveryNodeManager( - selector, - nodeInfo, - new NoOpFailureDetector(), - expectedVersion, - testHttpClient, - internalCommunicationConfig); - try { - AllNodes allNodes = manager.getAllNodes(); - - Set connectorNodes = manager.getNodes(ACTIVE); - assertThat(connectorNodes).hasSize(4); - assertThat(connectorNodes.stream().anyMatch(InternalNode::isCoordinator)).isTrue(); - - Set activeNodes = allNodes.getActiveNodes(); - assertThat(activeNodes).containsExactlyInAnyOrderElementsOf(this.activeNodes); - - for (InternalNode actual : activeNodes) { - for (InternalNode expected : this.activeNodes) { - assertThat(actual).isNotSameAs(expected); - } - } - - assertThat(activeNodes).containsExactlyInAnyOrderElementsOf(manager.getNodes(ACTIVE)); - - Set inactiveNodes = allNodes.getInactiveNodes(); - assertThat(inactiveNodes).containsExactlyInAnyOrderElementsOf(this.inactiveNodes); - - for (InternalNode actual : inactiveNodes) { - for (InternalNode expected : this.inactiveNodes) { - assertThat(actual).isNotSameAs(expected); - } - } - - assertThat(inactiveNodes).containsExactlyInAnyOrderElementsOf(manager.getNodes(INACTIVE)); - } - finally { - manager.stop(); - } - } - - @Test - public void testGetCurrentNode() - { - NodeInfo nodeInfo = new NodeInfo(new NodeConfig() - .setEnvironment("test") - .setNodeId(currentNode.getNodeIdentifier())); - - DiscoveryNodeManager manager = new DiscoveryNodeManager( - selector, - nodeInfo, - new NoOpFailureDetector(), - expectedVersion, - testHttpClient, - internalCommunicationConfig); - try { - assertThat(manager.getCurrentNode()).isEqualTo(currentNode); - } - finally { - manager.stop(); - } - } - - @Test - public void testGetCoordinators() - { - DiscoveryNodeManager manager = new DiscoveryNodeManager( - selector, - nodeInfo, - new NoOpFailureDetector(), - expectedVersion, - testHttpClient, - internalCommunicationConfig); - try { - assertThat(manager.getCoordinators()).isEqualTo(ImmutableSet.of(coordinator)); - } - finally { - manager.stop(); - } - } - - @SuppressWarnings("ResultOfObjectAllocationIgnored") - @Test - public void testGetCurrentNodeRequired() - { - assertThatThrownBy(() -> new DiscoveryNodeManager( - selector, - new NodeInfo("test"), - new NoOpFailureDetector(), - expectedVersion, - testHttpClient, - internalCommunicationConfig)) - .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("current node not returned"); - } - - @Test - @Timeout(60) - public void testNodeChangeListener() - throws Exception - { - DiscoveryNodeManager manager = new DiscoveryNodeManager( - selector, - nodeInfo, - new NoOpFailureDetector(), - expectedVersion, - testHttpClient, - internalCommunicationConfig); - try { - manager.startPollingNodeStates(); - - BlockingQueue notifications = new ArrayBlockingQueue<>(100); - manager.addNodeChangeListener(notifications::add); - AllNodes allNodes = notifications.take(); - assertThat(allNodes.getActiveNodes()).isEqualTo(activeNodes); - assertThat(allNodes.getInactiveNodes()).isEqualTo(inactiveNodes); - - selector.announceNodes(ImmutableSet.of(currentNode), ImmutableSet.of(coordinator)); - allNodes = notifications.take(); - assertThat(allNodes.getActiveNodes()).isEqualTo(ImmutableSet.of(currentNode, coordinator)); - assertThat(allNodes.getActiveCoordinators()).isEqualTo(ImmutableSet.of(coordinator)); - - selector.announceNodes(activeNodes, inactiveNodes); - allNodes = notifications.take(); - assertThat(allNodes.getActiveNodes()).isEqualTo(activeNodes); - assertThat(allNodes.getInactiveNodes()).isEqualTo(inactiveNodes); - } - finally { - manager.stop(); - } - } - - public static class TrinoNodeServiceSelector - implements ServiceSelector - { - @GuardedBy("this") - private List descriptors = ImmutableList.of(); - - private synchronized void announceNodes(Set activeNodes, Set inactiveNodes) - { - ImmutableList.Builder descriptors = ImmutableList.builder(); - for (InternalNode node : Iterables.concat(activeNodes, inactiveNodes)) { - descriptors.add(serviceDescriptor("trino") - .setNodeId(node.getNodeIdentifier()) - .addProperty("http", node.getInternalUri().toString()) - .addProperty("node_version", node.getNodeVersion().toString()) - .addProperty("coordinator", String.valueOf(node.isCoordinator())) - .build()); - } - - this.descriptors = descriptors.build(); - } - - @Override - public String getType() - { - return "trino"; - } - - @Override - public String getPool() - { - return DEFAULT_POOL; - } - - @Override - public synchronized List selectAllServices() - { - return descriptors; - } - - @Override - public ListenableFuture> refresh() - { - throw new UnsupportedOperationException(); - } - } -} diff --git a/core/trino-main/src/test/java/io/trino/node/TestCoordinatorNodeManager.java b/core/trino-main/src/test/java/io/trino/node/TestCoordinatorNodeManager.java new file mode 100644 index 000000000000..1bc3751a0b4d --- /dev/null +++ b/core/trino-main/src/test/java/io/trino/node/TestCoordinatorNodeManager.java @@ -0,0 +1,272 @@ +/* + * 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.node; + +import com.google.common.collect.ImmutableListMultimap; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import io.airlift.http.client.testing.TestingHttpClient; +import io.airlift.http.client.testing.TestingResponse; +import io.airlift.json.JsonCodec; +import io.airlift.testing.TestingTicker; +import io.airlift.units.Duration; +import io.trino.client.NodeVersion; +import io.trino.server.ServerInfo; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.net.URI; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.stream.Stream; + +import static com.google.common.net.HttpHeaders.CONTENT_TYPE; +import static com.google.common.net.MediaType.JSON_UTF_8; +import static io.airlift.http.client.HttpStatus.OK; +import static io.trino.node.NodeState.ACTIVE; +import static io.trino.node.NodeState.INACTIVE; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.assertj.core.api.Assertions.assertThat; + +class TestCoordinatorNodeManager +{ + private static final JsonCodec SERVER_INFO_JSON_CODEC = JsonCodec.jsonCodec(ServerInfo.class); + private static final String EXPECTED_ENVIRONMENT = "test_1"; + + private final Set activeNodes; + private final Set inactiveNodes; + private final Set invalidNodes; + private final InternalNode coordinator; + private final InternalNode currentNode; + private final TestingHttpClient testHttpClient; + + TestCoordinatorNodeManager() + { + NodeVersion expectedVersion = new NodeVersion("1"); + coordinator = new InternalNode(UUID.randomUUID().toString(), URI.create("https://192.0.2.8"), expectedVersion, true); + currentNode = new InternalNode(UUID.randomUUID().toString(), URI.create("https://192.0.1.1"), expectedVersion, false); + + activeNodes = ImmutableSet.of( + copy(currentNode), + new InternalNode(UUID.randomUUID().toString(), URI.create("https://192.0.2.1:8080"), expectedVersion, false), + new InternalNode(UUID.randomUUID().toString(), URI.create("https://192.0.2.3"), expectedVersion, false), + coordinator); + inactiveNodes = ImmutableSet.of( + new InternalNode(UUID.randomUUID().toString(), URI.create("https://192.0.3.9"), expectedVersion, false), + new InternalNode(UUID.randomUUID().toString(), URI.create("https://192.0.4.9"), expectedVersion, false)); + invalidNodes = ImmutableSet.of( + new InternalNode("bad_version_unknown", URI.create("https://192.0.7.1"), NodeVersion.UNKNOWN, false), + new InternalNode("bad_version_2", URI.create("https://192.0.7.2"), new NodeVersion("2"), false), + new InternalNode("bad_environment", URI.create("https://192.0.7.3"), expectedVersion, false)); + + ImmutableMap.Builder hostsBuilder = ImmutableMap.builder(); + for (InternalNode activeNode : activeNodes) { + hostsBuilder.put(activeNode.getInternalUri().getHost(), toServerInfo(activeNode, ACTIVE, EXPECTED_ENVIRONMENT)); + } + for (InternalNode inactiveNode : inactiveNodes) { + hostsBuilder.put(inactiveNode.getInternalUri().getHost(), toServerInfo(inactiveNode, INACTIVE, EXPECTED_ENVIRONMENT)); + } + for (InternalNode invalidNode : invalidNodes) { + hostsBuilder.put( + invalidNode.getInternalUri().getHost(), + toServerInfo(invalidNode, INACTIVE, invalidNode.getNodeVersion().equals(expectedVersion) ? "bad_environment" : EXPECTED_ENVIRONMENT)); + } + Map allHosts = hostsBuilder.buildOrThrow(); + + testHttpClient = new TestingHttpClient(input -> { + ServerInfo serverInfo = allHosts.get(input.getUri().getHost()); + if (serverInfo == null) { + throw new IllegalArgumentException("Unknown host: " + input.getUri().getHost()); + } + return new TestingResponse( + OK, + ImmutableListMultimap.of(CONTENT_TYPE, JSON_UTF_8.toString()), + SERVER_INFO_JSON_CODEC.toJsonBytes(serverInfo)); + }); + } + + @Test + void testGetAllNodes() + { + CoordinatorNodeManager manager = new CoordinatorNodeManager( + new TestingNodeInventory(), + copy(currentNode), + () -> ACTIVE, + EXPECTED_ENVIRONMENT, + testHttpClient, + new TestingTicker()); + try { + AllNodes allNodes = manager.getAllNodes(); + + Set connectorNodes = manager.getNodes(ACTIVE); + assertThat(connectorNodes).hasSize(4); + assertThat(connectorNodes.stream().anyMatch(InternalNode::isCoordinator)).isTrue(); + + Set activeNodes = allNodes.activeNodes(); + assertThat(activeNodes).containsExactlyInAnyOrderElementsOf(this.activeNodes); + + for (InternalNode actual : activeNodes) { + for (InternalNode expected : this.activeNodes) { + assertThat(actual).isNotSameAs(expected); + } + } + + assertThat(activeNodes).containsExactlyInAnyOrderElementsOf(manager.getNodes(ACTIVE)); + + Set inactiveNodes = allNodes.inactiveNodes(); + assertThat(inactiveNodes).containsExactlyInAnyOrderElementsOf(this.inactiveNodes); + + for (InternalNode actual : inactiveNodes) { + for (InternalNode expected : this.inactiveNodes) { + assertThat(actual).isNotSameAs(expected); + } + } + + assertThat(inactiveNodes).containsExactlyInAnyOrderElementsOf(manager.getNodes(INACTIVE)); + + assertThat(manager.getInvalidNodes()).containsExactlyInAnyOrderElementsOf(invalidNodes); + } + finally { + manager.stop(); + } + } + + @Test + void testGetCoordinators() + { + CoordinatorNodeManager manager = new CoordinatorNodeManager( + new TestingNodeInventory(), + copy(currentNode), + () -> ACTIVE, + EXPECTED_ENVIRONMENT, + testHttpClient, + new TestingTicker()); + try { + assertThat(manager.getCoordinators()).isEqualTo(ImmutableSet.of(coordinator)); + } + finally { + manager.stop(); + } + } + + @Test + @Timeout(60) + void testNodeChangeListener() + throws Exception + { + // initially only the current node is announced + TestingNodeInventory selector = new TestingNodeInventory(); + selector.announceNodes(ImmutableSet.of(currentNode)); + TestingTicker testingTicker = new TestingTicker(); + CoordinatorNodeManager manager = new CoordinatorNodeManager( + selector, + copy(currentNode), + () -> ACTIVE, + EXPECTED_ENVIRONMENT, + testHttpClient, + testingTicker); + try { + BlockingQueue notifications = new ArrayBlockingQueue<>(100); + manager.addNodeChangeListener(notifications::add); + AllNodes allNodes = notifications.take(); + assertThat(manager.getAllNodes()).isSameAs(allNodes); + assertThat(allNodes.activeNodes()).containsExactly(currentNode); + assertThat(allNodes.inactiveNodes()).isEmpty(); + + // announce all nodes + testingTicker.increment(5, SECONDS); + selector.announceNodes(activeNodes, inactiveNodes); + manager.refreshNodes(); + allNodes = notifications.take(); + assertThat(manager.getAllNodes()).isSameAs(allNodes); + assertThat(allNodes.activeNodes()).isEqualTo(activeNodes); + assertThat(allNodes.activeCoordinators()).isEqualTo(ImmutableSet.of(coordinator)); + + // only announce current node and inactive nodes + // node manager tracks all nodes until they have not been seen for a while + testingTicker.increment(30, SECONDS); + selector.announceNodes(ImmutableSet.of(currentNode), inactiveNodes); + manager.refreshNodes(); + allNodes = notifications.take(); + assertThat(manager.getAllNodes()).isSameAs(allNodes); + assertThat(allNodes.activeNodes()).containsExactly(currentNode); + assertThat(allNodes.inactiveNodes()).isEqualTo(inactiveNodes); + } + finally { + manager.stop(); + } + } + + private final class TestingNodeInventory + implements NodeInventory + { + @GuardedBy("this") + private Set nodes = ImmutableSet.of(); + + public TestingNodeInventory() + { + announceNodes(activeNodes, inactiveNodes); + } + + @SafeVarargs + private synchronized void announceNodes(Set... nodeSets) + { + ImmutableSet.Builder descriptors = ImmutableSet.builder(); + Stream.of(nodeSets) + .flatMap(Set::stream) + .map(InternalNode::getInternalUri) + .forEach(descriptors::add); + + // Add the invalid nodes + invalidNodes.stream() + .map(InternalNode::getInternalUri) + .forEach(descriptors::add); + + this.nodes = descriptors.build(); + } + + @Override + public synchronized Set getNodes() + { + return nodes; + } + } + + private static InternalNode copy(InternalNode node) + { + return new InternalNode( + node.getNodeIdentifier(), + node.getInternalUri(), + node.getNodeVersion(), + node.isCoordinator()); + } + + private static ServerInfo toServerInfo(InternalNode inactiveNode, NodeState nodeState, String environment) + { + return new ServerInfo( + inactiveNode.getNodeIdentifier(), + nodeState, + inactiveNode.getNodeVersion(), + environment, + inactiveNode.isCoordinator(), + Optional.empty(), + false, + Duration.ZERO); + } +} diff --git a/core/trino-main/src/test/java/io/trino/server/TestEmbeddedDiscoveryConfig.java b/core/trino-main/src/test/java/io/trino/server/TestEmbeddedDiscoveryConfig.java index c93813d5aa4b..6320db49fc19 100644 --- a/core/trino-main/src/test/java/io/trino/server/TestEmbeddedDiscoveryConfig.java +++ b/core/trino-main/src/test/java/io/trino/server/TestEmbeddedDiscoveryConfig.java @@ -14,6 +14,7 @@ package io.trino.server; import com.google.common.collect.ImmutableMap; +import io.trino.node.EmbeddedDiscoveryConfig; import org.junit.jupiter.api.Test; import java.util.Map; diff --git a/core/trino-main/src/test/java/io/trino/server/TestNodeResource.java b/core/trino-main/src/test/java/io/trino/server/TestNodeResource.java deleted file mode 100644 index 200a07aefcb3..000000000000 --- a/core/trino-main/src/test/java/io/trino/server/TestNodeResource.java +++ /dev/null @@ -1,86 +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.server; - -import io.airlift.http.client.HttpClient; -import io.airlift.http.client.jetty.JettyHttpClient; -import io.trino.server.testing.TestingTrinoServer; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.parallel.Execution; - -import java.util.List; - -import static io.airlift.http.client.JsonResponseHandler.createJsonResponseHandler; -import static io.airlift.http.client.Request.Builder.prepareGet; -import static io.airlift.json.JsonCodec.listJsonCodec; -import static io.airlift.testing.Closeables.closeAll; -import static io.trino.client.ProtocolHeaders.TRINO_HEADERS; -import static io.trino.failuredetector.HeartbeatFailureDetector.Stats; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; - -@TestInstance(PER_CLASS) -@Execution(CONCURRENT) -public class TestNodeResource -{ - private TestingTrinoServer server; - private HttpClient client; - - @BeforeAll - public void setup() - { - server = TestingTrinoServer.create(); - client = new JettyHttpClient(); - } - - @AfterAll - public void tearDown() - throws Exception - { - closeAll(server, client); - server = null; - client = null; - } - - @Test - public void testGetAllNodes() - { - List nodes = client.execute( - prepareGet() - .setUri(server.resolve("/v1/node")) - .setHeader(TRINO_HEADERS.requestUser(), "unknown") - .build(), - createJsonResponseHandler(listJsonCodec(Stats.class))); - - // we only have one node and the list never contains the current node - assertThat(nodes).isEmpty(); - } - - @Test - public void testGetFailedNodes() - { - List nodes = client.execute( - prepareGet() - .setUri(server.resolve("/v1/node/failed")) - .setHeader(TRINO_HEADERS.requestUser(), "unknown") - .build(), - createJsonResponseHandler(listJsonCodec(Stats.class))); - - assertThat(nodes).isEmpty(); - } -} diff --git a/core/trino-main/src/test/java/io/trino/server/TestNodeStateManager.java b/core/trino-main/src/test/java/io/trino/server/TestNodeStateManager.java index 91002bec950d..a07fca680a74 100644 --- a/core/trino-main/src/test/java/io/trino/server/TestNodeStateManager.java +++ b/core/trino-main/src/test/java/io/trino/server/TestNodeStateManager.java @@ -21,8 +21,9 @@ import io.trino.execution.TaskId; import io.trino.execution.TaskInfo; import io.trino.execution.TaskState; -import io.trino.metadata.NodeState; +import io.trino.node.NodeState; import io.trino.operator.TaskStats; +import io.trino.server.NodeStateManager.CurrentNodeState; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -47,10 +48,10 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; -import static io.trino.metadata.NodeState.ACTIVE; -import static io.trino.metadata.NodeState.DRAINED; -import static io.trino.metadata.NodeState.DRAINING; -import static io.trino.metadata.NodeState.SHUTTING_DOWN; +import static io.trino.node.NodeState.ACTIVE; +import static io.trino.node.NodeState.DRAINED; +import static io.trino.node.NodeState.DRAINING; +import static io.trino.node.NodeState.SHUTTING_DOWN; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; @@ -280,6 +281,7 @@ private NodeStateManager createNodeStateManager(int gracePeriodMillis) Supplier> taskInfoSupplier = () -> tasks.get(); return new NodeStateManager( + new CurrentNodeState(), sqlTasksObservable, taskInfoSupplier, serverConfig, diff --git a/core/trino-main/src/test/java/io/trino/server/remotetask/TestHttpRemoteTask.java b/core/trino-main/src/test/java/io/trino/server/remotetask/TestHttpRemoteTask.java index 0b63590dffd3..4e8df861fd0e 100644 --- a/core/trino-main/src/test/java/io/trino/server/remotetask/TestHttpRemoteTask.java +++ b/core/trino-main/src/test/java/io/trino/server/remotetask/TestHttpRemoteTask.java @@ -54,9 +54,9 @@ import io.trino.metadata.BlockEncodingManager; import io.trino.metadata.HandleJsonModule; import io.trino.metadata.InternalBlockEncodingSerde; -import io.trino.metadata.InternalNode; import io.trino.metadata.Metadata; import io.trino.metadata.Split; +import io.trino.node.InternalNode; import io.trino.server.DynamicFilterService; import io.trino.server.FailTaskRequest; import io.trino.server.HttpRemoteTaskFactory; diff --git a/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java b/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java index 603c7bcc4bb1..6a5de68710e8 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java +++ b/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java @@ -1401,7 +1401,7 @@ private static String getInternalLocation(URI baseUri) private static String getManagementLocation(URI baseUri) { - return getLocation(baseUri, "/v1/node"); + return getLocation(baseUri, "/v1/thread"); } private static String getAuthorizedUserLocation(URI baseUri) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestTableScanNodePartitioning.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestTableScanNodePartitioning.java index fe0f76c0dcb4..3d3266915aa5 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestTableScanNodePartitioning.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestTableScanNodePartitioning.java @@ -19,8 +19,6 @@ import io.trino.connector.MockConnectorColumnHandle; import io.trino.connector.MockConnectorFactory; import io.trino.connector.MockConnectorTableHandle; -import io.trino.metadata.InMemoryNodeManager; -import io.trino.metadata.InternalNodeManager; import io.trino.spi.connector.BucketFunction; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; @@ -44,6 +42,7 @@ import static io.trino.SystemSessionProperties.TASK_CONCURRENCY; import static io.trino.SystemSessionProperties.USE_TABLE_SCAN_NODE_PARTITIONING; +import static io.trino.node.TestingInternalNodeManager.CURRENT_NODE; import static io.trino.spi.connector.ConnectorBucketNodeMap.createBucketNodeMap; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.VarcharType.VARCHAR; @@ -61,7 +60,6 @@ import static io.trino.sql.planner.plan.ExchangeNode.Type.REPARTITION; import static io.trino.testing.TestingHandles.TEST_CATALOG_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; -import static java.util.Objects.requireNonNull; import static org.assertj.core.api.Assertions.assertThat; public class TestTableScanNodePartitioning @@ -174,7 +172,7 @@ void assertTableScanPlannedWithoutPartitioning(Session session, String table) public static MockConnectorFactory createMockFactory() { return MockConnectorFactory.builder() - .withPartitionProvider(new TestPartitioningProvider(new InMemoryNodeManager())) + .withPartitionProvider(new TestPartitioningProvider()) .withGetColumns(schemaTableName -> ImmutableList.of( new ColumnMetadata(COLUMN_A, BIGINT), new ColumnMetadata(COLUMN_B, VARCHAR))) @@ -209,13 +207,6 @@ public static MockConnectorFactory createMockFactory() public static class TestPartitioningProvider implements ConnectorNodePartitioningProvider { - private final InternalNodeManager nodeManager; - - public TestPartitioningProvider(InternalNodeManager nodeManager) - { - this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); - } - @Override public Optional getBucketNodeMapping(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorPartitioningHandle partitioningHandle) { @@ -226,7 +217,7 @@ public Optional getBucketNodeMapping(ConnectorTransactio return Optional.of(createBucketNodeMap(1)); } if (partitioningHandle.equals(FIXED_PARTITIONING_HANDLE)) { - return Optional.of(createBucketNodeMap(ImmutableList.of(nodeManager.getCurrentNode()))); + return Optional.of(createBucketNodeMap(ImmutableList.of(CURRENT_NODE))); } throw new IllegalArgumentException(); } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestLimitMaxWriterNodesCount.java b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestLimitMaxWriterNodesCount.java index 0d738e6a584f..5799f020738a 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestLimitMaxWriterNodesCount.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestLimitMaxWriterNodesCount.java @@ -20,7 +20,6 @@ import io.trino.connector.MockConnector; import io.trino.connector.MockConnectorFactory; import io.trino.connector.MockConnectorTableHandle; -import io.trino.metadata.InMemoryNodeManager; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.ConnectorPartitioningHandle; import io.trino.spi.connector.ConnectorTableLayout; @@ -130,7 +129,7 @@ private MockConnectorFactory prepareConnectorFactory(String catalogName, Optiona "OPTIMIZE", distributedWithFilteringAndRepartitioning(), ImmutableList.of(PropertyMetadata.stringProperty("file_size_threshold", "file_size_threshold", "10GB", false))))) - .withPartitionProvider(new TestTableScanNodePartitioning.TestPartitioningProvider(new InMemoryNodeManager())) + .withPartitionProvider(new TestTableScanNodePartitioning.TestPartitioningProvider()) .withMaxWriterTasks(maxWriterTasks) .withGetColumns(schemaTableName -> ImmutableList.of( new ColumnMetadata("column_a", VARCHAR), 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 index 0384c1291004..697bdc32ecc5 100644 --- 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 @@ -16,7 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import io.trino.client.NodeVersion; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import io.trino.spi.Node; import io.trino.testing.TestingNodeManager; import org.junit.jupiter.api.Test; 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 6d7a9e0e6b7d..5a3545fe2454 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 @@ -65,11 +65,11 @@ import java.util.stream.Stream; import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; +import static io.trino.node.TestingInternalNodeManager.CURRENT_NODE; import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_STATS; import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; -import static io.trino.testing.TestingNodeManager.DEFAULT_CURRENT_NODE; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static java.lang.Math.clamp; import static org.assertj.core.api.Assertions.assertThat; @@ -229,13 +229,13 @@ public Stream getActiveFiles( JsonCodec.jsonCodec(DataFileInfo.class), JsonCodec.jsonCodec(DeltaLakeMergeResult.class), new FileSystemTransactionLogWriterFactory(new TransactionLogSynchronizerManager(ImmutableMap.of(), new NoIsolationSynchronizer(hdfsFileSystemFactory))), - DEFAULT_CURRENT_NODE, + CURRENT_NODE, checkpointWriterManager, new CachingExtendedStatisticsAccess(new MetaDirStatisticsAccess(HDFS_FILE_SYSTEM_FACTORY, new JsonCodecFactory().jsonCodec(ExtendedStatistics.class))), true, false, new NodeVersion("test_version"), - new DeltaLakeTableMetadataScheduler(DEFAULT_CURRENT_NODE, TESTING_TYPE_MANAGER, new DeltaLakeFileMetastoreTableOperationsProvider(hiveMetastoreFactory), Integer.MAX_VALUE, new DeltaLakeConfig()), + new DeltaLakeTableMetadataScheduler(CURRENT_NODE, TESTING_TYPE_MANAGER, new DeltaLakeFileMetastoreTableOperationsProvider(hiveMetastoreFactory), Integer.MAX_VALUE, new DeltaLakeConfig()), newDirectExecutorService(), transactionLogReaderFactory); diff --git a/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxMetadata.java b/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxMetadata.java index 4452fd5b9c66..5c1770508c9a 100644 --- a/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxMetadata.java +++ b/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxMetadata.java @@ -17,7 +17,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.trino.client.NodeVersion; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import io.trino.spi.Node; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorTableHandle; diff --git a/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxSplitManager.java b/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxSplitManager.java index 3e66be565465..f3cfb1d13e35 100644 --- a/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxSplitManager.java +++ b/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxSplitManager.java @@ -18,7 +18,7 @@ import com.google.common.collect.ImmutableSet; import io.airlift.units.Duration; import io.trino.client.NodeVersion; -import io.trino.metadata.InternalNode; +import io.trino.node.InternalNode; import io.trino.spi.HostAddress; import io.trino.spi.Node; import io.trino.spi.NodeManager; diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestingPostgreSqlConnectorContext.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestingPostgreSqlConnectorContext.java index 2867d35d3c98..1bcba50fb60b 100644 --- a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestingPostgreSqlConnectorContext.java +++ b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestingPostgreSqlConnectorContext.java @@ -18,8 +18,8 @@ import io.opentelemetry.api.trace.Tracer; import io.trino.FeaturesConfig; import io.trino.connector.DefaultNodeManager; -import io.trino.metadata.InMemoryNodeManager; import io.trino.metadata.TypeRegistry; +import io.trino.node.TestingInternalNodeManager; import io.trino.operator.FlatHashStrategyCompiler; import io.trino.operator.GroupByHashPageIndexerFactory; import io.trino.operator.PagesIndex; @@ -37,6 +37,7 @@ import io.trino.type.InternalTypeManager; import io.trino.util.EmbedVersion; +import static io.trino.node.TestingInternalNodeManager.CURRENT_NODE; import static io.trino.spi.connector.MetadataProvider.NOOP_METADATA_PROVIDER; import static io.trino.testing.TestingHandles.TEST_CATALOG_HANDLE; @@ -52,7 +53,7 @@ public class TestingPostgreSqlConnectorContext public TestingPostgreSqlConnectorContext() { pageIndexerFactory = new GroupByHashPageIndexerFactory(new FlatHashStrategyCompiler(new TypeOperators())); - nodeManager = new DefaultNodeManager(new InMemoryNodeManager(), true); + nodeManager = new DefaultNodeManager(CURRENT_NODE, TestingInternalNodeManager.createDefault(), true); TypeRegistry typeRegistry = new TypeRegistry(new TypeOperators(), new FeaturesConfig()); typeRegistry.addType(GeometryType.GEOMETRY); typeManager = new InternalTypeManager(typeRegistry); diff --git a/service/trino-proxy/src/test/java/io/trino/proxy/TestProxyServer.java b/service/trino-proxy/src/test/java/io/trino/proxy/TestProxyServer.java index ad44532e9691..c81e327e197d 100644 --- a/service/trino-proxy/src/test/java/io/trino/proxy/TestProxyServer.java +++ b/service/trino-proxy/src/test/java/io/trino/proxy/TestProxyServer.java @@ -86,7 +86,6 @@ public void setupServer() server.createCatalog("tpch", "tpch"); server.installPlugin(new BlackHolePlugin()); server.createCatalog("blackhole", "blackhole"); - server.refreshNodes(); Bootstrap app = new Bootstrap( new TestingNodeModule("test"), diff --git a/testing/trino-testing/pom.xml b/testing/trino-testing/pom.xml index 0e6d437ee8c2..1fe6e7250e93 100644 --- a/testing/trino-testing/pom.xml +++ b/testing/trino-testing/pom.xml @@ -103,11 +103,6 @@ units - - io.airlift.discovery - discovery-server - - io.opentelemetry opentelemetry-api diff --git a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java index 23cf86a0928e..df80665a5127 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java @@ -19,7 +19,6 @@ import com.google.errorprone.annotations.CanIgnoreReturnValue; import com.google.inject.Key; import com.google.inject.Module; -import io.airlift.discovery.server.testing.TestingDiscoveryServer; import io.airlift.http.server.HttpServer; import io.airlift.log.Logger; import io.airlift.log.Logging; @@ -36,10 +35,10 @@ import io.trino.execution.FailureInjector.InjectedFailureType; import io.trino.execution.QueryManager; import io.trino.execution.warnings.WarningCollector; -import io.trino.metadata.AllNodes; import io.trino.metadata.FunctionBundle; import io.trino.metadata.QualifiedObjectName; import io.trino.metadata.SessionPropertyManager; +import io.trino.node.InternalNode; import io.trino.server.BasicQueryInfo; import io.trino.server.PluginManager; import io.trino.server.SessionPropertyDefaults; @@ -89,7 +88,6 @@ import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Throwables.throwIfInstanceOf; import static com.google.common.base.Throwables.throwIfUnchecked; -import static com.google.common.base.Verify.verify; import static com.google.common.collect.Iterables.getOnlyElement; import static com.google.inject.util.Modules.EMPTY_MODULE; import static io.airlift.log.Level.DEBUG; @@ -105,14 +103,13 @@ import static java.util.Objects.requireNonNull; import static org.assertj.core.api.Assertions.assertThat; -public class DistributedQueryRunner +public final class DistributedQueryRunner implements QueryRunner { private static final Logger log = Logger.get(DistributedQueryRunner.class); private static final String ENVIRONMENT = "testing"; private static final AtomicInteger unclosedInstances = new AtomicInteger(); - private TestingDiscoveryServer discoveryServer; private TestingTrinoServer coordinator; private Optional backupCoordinator; private Consumer> registerNewWorker; @@ -158,32 +155,6 @@ private DistributedQueryRunner( setupLogging(); try { - long discoveryStart = System.nanoTime(); - discoveryServer = new TestingDiscoveryServer(environment); - closer.register(() -> closeUnchecked(discoveryServer)); - extraCloseables.forEach(closeable -> closer.register(() -> closeUnchecked(closeable))); - log.debug("Created TestingDiscoveryServer in %s", nanosSince(discoveryStart)); - - registerNewWorker = additionalWorkerProperties -> { - @SuppressWarnings("resource") - TestingTrinoServer ignored = createServer( - false, - ImmutableMap.builder() - .putAll(extraProperties) - .putAll(additionalWorkerProperties) - .buildOrThrow(), - environment, - additionalModule, - baseDataDir, - Optional.empty(), - Optional.of(ImmutableList.of()), - ImmutableList.of()); - }; - - for (int i = 0; i < workerCount; i++) { - registerNewWorker.accept(Map.of()); - } - Map extraCoordinatorProperties = new HashMap<>(); extraCoordinatorProperties.putAll(extraProperties); extraCoordinatorProperties.putAll(coordinatorProperties); @@ -217,6 +188,32 @@ private DistributedQueryRunner( systemAccessControlConfiguration, systemAccessControls, eventListeners)); + backupCoordinator.ifPresent(backup -> { + coordinator.registerServer(backup.getCurrentNode()); + backup.registerServer(coordinator.getCurrentNode()); + }); + + extraCloseables.forEach(closeable -> closer.register(() -> closeUnchecked(closeable))); + + registerNewWorker = additionalWorkerProperties -> { + @SuppressWarnings("resource") + TestingTrinoServer ignored = createServer( + false, + ImmutableMap.builder() + .putAll(extraProperties) + .putAll(additionalWorkerProperties) + .buildOrThrow(), + environment, + additionalModule, + baseDataDir, + Optional.empty(), + Optional.of(ImmutableList.of()), + ImmutableList.of()); + }; + + for (int i = 0; i < workerCount; i++) { + registerNewWorker.accept(Map.of()); + } } catch (Exception e) { try { @@ -235,7 +232,6 @@ private DistributedQueryRunner( this.trinoClient = closer.register(testingTrinoClientFactory.create(coordinator, defaultSession)); - ensureNodesGloballyVisible(); log.info("Created DistributedQueryRunner in %s (unclosed instances = %s)", nanosSince(start), unclosedInstances.incrementAndGet()); } @@ -266,7 +262,6 @@ private TestingTrinoServer createServer( List eventListeners) { TestingTrinoServer server = closer.register(createTestingTrinoServer( - discoveryServer.getBaseUrl(), coordinator, extraCoordinatorProperties, environment, @@ -281,6 +276,11 @@ private TestingTrinoServer createServer( plugins.forEach(newServer::installPlugin); })); servers.add(server); + if (!coordinator) { + InternalNode currentNode = server.getCurrentNode(); + this.coordinator.registerServer(currentNode); + this.backupCoordinator.ifPresent(backup -> backup.registerServer(currentNode)); + } return server; } @@ -297,7 +297,6 @@ private static void setupLogging() } private static TestingTrinoServer createTestingTrinoServer( - URI discoveryUri, boolean coordinator, Map extraProperties, String environment, @@ -313,9 +312,7 @@ private static TestingTrinoServer createTestingTrinoServer( ImmutableMap.Builder propertiesBuilder = ImmutableMap.builder() .put("query.client.timeout", "10m") // Use few threads in tests to preserve resources on CI - .put("discovery.http-client.min-threads", "1") // default 8 .put("exchange.http-client.min-threads", "1") // default 8 - .put("node-manager.http-client.min-threads", "1") // default 8 .put("exchange.page-buffer-client.max-callback-threads", "5") // default 25 .put("exchange.http-client.idle-timeout", "1h") .put("task.max-index-memory", "16kB"); // causes index joins to fault load @@ -324,7 +321,7 @@ private static TestingTrinoServer createTestingTrinoServer( propertiesBuilder.put("join-distribution-type", "PARTITIONED"); // Use few threads in tests to preserve resources on CI - propertiesBuilder.put("failure-detector.http-client.min-threads", "1"); // default 8 + propertiesBuilder.put("catalog-prune.http-client.min-threads", "1"); // default 8 propertiesBuilder.put("memory-manager.http-client.min-threads", "1"); // default 8 propertiesBuilder.put("scheduler.http-client.min-threads", "1"); // default 8 propertiesBuilder.put("worker-info.http-client.min-threads", "1"); // default 8 @@ -336,7 +333,6 @@ private static TestingTrinoServer createTestingTrinoServer( .setCoordinator(coordinator) .setProperties(properties) .setEnvironment(environment) - .setDiscoveryUri(discoveryUri) .setAdditionalModule(additionalModule) .setBaseDataDir(baseDataDir) .setSpanProcessor(spanProcessor) @@ -357,7 +353,27 @@ public void addServers(int nodeCount) for (int i = 0; i < nodeCount; i++) { registerNewWorker.accept(Map.of()); } - ensureNodesGloballyVisible(); + } + + /** + * Shutdown and remove a worker + */ + public void removeWorker() + throws Exception + { + TestingTrinoServer worker = null; + for (int i = 0; i < servers.size(); i++) { + if (!servers.get(i).isCoordinator()) { + worker = servers.get(i); + servers.remove(i); + break; + } + } + if (worker == null) { + throw new IllegalStateException("No workers"); + } + worker.close(); + coordinator.unregisterServer(worker.getCurrentNode()); } /** @@ -375,6 +391,10 @@ public void restartWorker(TestingTrinoServer server) Connector httpConnector = getOnlyElement(asList(((Server) serverField.get(workerHttpServer)).getConnectors())); httpConnector.stop(); server.close(); + if (!server.isCoordinator()) { + coordinator.unregisterServer(server.getCurrentNode()); + backupCoordinator.ifPresent(backup -> backup.unregisterServer(server.getCurrentNode())); + } Map reusePort = Map.of("http-server.http.port", Integer.toString(baseUrl.getPort())); registerNewWorker.accept(reusePort); @@ -386,15 +406,6 @@ public void restartWorker(TestingTrinoServer server) // Do not wait for new server to be fully registered with other servers } - private void ensureNodesGloballyVisible() - { - for (TestingTrinoServer server : servers) { - AllNodes nodes = server.refreshNodes(); - verify(nodes.getInactiveNodes().isEmpty(), "Node manager has inactive nodes"); - verify(nodes.getActiveNodes().size() == servers.size(), "Node manager has wrong active node count"); - } - } - public TestingTrinoClient getClient() { return trinoClient; @@ -664,7 +675,6 @@ public final void close() catch (IOException e) { throw new UncheckedIOException(e); } - discoveryServer = null; coordinator = null; backupCoordinator = Optional.empty(); registerNewWorker = _ -> { diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestGracefulShutdown.java b/testing/trino-tests/src/test/java/io/trino/tests/TestGracefulShutdown.java index 06545ea9bffc..977650279c7b 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestGracefulShutdown.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestGracefulShutdown.java @@ -20,7 +20,7 @@ import com.google.common.util.concurrent.MoreExecutors; import io.trino.Session; import io.trino.execution.SqlTaskManager; -import io.trino.metadata.NodeState; +import io.trino.node.NodeState; import io.trino.server.BasicQueryInfo; import io.trino.server.testing.TestingTrinoServer; import io.trino.server.testing.TestingTrinoServer.TestShutdownAction; diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestMinWorkerRequirement.java b/testing/trino-tests/src/test/java/io/trino/tests/TestMinWorkerRequirement.java index 095e5fab7512..bbe511224e00 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestMinWorkerRequirement.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestMinWorkerRequirement.java @@ -26,6 +26,7 @@ import io.trino.tests.tpch.TpchQueryRunner; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.parallel.Execution; import static io.trino.SystemSessionProperties.REQUIRED_WORKERS_COUNT; @@ -107,6 +108,7 @@ public void testInsufficientWorkerNodesInternalSystemQuery() } @Test + @Timeout(60) public void testInsufficientWorkerNodesAfterDrop() throws Exception { @@ -118,10 +120,10 @@ public void testInsufficientWorkerNodesAfterDrop() .setWorkerCount(3) .build()) { queryRunner.execute("SELECT COUNT(*) from lineitem"); - assertThat(queryRunner.getCoordinator().refreshNodes().getActiveNodes()).hasSize(4); + assertThat(queryRunner.getCoordinator().getWorkerCount()).isEqualTo(3); - queryRunner.getServers().get(0).close(); - assertThat(queryRunner.getCoordinator().refreshNodes().getActiveNodes()).hasSize(3); + // Stop one worker + queryRunner.removeWorker(); assertThatThrownBy(() -> queryRunner.execute("SELECT COUNT(*) from lineitem")) .isInstanceOf(RuntimeException.class) .hasMessage("Insufficient active worker nodes. Waited 1.00ns for at least 4 workers, but only 3 workers are active"); @@ -182,7 +184,7 @@ public void testRequiredWorkerNodesSessionOverride() // After adding 2 nodes, query should run queryRunner.addServers(2); - assertThat(queryRunner.getCoordinator().refreshNodes().getActiveNodes()).hasSize(6); + assertThat(queryRunner.getCoordinator().getWorkerCount()).isEqualTo(5); queryRunner.execute(require6Workers, "SELECT COUNT(*) from lineitem"); } } @@ -215,7 +217,7 @@ public void testMultipleRequiredWorkerNodesSessionOverride() assertThat(queryFuture3.isDone()).isFalse(); queryRunner.addServers(1); - assertThat(queryRunner.getCoordinator().refreshNodes().getActiveNodes()).hasSize(2); + assertThat(queryRunner.getCoordinator().getWorkerCount()).isEqualTo(1); // After adding 1 node, only 1st query should run MILLISECONDS.sleep(1000); assertThat(queryFuture1.get().result().getRowCount() > 0).isTrue(); @@ -228,7 +230,7 @@ public void testMultipleRequiredWorkerNodesSessionOverride() // After adding 2 nodes, 2nd and 3rd query should also run queryRunner.addServers(2); - assertThat(queryRunner.getCoordinator().refreshNodes().getActiveNodes()).hasSize(4); + assertThat(queryRunner.getCoordinator().getWorkerCount()).isEqualTo(3); assertThat(queryFuture2.get().result().getRowCount() > 0).isTrue(); completedQueryInfo = queryManager.getFullQueryInfo(queryFuture2.get().queryId()); assertThat(completedQueryInfo.getQueryStats().getResourceWaitingTime().roundTo(SECONDS) >= 2).isTrue();