Skip to content

Commit ddba4ba

Browse files
shelton408facebook-github-bot
authored andcommitted
Remove DeleteScanInfo (#24857)
Summary: More details in #24807. Remove DeleteScanInfo because after the beginDelete update, this returns the same information as the tableScanNode tablehandle. Instead, we just pull the tablehandle directly from the tableScanNode. This diff removes the usage of TableWriteInfo.DeleteScanInfo, the supporting code, and the protocol. Protocol has also been remade. One concern I have is that I didn't see any usages of DeleteScanInfo in the native workers. If there's no usages, the protocol change won't have any impact, but I'm unsure if there might be something I missed on that end. For Reviewer: Only significant changes are in TableWriteInfo, SplitSourceFactory, and LocalExecutionPlanner. Everthing else is either adjusting test code for parameter changes, or protocol generation Details in #24807 If there is a user that used DeleteScanInfo to return specific tablescan information they required, that functionality will no longer be available. In open source connectors and meta internal code, there doesn't seem to be any impact. Test Plan: Test on IcebergHiveIntegrationSmokeTest for deletes. TODO: test with meta internal DELETE build - [ ] Please make sure your submission complies with our [contributing guide](https://github.com/prestodb/presto/blob/master/CONTRIBUTING.md), in particular [code style](https://github.com/prestodb/presto/blob/master/CONTRIBUTING.md#code-style) and [commit standards](https://github.com/prestodb/presto/blob/master/CONTRIBUTING.md#commit-standards). - [ ] PR description addresses the issue accurately and concisely. If the change is non-trivial, a GitHub Issue is referenced. - [ ] Documented new properties (with its default value), SQL syntax, functions, or other functionality. - [ ] If release notes are required, they follow the [release notes guidelines](https://github.com/prestodb/presto/wiki/Release-Notes-Guidelines). - [ ] Adequate tests were added if applicable. - [ ] CI passed. Please follow [release notes guidelines](https://github.com/prestodb/presto/wiki/Release-Notes-Guidelines) and fill in the release notes below. ``` == RELEASE NOTES == General Changes * ... * ... Hive Connector Changes * ... * ... ``` If release note is NOT required, use: ``` == NO RELEASE NOTE == ``` Differential Revision: D72352854 Pulled By: shelton408
1 parent a972bba commit ddba4ba

File tree

19 files changed

+141
-317
lines changed

19 files changed

+141
-317
lines changed

presto-main-base/src/main/java/com/facebook/presto/execution/scheduler/TableWriteInfo.java

Lines changed: 4 additions & 109 deletions
Original file line numberDiff line numberDiff line change
@@ -15,29 +15,16 @@
1515
package com.facebook.presto.execution.scheduler;
1616

1717
import com.facebook.presto.Session;
18-
import com.facebook.presto.common.predicate.TupleDomain;
1918
import com.facebook.presto.metadata.AnalyzeTableHandle;
2019
import com.facebook.presto.metadata.Metadata;
21-
import com.facebook.presto.metadata.TableLayoutResult;
22-
import com.facebook.presto.spi.ColumnHandle;
23-
import com.facebook.presto.spi.Constraint;
24-
import com.facebook.presto.spi.TableHandle;
25-
import com.facebook.presto.spi.plan.DeleteNode;
26-
import com.facebook.presto.spi.plan.FilterNode;
27-
import com.facebook.presto.spi.plan.JoinNode;
2820
import com.facebook.presto.spi.plan.PlanNode;
29-
import com.facebook.presto.spi.plan.PlanNodeId;
30-
import com.facebook.presto.spi.plan.ProjectNode;
31-
import com.facebook.presto.spi.plan.SemiJoinNode;
3221
import com.facebook.presto.spi.plan.TableFinishNode;
33-
import com.facebook.presto.spi.plan.TableScanNode;
3422
import com.facebook.presto.spi.plan.TableWriterNode;
3523
import com.facebook.presto.sql.planner.optimizations.PlanNodeSearcher;
3624
import com.facebook.presto.sql.planner.plan.StatisticsWriterNode;
3725
import com.fasterxml.jackson.annotation.JsonCreator;
3826
import com.fasterxml.jackson.annotation.JsonProperty;
3927
import com.google.common.base.VerifyException;
40-
import com.google.common.collect.ImmutableSet;
4128

4229
import java.util.Collection;
4330
import java.util.List;
@@ -55,35 +42,29 @@ public class TableWriteInfo
5542
{
5643
private final Optional<ExecutionWriterTarget> writerTarget;
5744
private final Optional<AnalyzeTableHandle> analyzeTableHandle;
58-
private final Optional<DeleteScanInfo> deleteScanInfo;
5945

6046
@JsonCreator
6147
public TableWriteInfo(
6248
@JsonProperty("writerTarget") Optional<ExecutionWriterTarget> writerTarget,
63-
@JsonProperty("analyzeTableHandle") Optional<AnalyzeTableHandle> analyzeTableHandle,
64-
@JsonProperty("deleteScanInfo") Optional<DeleteScanInfo> deleteScanInfo)
49+
@JsonProperty("analyzeTableHandle") Optional<AnalyzeTableHandle> analyzeTableHandle)
6550
{
6651
this.writerTarget = requireNonNull(writerTarget, "writerTarget is null");
6752
this.analyzeTableHandle = requireNonNull(analyzeTableHandle, "analyzeTableHandle is null");
68-
this.deleteScanInfo = requireNonNull(deleteScanInfo, "deleteScanInfo is null");
69-
checkArgument(!analyzeTableHandle.isPresent() || !writerTarget.isPresent() && !deleteScanInfo.isPresent(), "analyzeTableHandle is present, so no other fields should be present");
70-
checkArgument(!deleteScanInfo.isPresent() || writerTarget.isPresent(), "deleteScanInfo is present, but writerTarget is not present");
53+
checkArgument(!analyzeTableHandle.isPresent() || !writerTarget.isPresent(), "analyzeTableHandle is present, so no other fields should be present");
7154
}
7255

7356
public static TableWriteInfo createTableWriteInfo(StreamingSubPlan plan, Metadata metadata, Session session)
7457
{
7558
Optional<ExecutionWriterTarget> writerTarget = createWriterTarget(plan, metadata, session);
7659
Optional<AnalyzeTableHandle> analyzeTableHandle = createAnalyzeTableHandle(plan, metadata, session);
77-
Optional<DeleteScanInfo> deleteScanInfo = createDeleteScanInfo(plan, writerTarget, metadata, session);
78-
return new TableWriteInfo(writerTarget, analyzeTableHandle, deleteScanInfo);
60+
return new TableWriteInfo(writerTarget, analyzeTableHandle);
7961
}
8062

8163
public static TableWriteInfo createTableWriteInfo(PlanNode planNode, Metadata metadata, Session session)
8264
{
8365
Optional<ExecutionWriterTarget> writerTarget = createWriterTarget(planNode, metadata, session);
8466
Optional<AnalyzeTableHandle> analyzeTableHandle = createAnalyzeTableHandle(planNode, metadata, session);
85-
Optional<DeleteScanInfo> deleteScanInfo = createDeleteScanInfo(planNode, writerTarget, metadata, session);
86-
return new TableWriteInfo(writerTarget, analyzeTableHandle, deleteScanInfo);
67+
return new TableWriteInfo(writerTarget, analyzeTableHandle);
8768
}
8869

8970
private static Optional<ExecutionWriterTarget> createWriterTarget(Optional<TableFinishNode> finishNodeOptional, Metadata metadata, Session session)
@@ -141,37 +122,6 @@ private static Optional<AnalyzeTableHandle> createAnalyzeTableHandle(Optional<St
141122
return statisticsWriterNodeOptional.map(node -> metadata.beginStatisticsCollection(session, node.getTableHandle()));
142123
}
143124

144-
private static Optional<DeleteScanInfo> createDeleteScanInfo(StreamingSubPlan plan, Optional<ExecutionWriterTarget> writerTarget, Metadata metadata, Session session)
145-
{
146-
if (writerTarget.isPresent() && writerTarget.get() instanceof ExecutionWriterTarget.DeleteHandle) {
147-
DeleteNode delete = getOnlyElement(findPlanNodes(plan, DeleteNode.class));
148-
return createDeleteScanInfo(delete, metadata, session);
149-
}
150-
return Optional.empty();
151-
}
152-
153-
private static Optional<DeleteScanInfo> createDeleteScanInfo(PlanNode planNode, Optional<ExecutionWriterTarget> writerTarget, Metadata metadata, Session session)
154-
{
155-
if (writerTarget.isPresent() && writerTarget.get() instanceof ExecutionWriterTarget.DeleteHandle) {
156-
DeleteNode delete = findSinglePlanNode(planNode, DeleteNode.class).get();
157-
return createDeleteScanInfo(delete, metadata, session);
158-
}
159-
return Optional.empty();
160-
}
161-
162-
private static Optional<DeleteScanInfo> createDeleteScanInfo(DeleteNode delete, Metadata metadata, Session session)
163-
{
164-
TableScanNode tableScan = getDeleteTableScan(delete);
165-
TupleDomain<ColumnHandle> originalEnforcedConstraint = tableScan.getEnforcedConstraint();
166-
TableLayoutResult layoutResult = metadata.getLayout(
167-
session,
168-
tableScan.getTable(),
169-
new Constraint<>(originalEnforcedConstraint),
170-
Optional.of(ImmutableSet.copyOf(tableScan.getAssignments().values())));
171-
172-
return Optional.of(new DeleteScanInfo(tableScan.getId(), layoutResult.getLayout().getNewTableHandle()));
173-
}
174-
175125
private static <T extends PlanNode> Optional<T> findSinglePlanNode(PlanNode planNode, Class<T> clazz)
176126
{
177127
return PlanNodeSearcher
@@ -203,30 +153,6 @@ private static <T extends PlanNode> List<T> findPlanNodes(StreamingSubPlan plan,
203153
.collect(toImmutableList());
204154
}
205155

206-
private static TableScanNode getDeleteTableScan(PlanNode node)
207-
{
208-
if (node instanceof TableScanNode) {
209-
return (TableScanNode) node;
210-
}
211-
if (node instanceof DeleteNode) {
212-
return getDeleteTableScan(((DeleteNode) node).getSource());
213-
}
214-
if (node instanceof FilterNode) {
215-
return getDeleteTableScan(((FilterNode) node).getSource());
216-
}
217-
if (node instanceof ProjectNode) {
218-
return getDeleteTableScan(((ProjectNode) node).getSource());
219-
}
220-
if (node instanceof SemiJoinNode) {
221-
return getDeleteTableScan(((SemiJoinNode) node).getSource());
222-
}
223-
if (node instanceof JoinNode) {
224-
JoinNode joinNode = (JoinNode) node;
225-
return getDeleteTableScan(joinNode.getLeft());
226-
}
227-
throw new IllegalArgumentException("Invalid descendant for DeleteNode: " + node.getClass().getName());
228-
}
229-
230156
@JsonProperty
231157
public Optional<ExecutionWriterTarget> getWriterTarget()
232158
{
@@ -238,35 +164,4 @@ public Optional<AnalyzeTableHandle> getAnalyzeTableHandle()
238164
{
239165
return analyzeTableHandle;
240166
}
241-
242-
@JsonProperty
243-
public Optional<DeleteScanInfo> getDeleteScanInfo()
244-
{
245-
return deleteScanInfo;
246-
}
247-
248-
public static class DeleteScanInfo
249-
{
250-
private final PlanNodeId id;
251-
private final TableHandle tableHandle;
252-
253-
@JsonCreator
254-
public DeleteScanInfo(@JsonProperty("id") PlanNodeId id, @JsonProperty("tableHandle") TableHandle tableHandle)
255-
{
256-
this.id = id;
257-
this.tableHandle = tableHandle;
258-
}
259-
260-
@JsonProperty
261-
public PlanNodeId getId()
262-
{
263-
return id;
264-
}
265-
266-
@JsonProperty
267-
public TableHandle getTableHandle()
268-
{
269-
return tableHandle;
270-
}
271-
}
272167
}

presto-main-base/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java

Lines changed: 2 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,6 @@
4040
import com.facebook.presto.execution.scheduler.ExecutionWriterTarget.RefreshMaterializedViewHandle;
4141
import com.facebook.presto.execution.scheduler.ExecutionWriterTarget.UpdateHandle;
4242
import com.facebook.presto.execution.scheduler.TableWriteInfo;
43-
import com.facebook.presto.execution.scheduler.TableWriteInfo.DeleteScanInfo;
4443
import com.facebook.presto.expressions.DynamicFilters;
4544
import com.facebook.presto.expressions.DynamicFilters.DynamicFilterExtractResult;
4645
import com.facebook.presto.expressions.DynamicFilters.DynamicFilterPlaceholder;
@@ -1453,13 +1452,7 @@ private PhysicalOperation visitScanFilterAndProject(
14531452
PhysicalOperation source = null;
14541453
if (sourceNode instanceof TableScanNode && locality.equals(LOCAL)) {
14551454
TableScanNode tableScanNode = (TableScanNode) sourceNode;
1456-
Optional<DeleteScanInfo> deleteScanInfo = context.getTableWriteInfo().getDeleteScanInfo();
1457-
if (deleteScanInfo.isPresent() && deleteScanInfo.get().getId() == tableScanNode.getId()) {
1458-
table = deleteScanInfo.get().getTableHandle();
1459-
}
1460-
else {
1461-
table = tableScanNode.getTable();
1462-
}
1455+
table = tableScanNode.getTable();
14631456

14641457
// extract the column handles and channel to type mapping
14651458
sourceLayout = new LinkedHashMap<>();
@@ -1613,14 +1606,7 @@ public PhysicalOperation visitTableScan(TableScanNode node, LocalExecutionPlanCo
16131606
columns.add(node.getAssignments().get(variable));
16141607
}
16151608

1616-
TableHandle tableHandle;
1617-
Optional<DeleteScanInfo> deleteScanInfo = context.getTableWriteInfo().getDeleteScanInfo();
1618-
if (deleteScanInfo.isPresent() && deleteScanInfo.get().getId() == node.getId()) {
1619-
tableHandle = deleteScanInfo.get().getTableHandle();
1620-
}
1621-
else {
1622-
tableHandle = node.getTable();
1623-
}
1609+
TableHandle tableHandle = node.getTable();
16241610
OperatorFactory operatorFactory = new TableScanOperatorFactory(context.getNextOperatorId(), node.getId(), pageSourceProvider, tableHandle, columns);
16251611
return new PhysicalOperation(operatorFactory, makeLayout(node), context, stageExecutionDescriptor.isScanGroupedExecution(node.getId()) ? GROUPED_EXECUTION : UNGROUPED_EXECUTION);
16261612
}

presto-main-base/src/main/java/com/facebook/presto/sql/planner/SplitSourceFactory.java

Lines changed: 1 addition & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,6 @@
1616
import com.facebook.airlift.log.Logger;
1717
import com.facebook.presto.Session;
1818
import com.facebook.presto.execution.scheduler.TableWriteInfo;
19-
import com.facebook.presto.execution.scheduler.TableWriteInfo.DeleteScanInfo;
2019
import com.facebook.presto.spi.TableHandle;
2120
import com.facebook.presto.spi.WarningCollector;
2221
import com.facebook.presto.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy;
@@ -67,7 +66,6 @@
6766

6867
import java.util.List;
6968
import java.util.Map;
70-
import java.util.Optional;
7169
import java.util.function.Supplier;
7270

7371
import static com.facebook.presto.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.GROUPED_SCHEDULING;
@@ -146,14 +144,7 @@ public Map<PlanNodeId, SplitSource> visitExplainAnalyze(ExplainAnalyzeNode node,
146144
public Map<PlanNodeId, SplitSource> visitTableScan(TableScanNode node, Context context)
147145
{
148146
// get dataSource for table
149-
TableHandle table;
150-
Optional<DeleteScanInfo> deleteScanInfo = context.getTableWriteInfo().getDeleteScanInfo();
151-
if (deleteScanInfo.isPresent() && deleteScanInfo.get().getId() == node.getId()) {
152-
table = deleteScanInfo.get().getTableHandle();
153-
}
154-
else {
155-
table = node.getTable();
156-
}
147+
TableHandle table = node.getTable();
157148
Supplier<SplitSource> splitSourceSupplier = () -> splitSourceProvider.getSplits(
158149
session,
159150
table,

presto-main-base/src/test/java/com/facebook/presto/execution/MockRemoteTaskFactory.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -146,7 +146,7 @@ public MockRemoteTask createTableScanTask(TaskId taskId, InternalNode newNode, L
146146
createInitialEmptyOutputBuffers(BROADCAST),
147147
nodeStatsTracker,
148148
true,
149-
new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty()),
149+
new TableWriteInfo(Optional.empty(), Optional.empty()),
150150
SchedulerStatsTracker.NOOP);
151151
}
152152

presto-main-base/src/test/java/com/facebook/presto/execution/TaskTestUtils.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -196,7 +196,7 @@ public static LocalExecutionPlanner createTestingPlanner()
196196

197197
public static TaskInfo updateTask(SqlTask sqlTask, List<TaskSource> taskSources, OutputBuffers outputBuffers)
198198
{
199-
return sqlTask.updateTask(TEST_SESSION, Optional.of(PLAN_FRAGMENT), taskSources, outputBuffers, Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty())));
199+
return sqlTask.updateTask(TEST_SESSION, Optional.of(PLAN_FRAGMENT), taskSources, outputBuffers, Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty())));
200200
}
201201

202202
public static SplitMonitor createTestSplitMonitor()

presto-main-base/src/test/java/com/facebook/presto/execution/TestSqlStageExecution.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -109,7 +109,7 @@ private void testFinalStageInfoInternal()
109109
executor,
110110
new NoOpFailureDetector(),
111111
new SplitSchedulerStats(),
112-
new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty()));
112+
new TableWriteInfo(Optional.empty(), Optional.empty()));
113113
stage.setOutputBuffers(createInitialEmptyOutputBuffers(ARBITRARY));
114114

115115
// add listener that fetches stage info when the final status is available

presto-main-base/src/test/java/com/facebook/presto/execution/TestSqlTaskManager.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -349,7 +349,7 @@ private TaskInfo createTask(SqlTaskManager sqlTaskManager, TaskId taskId, Immuta
349349
Optional.of(PLAN_FRAGMENT),
350350
ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, splits, true)),
351351
outputBuffers,
352-
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty())));
352+
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty())));
353353
}
354354

355355
private TaskInfo createTask(SqlTaskManager sqlTaskManager, TaskId taskId, OutputBuffers outputBuffers)
@@ -370,7 +370,7 @@ private TaskInfo createTask(SqlTaskManager sqlTaskManager, TaskId taskId, Output
370370
Optional.of(PLAN_FRAGMENT),
371371
ImmutableList.of(),
372372
outputBuffers,
373-
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty())));
373+
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty())));
374374
}
375375

376376
public static class MockExchangeClientSupplier

presto-main-base/src/test/java/com/facebook/presto/execution/scheduler/TestAdaptivePhasedExecutionPolicy.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -143,7 +143,7 @@ private StageExecutionAndScheduler getStageExecutionAndScheduler(int stage, Plan
143143
newDirectExecutorService(),
144144
new NoOpFailureDetector(),
145145
new SplitSchedulerStats(),
146-
new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty()));
146+
new TableWriteInfo(Optional.empty(), Optional.empty()));
147147
StageLinkage stageLinkage = new StageLinkage(fragmentId, (id, tasks, noMoreExchangeLocations) -> {}, ImmutableSet.of());
148148
StageScheduler stageScheduler = new FixedCountScheduler(stageExecution, ImmutableList.of());
149149
StageExecutionAndScheduler scheduler = new StageExecutionAndScheduler(stageExecution, stageLinkage, stageScheduler);

presto-main-base/src/test/java/com/facebook/presto/execution/scheduler/TestSourcePartitionedScheduler.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -546,7 +546,7 @@ private SqlStageExecution createSqlStageExecution(SubPlan tableScanPlan, NodeTas
546546
queryExecutor,
547547
new NoOpFailureDetector(),
548548
new SplitSchedulerStats(),
549-
new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty()));
549+
new TableWriteInfo(Optional.empty(), Optional.empty()));
550550

551551
stage.setOutputBuffers(createInitialEmptyOutputBuffers(PARTITIONED)
552552
.withBuffer(OUT, 0)

presto-main-base/src/test/java/com/facebook/presto/sql/planner/TestLocalExecutionPlanner.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -199,7 +199,7 @@ private LocalExecutionPlan getLocalExecutionPlan(Session session, PlanNode plan,
199199
testFragment,
200200
new TestingOutputBuffer(),
201201
new TestingRemoteSourceFactory(),
202-
new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty()),
202+
new TableWriteInfo(Optional.empty(), Optional.empty()),
203203
customPlanTranslators);
204204
}
205205

@@ -218,7 +218,7 @@ private LocalExecutionPlan getLocalExecutionPlan(Session session)
218218
leafFragment,
219219
new TestingOutputBuffer(),
220220
new TestingRemoteSourceFactory(),
221-
new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty()));
221+
new TableWriteInfo(Optional.empty(), Optional.empty()));
222222
}
223223

224224
private static class CustomNodeA

presto-main/src/test/java/com/facebook/presto/execution/TestSqlTask.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -123,7 +123,7 @@ public void testEmptyQuery()
123123
ImmutableList.of(),
124124
createInitialEmptyOutputBuffers(PARTITIONED)
125125
.withNoMoreBufferIds(),
126-
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty())));
126+
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty())));
127127
assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING);
128128

129129
taskInfo = sqlTask.getTaskInfo();
@@ -134,7 +134,7 @@ public void testEmptyQuery()
134134
ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(), true)),
135135
createInitialEmptyOutputBuffers(PARTITIONED)
136136
.withNoMoreBufferIds(),
137-
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty())));
137+
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty())));
138138
assertEquals(taskInfo.getTaskStatus().getState(), TaskState.FINISHED);
139139

140140
taskInfo = sqlTask.getTaskInfo();
@@ -151,7 +151,7 @@ public void testSimpleQuery()
151151
Optional.of(PLAN_FRAGMENT),
152152
ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(SPLIT), true)),
153153
createInitialEmptyOutputBuffers(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds(),
154-
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty())));
154+
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty())));
155155
assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING);
156156

157157
taskInfo = sqlTask.getTaskInfo();
@@ -201,7 +201,7 @@ public void testCancel()
201201
createInitialEmptyOutputBuffers(PARTITIONED)
202202
.withBuffer(OUT, 0)
203203
.withNoMoreBufferIds(),
204-
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty())));
204+
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty())));
205205
assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING);
206206
assertEquals(taskInfo.getStats().getEndTimeInMillis(), 0);
207207

@@ -228,7 +228,7 @@ public void testAbort()
228228
Optional.of(PLAN_FRAGMENT),
229229
ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(SPLIT), true)),
230230
createInitialEmptyOutputBuffers(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds(),
231-
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty())));
231+
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty())));
232232
assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING);
233233

234234
taskInfo = sqlTask.getTaskInfo();

presto-main/src/test/java/com/facebook/presto/memory/TestHighMemoryTaskKiller.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -128,7 +128,7 @@ public void updateTaskMemory(SqlTask sqlTask, long systemMemory)
128128
ImmutableList.of(),
129129
createInitialEmptyOutputBuffers(PARTITIONED)
130130
.withNoMoreBufferIds(),
131-
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty(), Optional.empty())));
131+
Optional.of(new TableWriteInfo(Optional.empty(), Optional.empty())));
132132
assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING);
133133

134134
TaskContext taskContext = sqlTask.getTaskContext().get();

0 commit comments

Comments
 (0)