Skip to content

Commit 1d92bda

Browse files
committed
Add LocalQueryRunner builder
1 parent 9d68bae commit 1d92bda

File tree

40 files changed

+146
-88
lines changed

40 files changed

+146
-88
lines changed

Diff for: presto-atop/src/test/java/io/prestosql/plugin/atop/LocalAtopQueryRunner.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,7 @@ public static LocalQueryRunner createQueryRunner(Map<String, String> catalogProp
4141
.setTimeZoneKey(TimeZoneKey.getTimeZoneKey(TimeZone.getDefault().getID()))
4242
.build();
4343

44-
LocalQueryRunner queryRunner = new LocalQueryRunner(session);
44+
LocalQueryRunner queryRunner = LocalQueryRunner.create(session);
4545

4646
try {
4747
AtopConnectorFactory connectorFactory = new AtopConnectorFactory(factoryClass, LocalAtopQueryRunner.class.getClassLoader());

Diff for: presto-benchmark/src/main/java/io/prestosql/benchmark/BenchmarkQueryRunner.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,7 @@ public static LocalQueryRunner createLocalQueryRunner(Map<String, String> extraS
4747
extraSessionProperties.forEach(sessionBuilder::setSystemProperty);
4848

4949
Session session = sessionBuilder.build();
50-
LocalQueryRunner localQueryRunner = new LocalQueryRunner(session);
50+
LocalQueryRunner localQueryRunner = LocalQueryRunner.create(session);
5151

5252
// add tpch
5353
localQueryRunner.createCatalog("tpch", new TpchConnectorFactory(1), ImmutableMap.of());

Diff for: presto-benchmark/src/test/java/io/prestosql/benchmark/MemoryLocalQueryRunner.java

+3-1
Original file line numberDiff line numberDiff line change
@@ -118,7 +118,9 @@ public List<Page> execute(@Language("SQL") String query)
118118

119119
private static LocalQueryRunner createMemoryLocalQueryRunner(Session session)
120120
{
121-
LocalQueryRunner localQueryRunner = LocalQueryRunner.queryRunnerWithInitialTransaction(session);
121+
LocalQueryRunner localQueryRunner = LocalQueryRunner.builder(session)
122+
.withInitialTransaction()
123+
.build();
122124

123125
// add tpch
124126
localQueryRunner.createCatalog("tpch", new TpchConnectorFactory(1), ImmutableMap.of());

Diff for: presto-benchto-benchmarks/src/test/java/io/prestosql/sql/planner/TestTpcdsCostBasedPlan.java

+3-1
Original file line numberDiff line numberDiff line change
@@ -54,7 +54,9 @@ public TestTpcdsCostBasedPlan()
5454
.setSystemProperty(JOIN_REORDERING_STRATEGY, JoinReorderingStrategy.AUTOMATIC.name())
5555
.setSystemProperty(JOIN_DISTRIBUTION_TYPE, JoinDistributionType.AUTOMATIC.name());
5656

57-
LocalQueryRunner queryRunner = LocalQueryRunner.queryRunnerWithFakeNodeCountForStats(sessionBuilder.build(), 8);
57+
LocalQueryRunner queryRunner = LocalQueryRunner.builder(sessionBuilder.build())
58+
.withNodeCountForStats(8)
59+
.build();
5860
queryRunner.createCatalog(
5961
catalog,
6062
new TpcdsConnectorFactory(1),

Diff for: presto-benchto-benchmarks/src/test/java/io/prestosql/sql/planner/TestTpchCostBasedPlan.java

+3-1
Original file line numberDiff line numberDiff line change
@@ -56,7 +56,9 @@ public TestTpchCostBasedPlan()
5656
.setSystemProperty(JOIN_REORDERING_STRATEGY, JoinReorderingStrategy.AUTOMATIC.name())
5757
.setSystemProperty(JOIN_DISTRIBUTION_TYPE, JoinDistributionType.AUTOMATIC.name());
5858

59-
LocalQueryRunner queryRunner = LocalQueryRunner.queryRunnerWithFakeNodeCountForStats(sessionBuilder.build(), 8);
59+
LocalQueryRunner queryRunner = LocalQueryRunner.builder(sessionBuilder.build())
60+
.withNodeCountForStats(8)
61+
.build();
6062
queryRunner.createCatalog(
6163
catalog,
6264
new TpchConnectorFactory(1, false, false),

Diff for: presto-geospatial/src/test/java/io/prestosql/plugin/geospatial/BenchmarkGeometryAggregations.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -65,7 +65,7 @@ public LocalQueryRunner getQueryRunner()
6565
public void setUp()
6666
throws IOException
6767
{
68-
queryRunner = new LocalQueryRunner(testSessionBuilder()
68+
queryRunner = LocalQueryRunner.create(testSessionBuilder()
6969
.setCatalog("memory")
7070
.setSchema("default")
7171
.build());

Diff for: presto-geospatial/src/test/java/io/prestosql/plugin/geospatial/BenchmarkSpatialJoin.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -77,7 +77,7 @@ public LocalQueryRunner getQueryRunner()
7777
public void setUp()
7878
throws IOException
7979
{
80-
queryRunner = new LocalQueryRunner(testSessionBuilder()
80+
queryRunner = LocalQueryRunner.create(testSessionBuilder()
8181
.setCatalog("memory")
8282
.setSchema("default")
8383
.build());

Diff for: presto-geospatial/src/test/java/io/prestosql/plugin/geospatial/TestSpatialJoinPlanning.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -66,7 +66,7 @@ public TestSpatialJoinPlanning()
6666

6767
private static LocalQueryRunner createQueryRunner()
6868
{
69-
LocalQueryRunner queryRunner = new LocalQueryRunner(testSessionBuilder()
69+
LocalQueryRunner queryRunner = LocalQueryRunner.create(testSessionBuilder()
7070
.setCatalog("memory")
7171
.setSchema("default")
7272
.build());

Diff for: presto-hive/src/test/java/io/prestosql/plugin/hive/HiveBenchmarkQueryRunner.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,7 @@ public static LocalQueryRunner createLocalQueryRunner(File tempDir)
5959
.setSchema("tpch")
6060
.build();
6161

62-
LocalQueryRunner localQueryRunner = new LocalQueryRunner(session);
62+
LocalQueryRunner localQueryRunner = LocalQueryRunner.create(session);
6363

6464
// add tpch
6565
localQueryRunner.createCatalog("tpch", new TpchConnectorFactory(1), ImmutableMap.of());

Diff for: presto-main/src/main/java/io/prestosql/testing/LocalQueryRunner.java

+81-40
Original file line numberDiff line numberDiff line change
@@ -222,8 +222,6 @@ public class LocalQueryRunner
222222
private final SqlParser sqlParser;
223223
private final PlanFragmenter planFragmenter;
224224
private final InMemoryNodeManager nodeManager;
225-
private final PageSorter pageSorter;
226-
private final PageIndexerFactory pageIndexerFactory;
227225
private final MetadataManager metadata;
228226
private final StatsCalculator statsCalculator;
229227
private final CostCalculator costCalculator;
@@ -251,45 +249,35 @@ public class LocalQueryRunner
251249
private final TaskManagerConfig taskManagerConfig;
252250
private final boolean alwaysRevokeMemory;
253251
private final NodeSpillConfig nodeSpillConfig;
254-
private final NodeSchedulerConfig nodeSchedulerConfig;
255252
private final FeaturesConfig featuresConfig;
256-
private final Map<String, List<PropertyMetadata<?>>> defaultSessionProperties;
257253
private boolean printPlan;
258254

259255
private final ReadWriteLock lock = new ReentrantReadWriteLock();
260256

261-
public LocalQueryRunner(Session defaultSession)
257+
public static LocalQueryRunner create(Session defaultSession)
262258
{
263-
this(defaultSession, new FeaturesConfig(), new NodeSpillConfig(), false, false);
259+
return builder(defaultSession).build();
264260
}
265261

266-
public LocalQueryRunner(Session defaultSession, Map<String, List<PropertyMetadata<?>>> defaultSessionProperties)
262+
public static Builder builder(Session defaultSession)
267263
{
268-
this(defaultSession, new FeaturesConfig(), new NodeSpillConfig(), false, false, defaultSessionProperties);
264+
return new Builder(defaultSession);
269265
}
270266

271-
public LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig)
272-
{
273-
this(defaultSession, featuresConfig, new NodeSpillConfig(), false, false);
274-
}
275-
276-
public LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig, NodeSpillConfig nodeSpillConfig, boolean withInitialTransaction, boolean alwaysRevokeMemory)
277-
{
278-
this(defaultSession, featuresConfig, nodeSpillConfig, withInitialTransaction, alwaysRevokeMemory, 1, ImmutableMap.of());
279-
}
280-
281-
public LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig, NodeSpillConfig nodeSpillConfig, boolean withInitialTransaction, boolean alwaysRevokeMemory, Map<String, List<PropertyMetadata<?>>> defaultSessionProperties)
282-
{
283-
this(defaultSession, featuresConfig, nodeSpillConfig, withInitialTransaction, alwaysRevokeMemory, 1, defaultSessionProperties);
284-
}
285-
286-
private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig, NodeSpillConfig nodeSpillConfig, boolean withInitialTransaction, boolean alwaysRevokeMemory, int nodeCountForStats, Map<String, List<PropertyMetadata<?>>> defaultSessionProperties)
267+
private LocalQueryRunner(
268+
Session defaultSession,
269+
FeaturesConfig featuresConfig,
270+
NodeSpillConfig nodeSpillConfig,
271+
boolean withInitialTransaction,
272+
boolean alwaysRevokeMemory,
273+
int nodeCountForStats,
274+
Map<String, List<PropertyMetadata<?>>> defaultSessionProperties)
287275
{
288276
requireNonNull(defaultSession, "defaultSession is null");
277+
requireNonNull(defaultSessionProperties, "defaultSessionProperties is null");
289278
checkArgument(!defaultSession.getTransactionId().isPresent() || !withInitialTransaction, "Already in transaction");
290279

291280
this.taskManagerConfig = new TaskManagerConfig().setTaskConcurrency(4);
292-
this.defaultSessionProperties = requireNonNull(defaultSessionProperties, "defaultSessionProperties is null");
293281
this.nodeSpillConfig = requireNonNull(nodeSpillConfig, "nodeSpillConfig is null");
294282
this.alwaysRevokeMemory = alwaysRevokeMemory;
295283
this.notificationExecutor = newCachedThreadPool(daemonThreadsNamed("local-query-runner-executor-%s"));
@@ -299,9 +287,9 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig,
299287

300288
this.sqlParser = new SqlParser();
301289
this.nodeManager = new InMemoryNodeManager();
302-
this.pageSorter = new PagesIndexPageSorter(new PagesIndex.TestingFactory(false));
290+
PageSorter pageSorter = new PagesIndexPageSorter(new PagesIndex.TestingFactory(false));
303291
this.indexManager = new IndexManager();
304-
this.nodeSchedulerConfig = new NodeSchedulerConfig().setIncludeCoordinator(true);
292+
NodeSchedulerConfig nodeSchedulerConfig = new NodeSchedulerConfig().setIncludeCoordinator(true);
305293
NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory(nodeManager, nodeSchedulerConfig, new NodeTaskMap(finalizerService)));
306294
this.featuresConfig = requireNonNull(featuresConfig, "featuresConfig is null");
307295
this.pageSinkManager = new PageSinkManager();
@@ -324,7 +312,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig,
324312
this.splitManager = new SplitManager(new QueryManagerConfig(), metadata);
325313
this.planFragmenter = new PlanFragmenter(this.metadata, this.nodePartitioningManager, new QueryManagerConfig());
326314
this.joinCompiler = new JoinCompiler(metadata);
327-
this.pageIndexerFactory = new GroupByHashPageIndexerFactory(joinCompiler);
315+
PageIndexerFactory pageIndexerFactory = new GroupByHashPageIndexerFactory(joinCompiler);
328316
this.statsCalculator = createNewStatsCalculator(metadata, new TypeAnalyzer(sqlParser, metadata));
329317
this.taskCountEstimator = new TaskCountEstimator(() -> nodeCountForStats);
330318
this.costCalculator = new CostCalculatorUsingExchanges(taskCountEstimator);
@@ -381,7 +369,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig,
381369

382370
// add bogus connector for testing session properties
383371
catalogManager.registerCatalog(createBogusTestingCatalog(TESTING_CATALOG));
384-
metadata.getSessionPropertyManager().addConnectorSessionProperties(new CatalogName(TESTING_CATALOG), this.defaultSessionProperties.getOrDefault(TESTING_CATALOG, ImmutableList.of()));
372+
metadata.getSessionPropertyManager().addConnectorSessionProperties(new CatalogName(TESTING_CATALOG), defaultSessionProperties.getOrDefault(TESTING_CATALOG, ImmutableList.of()));
385373

386374
// rewrite session to use managed SessionPropertyMetadata
387375
this.defaultSession = new Session(
@@ -434,17 +422,6 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig,
434422
this.spillerFactory = new GenericSpillerFactory(singleStreamSpillerFactory);
435423
}
436424

437-
public static LocalQueryRunner queryRunnerWithInitialTransaction(Session defaultSession)
438-
{
439-
checkArgument(!defaultSession.getTransactionId().isPresent(), "Already in transaction!");
440-
return new LocalQueryRunner(defaultSession, new FeaturesConfig(), new NodeSpillConfig(), true, false);
441-
}
442-
443-
public static LocalQueryRunner queryRunnerWithFakeNodeCountForStats(Session defaultSession, int nodeCount)
444-
{
445-
return new LocalQueryRunner(defaultSession, new FeaturesConfig(), new NodeSpillConfig(), false, false, nodeCount, ImmutableMap.of());
446-
}
447-
448425
@Override
449426
public void close()
450427
{
@@ -877,4 +854,68 @@ private static List<TableScanNode> findTableScanNodes(PlanNode node)
877854
.where(TableScanNode.class::isInstance)
878855
.findAll();
879856
}
857+
858+
public static class Builder
859+
{
860+
private Session defaultSession;
861+
private FeaturesConfig featuresConfig = new FeaturesConfig();
862+
private NodeSpillConfig nodeSpillConfig = new NodeSpillConfig();
863+
private boolean initialTransaction;
864+
private boolean alwaysRevokeMemory;
865+
private Map<String, List<PropertyMetadata<?>>> defaultSessionProperties = ImmutableMap.of();
866+
private int nodeCountForStats;
867+
868+
private Builder(Session defaultSession)
869+
{
870+
this.defaultSession = requireNonNull(defaultSession, "defaultSession is null");
871+
}
872+
873+
public Builder withFeaturesConfig(FeaturesConfig featuresConfig)
874+
{
875+
this.featuresConfig = requireNonNull(featuresConfig, "featuresConfig is null");
876+
return this;
877+
}
878+
879+
public Builder withNodeSpillConfig(NodeSpillConfig nodeSpillConfig)
880+
{
881+
this.nodeSpillConfig = requireNonNull(nodeSpillConfig, "nodeSpillConfig is null");
882+
return this;
883+
}
884+
885+
public Builder withInitialTransaction()
886+
{
887+
this.initialTransaction = true;
888+
return this;
889+
}
890+
891+
public Builder withAlwaysRevokeMemory()
892+
{
893+
this.alwaysRevokeMemory = true;
894+
return this;
895+
}
896+
897+
public Builder withDefaultSessionProperties(Map<String, List<PropertyMetadata<?>>> defaultSessionProperties)
898+
{
899+
this.defaultSessionProperties = requireNonNull(defaultSessionProperties, "defaultSessionProperties is null");
900+
return this;
901+
}
902+
903+
public Builder withNodeCountForStats(int nodeCountForStats)
904+
{
905+
this.nodeCountForStats = nodeCountForStats;
906+
return this;
907+
}
908+
909+
public LocalQueryRunner build()
910+
{
911+
return new LocalQueryRunner(
912+
defaultSession,
913+
featuresConfig,
914+
nodeSpillConfig,
915+
initialTransaction,
916+
alwaysRevokeMemory,
917+
nodeCountForStats,
918+
defaultSessionProperties);
919+
}
920+
}
880921
}

Diff for: presto-main/src/test/java/io/prestosql/TestHiddenColumns.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ public class TestHiddenColumns
3232
@BeforeClass
3333
public void setUp()
3434
{
35-
runner = new LocalQueryRunner(TEST_SESSION);
35+
runner = LocalQueryRunner.create(TEST_SESSION);
3636
runner.createCatalog(TEST_SESSION.getCatalog().get(), new TpchConnectorFactory(1), ImmutableMap.of());
3737
}
3838

Diff for: presto-main/src/test/java/io/prestosql/cost/StatsCalculatorTester.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,7 @@ public Metadata getMetadata()
5959

6060
private static LocalQueryRunner createQueryRunner(Session session)
6161
{
62-
LocalQueryRunner queryRunner = new LocalQueryRunner(session);
62+
LocalQueryRunner queryRunner = LocalQueryRunner.create(session);
6363
queryRunner.createCatalog(session.getCatalog().get(),
6464
new TpchConnectorFactory(1),
6565
ImmutableMap.of());

Diff for: presto-main/src/test/java/io/prestosql/cost/TestCostCalculator.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -103,7 +103,7 @@ public void setUp()
103103

104104
session = testSessionBuilder().setCatalog("tpch").build();
105105

106-
localQueryRunner = new LocalQueryRunner(session);
106+
localQueryRunner = LocalQueryRunner.create(session);
107107
localQueryRunner.createCatalog("tpch", new TpchConnectorFactory(), ImmutableMap.of());
108108

109109
planFragmenter = new PlanFragmenter(localQueryRunner.getMetadata(), localQueryRunner.getNodePartitioningManager(), new QueryManagerConfig());

Diff for: presto-main/src/test/java/io/prestosql/cost/TestStatsCalculator.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ public class TestStatsCalculator
3535

3636
public TestStatsCalculator()
3737
{
38-
this.queryRunner = new LocalQueryRunner(testSessionBuilder()
38+
this.queryRunner = LocalQueryRunner.create(testSessionBuilder()
3939
.setCatalog("local")
4040
.setSchema("tiny")
4141
.setSystemProperty("task_concurrency", "1") // these tests don't handle exchanges from local parallel

Diff for: presto-main/src/test/java/io/prestosql/execution/TestPlannerWarnings.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -61,7 +61,7 @@ public class TestPlannerWarnings
6161
@BeforeClass
6262
public void setUp()
6363
{
64-
queryRunner = new LocalQueryRunner(testSessionBuilder()
64+
queryRunner = LocalQueryRunner.create(testSessionBuilder()
6565
.setCatalog("local")
6666
.setSchema("tiny")
6767
.build());

Diff for: presto-main/src/test/java/io/prestosql/memory/TestMemoryPools.java

+3-2
Original file line numberDiff line numberDiff line change
@@ -52,7 +52,6 @@
5252
import static io.airlift.units.DataSize.Unit.BYTE;
5353
import static io.airlift.units.DataSize.Unit.GIGABYTE;
5454
import static io.airlift.units.DataSize.Unit.MEGABYTE;
55-
import static io.prestosql.testing.LocalQueryRunner.queryRunnerWithInitialTransaction;
5655
import static io.prestosql.testing.TestingSession.testSessionBuilder;
5756
import static io.prestosql.testing.TestingTaskContext.createTaskContext;
5857
import static java.lang.String.format;
@@ -85,7 +84,9 @@ private void setUp(Supplier<List<Driver>> driversSupplier)
8584
.setSystemProperty("task_default_concurrency", "1")
8685
.build();
8786

88-
localQueryRunner = queryRunnerWithInitialTransaction(session);
87+
localQueryRunner = LocalQueryRunner.builder(session)
88+
.withInitialTransaction()
89+
.build();
8990

9091
// add tpch
9192
localQueryRunner.createCatalog("tpch", new TpchConnectorFactory(1), ImmutableMap.of());

Diff for: presto-main/src/test/java/io/prestosql/memory/TestQueryContext.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -73,7 +73,7 @@ public void testSetMemoryPool(boolean useReservedPool)
7373
assertTrue(reservedPool.reserve(secondQuery, "test", secondQueryMemory).isDone());
7474
}
7575

76-
try (LocalQueryRunner localQueryRunner = new LocalQueryRunner(TEST_SESSION)) {
76+
try (LocalQueryRunner localQueryRunner = LocalQueryRunner.create(TEST_SESSION)) {
7777
QueryContext queryContext = new QueryContext(
7878
new QueryId("query"),
7979
new DataSize(10, BYTE),

Diff for: presto-main/src/test/java/io/prestosql/operator/scalar/FunctionAssertions.java

+3-1
Original file line numberDiff line numberDiff line change
@@ -214,7 +214,9 @@ public FunctionAssertions(Session session)
214214
public FunctionAssertions(Session session, FeaturesConfig featuresConfig)
215215
{
216216
this.session = requireNonNull(session, "session is null");
217-
runner = new LocalQueryRunner(session, featuresConfig);
217+
runner = LocalQueryRunner.builder(session)
218+
.withFeaturesConfig(featuresConfig)
219+
.build();
218220
metadata = runner.getMetadata();
219221
compiler = runner.getExpressionCompiler();
220222
typeAnalyzer = new TypeAnalyzer(SQL_PARSER, metadata);

Diff for: presto-main/src/test/java/io/prestosql/operator/scalar/TestFailureFunction.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,7 @@ public void testQuery()
3939
{
4040
// The other test does not exercise this function during execution (i.e. inside a page processor).
4141
// It only verifies constant folding works.
42-
try (LocalQueryRunner runner = new LocalQueryRunner(TEST_SESSION)) {
42+
try (LocalQueryRunner runner = LocalQueryRunner.create(TEST_SESSION)) {
4343
runner.execute("select if(x, 78, 0/0) from (values rand() >= 0, rand() < 0) t(x)");
4444
}
4545
}

Diff for: presto-main/src/test/java/io/prestosql/operator/window/AbstractTestWindowFunction.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ public abstract class AbstractTestWindowFunction
3232
@BeforeClass
3333
public final void initTestWindowFunction()
3434
{
35-
queryRunner = new LocalQueryRunner(TEST_SESSION);
35+
queryRunner = LocalQueryRunner.create(TEST_SESSION);
3636
}
3737

3838
@AfterClass(alwaysRun = true)

Diff for: presto-main/src/test/java/io/prestosql/sql/planner/BenchmarkPlanner.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -86,7 +86,7 @@ public void setup()
8686
.setSystemProperty("iterative_optimizer_enabled", iterativeOptimizerEnabled)
8787
.build();
8888

89-
queryRunner = new LocalQueryRunner(session);
89+
queryRunner = LocalQueryRunner.create(session);
9090
queryRunner.createCatalog(tpch, new TpchConnectorFactory(4), ImmutableMap.of(TPCH_COLUMN_NAMING_PROPERTY, ColumnNaming.STANDARD.name()));
9191

9292
queries = IntStream.rangeClosed(1, 22)

Diff for: presto-main/src/test/java/io/prestosql/sql/planner/TestLocalExecutionPlanner.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@ public class TestLocalExecutionPlanner
3434
@BeforeClass
3535
public void setUp()
3636
{
37-
runner = new LocalQueryRunner(TEST_SESSION);
37+
runner = LocalQueryRunner.create(TEST_SESSION);
3838
}
3939

4040
@AfterClass(alwaysRun = true)

0 commit comments

Comments
 (0)