Skip to content

Commit

Permalink
Remove experimental preallocated memory system
Browse files Browse the repository at this point in the history
  • Loading branch information
dain committed Nov 4, 2018
1 parent b77fa0b commit 105a057
Show file tree
Hide file tree
Showing 6 changed files with 2 additions and 504 deletions.

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@
import com.facebook.presto.execution.scheduler.SqlQueryScheduler;
import com.facebook.presto.execution.warnings.WarningCollector;
import com.facebook.presto.failureDetector.FailureDetector;
import com.facebook.presto.memory.ClusterMemoryManager;
import com.facebook.presto.memory.VersionedMemoryPoolId;
import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.metadata.TableHandle;
Expand All @@ -43,7 +42,6 @@
import com.facebook.presto.split.SplitSource;
import com.facebook.presto.sql.analyzer.Analysis;
import com.facebook.presto.sql.analyzer.Analyzer;
import com.facebook.presto.sql.analyzer.FeaturesConfig;
import com.facebook.presto.sql.analyzer.QueryExplainer;
import com.facebook.presto.sql.parser.SqlParser;
import com.facebook.presto.sql.planner.DistributedExecutionPlanner;
Expand Down Expand Up @@ -313,20 +311,6 @@ public BasicQueryInfo getBasicQueryInfo()
.orElseGet(() -> stateMachine.getBasicQueryInfo(Optional.ofNullable(queryScheduler.get()).map(SqlQueryScheduler::getBasicStageStats)));
}

public void startWaitingForResources()
{
try (SetThreadName ignored = new SetThreadName("Query-%s", stateMachine.getQueryId())) {
try {
// transition to waiting for resources
stateMachine.transitionToWaitingForResources();
}
catch (Throwable e) {
fail(e);
throwIfInstanceOf(e, Error.class);
}
}
}

@Override
public void start()
{
Expand Down Expand Up @@ -677,14 +661,11 @@ public static class SqlQueryExecutionFactory
private final FailureDetector failureDetector;
private final NodeTaskMap nodeTaskMap;
private final Map<String, ExecutionPolicy> executionPolicies;
private final ClusterMemoryManager clusterMemoryManager;
private final DataSize preAllocateMemoryThreshold;
private final StatsCalculator statsCalculator;
private final CostCalculator costCalculator;

@Inject
SqlQueryExecutionFactory(QueryManagerConfig config,
FeaturesConfig featuresConfig,
Metadata metadata,
AccessControl accessControl,
SqlParser sqlParser,
Expand All @@ -703,7 +684,6 @@ public static class SqlQueryExecutionFactory
QueryExplainer queryExplainer,
Map<String, ExecutionPolicy> executionPolicies,
SplitSchedulerStats schedulerStats,
ClusterMemoryManager clusterMemoryManager,
StatsCalculator statsCalculator,
CostCalculator costCalculator)
{
Expand All @@ -727,8 +707,6 @@ public static class SqlQueryExecutionFactory
this.nodeTaskMap = requireNonNull(nodeTaskMap, "nodeTaskMap is null");
this.queryExplainer = requireNonNull(queryExplainer, "queryExplainer is null");
this.executionPolicies = requireNonNull(executionPolicies, "schedulerPolicies is null");
this.clusterMemoryManager = requireNonNull(clusterMemoryManager, "clusterMemoryManager is null");
this.preAllocateMemoryThreshold = requireNonNull(featuresConfig, "featuresConfig is null").getPreAllocateMemoryThreshold();
this.planOptimizers = planOptimizers.get();
this.statsCalculator = requireNonNull(statsCalculator, "statsCalculator is null");
this.costCalculator = requireNonNull(costCalculator, "costCalculator is null");
Expand Down Expand Up @@ -775,11 +753,6 @@ public QueryExecution createQueryExecution(
costCalculator,
warningCollector);

if (preAllocateMemoryThreshold.toBytes() > 0 && session.getResourceEstimates().getPeakMemory().isPresent() &&
session.getResourceEstimates().getPeakMemory().get().compareTo(preAllocateMemoryThreshold) >= 0) {
return new MemoryAwareQueryExecution(clusterMemoryManager, execution);
}

return execution;
}
}
Expand Down
Loading

0 comments on commit 105a057

Please sign in to comment.