Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Do not allocate task memory for VALUES/metadata-only queries #19027

Merged
merged 2 commits into from
Sep 14, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -512,7 +512,7 @@ public BinPackingSimulation(
continue;
}
long nodeReservedMemory = preReservedMemory.getOrDefault(node.getNodeIdentifier(), 0L);
nodesRemainingMemory.put(node.getNodeIdentifier(), memoryPoolInfo.getMaxBytes() - nodeReservedMemory);
nodesRemainingMemory.put(node.getNodeIdentifier(), max(memoryPoolInfo.getMaxBytes() - nodeReservedMemory, 0L));
}

nodesRemainingMemoryRuntimeAdjusted = new HashMap<>();
Expand Down Expand Up @@ -540,7 +540,7 @@ public BinPackingSimulation(
// if globally reported memory usage of node is greater than computed one lets use that.
// it can be greater if there are tasks executed on cluster which do not have task retries enabled.
nodeUsedMemoryRuntimeAdjusted = max(nodeUsedMemoryRuntimeAdjusted, memoryPoolInfo.getReservedBytes());
nodesRemainingMemoryRuntimeAdjusted.put(node.getNodeIdentifier(), memoryPoolInfo.getMaxBytes() - nodeUsedMemoryRuntimeAdjusted);
nodesRemainingMemoryRuntimeAdjusted.put(node.getNodeIdentifier(), max(memoryPoolInfo.getMaxBytes() - nodeUsedMemoryRuntimeAdjusted, 0L));
}
}

Expand Down Expand Up @@ -610,10 +610,10 @@ private void subtractFromRemainingMemory(String nodeIdentifier, long memoryLease
{
nodesRemainingMemoryRuntimeAdjusted.compute(
nodeIdentifier,
(key, free) -> free - memoryLease);
(key, free) -> max(free - memoryLease, 0));
nodesRemainingMemory.compute(
nodeIdentifier,
(key, free) -> free - memoryLease);
(key, free) -> max(free - memoryLease, 0));
}

private boolean isNodeEmpty(String nodeIdentifier)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,9 @@
import io.airlift.units.Duration;
import io.opentelemetry.api.trace.Tracer;
import io.trino.Session;
import io.trino.connector.informationschema.InformationSchemaTableHandle;
import io.trino.connector.system.GlobalSystemConnector;
import io.trino.connector.system.SystemTableHandle;
import io.trino.exchange.SpoolingExchangeInput;
import io.trino.execution.BasicStageStats;
import io.trino.execution.ExecutionFailureInfo;
Expand Down Expand Up @@ -85,12 +88,14 @@
import io.trino.sql.planner.PlanFragmentIdAllocator;
import io.trino.sql.planner.PlanNodeIdAllocator;
import io.trino.sql.planner.SubPlan;
import io.trino.sql.planner.optimizations.PlanNodeSearcher;
import io.trino.sql.planner.plan.AggregationNode;
import io.trino.sql.planner.plan.PlanFragmentId;
import io.trino.sql.planner.plan.PlanNode;
import io.trino.sql.planner.plan.PlanNodeId;
import io.trino.sql.planner.plan.RefreshMaterializedViewNode;
import io.trino.sql.planner.plan.RemoteSourceNode;
import io.trino.sql.planner.plan.TableScanNode;
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
import it.unimi.dsi.fastutil.ints.IntOpenHashSet;
Expand Down Expand Up @@ -1150,14 +1155,16 @@ private void createStageExecution(SubPlan subPlan, boolean rootFragment, Map<Sta

boolean coordinatorStage = stage.getFragment().getPartitioning().equals(COORDINATOR_DISTRIBUTION);

boolean noMemoryFragment = isNoMemoryFragment(fragment);
StageExecution execution = new StageExecution(
queryStateMachine,
taskDescriptorStorage,
stage,
taskSource,
sinkPartitioningScheme,
exchange,
memoryEstimatorFactory.createPartitionMemoryEstimator(),
noMemoryFragment,
noMemoryFragment ? new NoMemoryPartitionMemoryEstimator() : memoryEstimatorFactory.createPartitionMemoryEstimator(),
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can the standard estimator be modified to more adequatly estimate memory usage of information_schema queries?
like

  • if source stage and this is information_schema assume 0 memory need and ~0 (negligible) number of rows produced
  • for intermediate stage, and sources produce ~0 rows, assume 0 memory need

(actually we should get rid of intermediate stages for DESCRIBE, it's nonsense)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Having smarter memory estimation in general is a planned item - but much more complex than this fix. And will require some of a restructuring of interfaces. I think this is fine to have this as a stopgap for sake of metadata queries which are a very common outlier even for clusters which tend to run massing ETL workloads.

Or maybe I did not understand your suggestion here. Let's chat tomorrow.

// do not retry coordinator only tasks
coordinatorStage ? 1 : maxTaskExecutionAttempts,
schedulingPriority,
Expand Down Expand Up @@ -1190,6 +1197,29 @@ private void createStageExecution(SubPlan subPlan, boolean rootFragment, Map<Sta
}
}

private boolean isNoMemoryFragment(PlanFragment fragment)
{
// If source fragments are not tagged as "no-memory" assume that they may produce significant amount of data.
// We stay on the safe side an assume that we should use standard memory estimation for this fragment
if (!fragment.getRemoteSourceNodes().stream().flatMap(node -> node.getSourceFragmentIds().stream())
.allMatch(sourceFragmentId -> stageExecutions.get(getStageId(sourceFragmentId)).isNoMemoryFragment())) {
return false;
Comment on lines +1204 to +1206
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why remote sources matter?
shouldn't isNoMemoryFragment(PlanFragment) be a local assessment about the fragment?

is this is about whole execution subtree, let's rename the method to indicate that

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is about local fragment. It just uses this bit of information for source stages as a shortcut for "source will produce negligible amount of data". Which is a simplification - but not that much given logic we use to set this bit. I will add a comment.

}

// If fragment source is not reading any external tables or only accesses information_schema assume it does not need significant amount of memory.
// Allow scheduling even if whole server memory is pre allocated.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

<3

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will we schedule unbounded number of information_schema queries? there must be some sort of a limit for them.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we do not have resource queues in place yes. But one can argue this is a mis-setup anyway - and very much possible in streaming mode right now.

List<PlanNode> tableScanNodes = PlanNodeSearcher.searchFrom(fragment.getRoot()).whereIsInstanceOfAny(TableScanNode.class).findAll();
return tableScanNodes.stream().allMatch(node -> isMetadataTableScan((TableScanNode) node));
}

private static boolean isMetadataTableScan(TableScanNode tableScanNode)
{
return (tableScanNode.getTable().getConnectorHandle() instanceof InformationSchemaTableHandle) ||
(tableScanNode.getTable().getCatalogHandle().getCatalogName().equals(GlobalSystemConnector.NAME) &&
(tableScanNode.getTable().getConnectorHandle() instanceof SystemTableHandle systemHandle) &&
systemHandle.getSchemaName().equals("jdbc"));
}

private StageId getStageId(PlanFragmentId fragmentId)
{
return StageId.create(queryStateMachine.getQueryId(), fragmentId);
Expand Down Expand Up @@ -1523,6 +1553,7 @@ private static class StageExecution
private final EventDrivenTaskSource taskSource;
private final FaultTolerantPartitioningScheme sinkPartitioningScheme;
private final Exchange exchange;
private final boolean noMemoryFragment;
private final PartitionMemoryEstimator partitionMemoryEstimator;
private final int maxTaskExecutionAttempts;
private final int schedulingPriority;
Expand Down Expand Up @@ -1559,6 +1590,7 @@ private StageExecution(
EventDrivenTaskSource taskSource,
FaultTolerantPartitioningScheme sinkPartitioningScheme,
Exchange exchange,
boolean noMemoryFragment,
PartitionMemoryEstimator partitionMemoryEstimator,
int maxTaskExecutionAttempts,
int schedulingPriority,
Expand All @@ -1576,6 +1608,7 @@ private StageExecution(
this.taskSource = requireNonNull(taskSource, "taskSource is null");
this.sinkPartitioningScheme = requireNonNull(sinkPartitioningScheme, "sinkPartitioningScheme is null");
this.exchange = requireNonNull(exchange, "exchange is null");
this.noMemoryFragment = noMemoryFragment;
this.partitionMemoryEstimator = requireNonNull(partitionMemoryEstimator, "partitionMemoryEstimator is null");
this.maxTaskExecutionAttempts = maxTaskExecutionAttempts;
this.schedulingPriority = schedulingPriority;
Expand Down Expand Up @@ -1628,6 +1661,11 @@ public boolean isExchangeClosed()
return exchangeClosed;
}

public boolean isNoMemoryFragment()
{
return noMemoryFragment;
}

public void addPartition(int partitionId, NodeRequirements nodeRequirements)
{
if (getState().isDone()) {
Expand Down
Original file line number Diff line number Diff line change
@@ -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.execution.scheduler;

import io.airlift.units.DataSize;
import io.trino.Session;
import io.trino.spi.ErrorCode;

import java.util.Optional;

public class NoMemoryPartitionMemoryEstimator
implements PartitionMemoryEstimator
{
@Override
public MemoryRequirements getInitialMemoryRequirements(Session session, DataSize defaultMemoryLimit)
{
return new MemoryRequirements(DataSize.ofBytes(0));
}

@Override
public MemoryRequirements getNextRetryMemoryRequirements(Session session, MemoryRequirements previousMemoryRequirements, DataSize peakMemoryUsage, ErrorCode errorCode)
{
return new MemoryRequirements(DataSize.ofBytes(0));
}

@Override
public void registerPartitionFinished(Session session, MemoryRequirements previousMemoryRequirements, DataSize peakMemoryUsage, boolean success, Optional<ErrorCode> errorCode) {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -14,19 +14,29 @@
package io.trino.faulttolerant;

import com.google.common.collect.ImmutableMap;
import com.google.common.collect.MoreCollectors;
import io.trino.Session;
import io.trino.connector.MockConnectorFactory;
import io.trino.connector.MockConnectorPlugin;
import io.trino.execution.QueryState;
import io.trino.plugin.exchange.filesystem.FileSystemExchangePlugin;
import io.trino.plugin.memory.MemoryQueryRunner;
import io.trino.server.BasicQueryInfo;
import io.trino.testing.AbstractDistributedEngineOnlyQueries;
import io.trino.testing.DistributedQueryRunner;
import io.trino.testing.FaultTolerantExecutionConnectorTestHelper;
import io.trino.testing.QueryRunner;
import io.trino.tpch.TpchTable;
import org.testng.annotations.Test;

import java.util.Optional;
import java.util.concurrent.ExecutorService;

import static io.airlift.testing.Closeables.closeAllSuppress;
import static io.trino.testing.TestingNames.randomNameSuffix;
import static io.trino.testing.assertions.Assert.assertEventually;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static org.assertj.core.api.Assertions.assertThat;

public class TestDistributedFaultTolerantEngineOnlyQueries
extends AbstractDistributedEngineOnlyQueries
Expand Down Expand Up @@ -97,4 +107,66 @@ t2 AS (

assertUpdate("DROP TABLE " + tableName);
}

@Test(timeOut = 30_000)
public void testMetadataOnlyQueries()
throws InterruptedException
{
// enforce single task uses whole node
Session highTaskMemorySession = Session.builder(getSession())
.setSystemProperty("fault_tolerant_execution_coordinator_task_memory", "500GB")
.setSystemProperty("fault_tolerant_execution_task_memory", "500GB")
.build();

ExecutorService backgroundExecutor = newCachedThreadPool();
try {
String longQuery = "select count(*) long_query_count FROM lineitem l1 cross join lineitem l2 cross join lineitem l3 where l1.orderkey * l2.orderkey * l3.orderkey = 1";
backgroundExecutor.submit(() -> {
query(highTaskMemorySession, longQuery);
});
assertEventually(() -> queryIsInState(longQuery, QueryState.RUNNING));

assertThat(query("DESCRIBE lineitem")).succeeds();
assertThat(query("SHOW TABLES")).succeeds();
assertThat(query("SHOW TABLES LIKE 'line%'")).succeeds();
assertThat(query("SHOW SCHEMAS")).succeeds();
assertThat(query("SHOW SCHEMAS LIKE 'def%'")).succeeds();
assertThat(query("SHOW CATALOGS")).succeeds();
assertThat(query("SHOW CATALOGS LIKE 'mem%'")).succeeds();
assertThat(query("SHOW FUNCTIONS")).succeeds();
assertThat(query("SHOW FUNCTIONS LIKE 'split%'")).succeeds();
assertThat(query("SHOW COLUMNS FROM lineitem")).succeeds();
assertThat(query("SHOW SESSION")).succeeds();
assertThat(query("SELECT count(*) FROM information_schema.tables")).succeeds();
assertThat(query("SELECT * FROM system.jdbc.tables WHERE table_schem LIKE 'def%'")).succeeds();

// check non-metadata queries still wait for resources
String nonMetadataQuery = "select count(*) non_metadata_query_count from nation";
backgroundExecutor.submit(() -> {
query(nonMetadataQuery);
});
assertEventually(() -> queryIsInState(nonMetadataQuery, QueryState.STARTING));
Thread.sleep(1000); // wait a bit longer and query should be still STARTING
assertThat(queryState(nonMetadataQuery).orElseThrow()).isEqualTo(QueryState.STARTING);

// long query should be still running
assertThat(queryState(longQuery).orElseThrow()).isEqualTo(QueryState.RUNNING);
}
finally {
backgroundExecutor.shutdownNow();
}
}

private Optional<QueryState> queryState(String queryText)
{
return getDistributedQueryRunner().getCoordinator().getQueryManager().getQueries().stream()
.filter(query -> query.getQuery().equals(queryText))
.collect(MoreCollectors.toOptional())
.map(BasicQueryInfo::getState);
}

private boolean queryIsInState(String queryText, QueryState queryState)
{
return queryState(queryText).map(state -> state == queryState).orElse(false);
}
}
Loading