Skip to content

Commit

Permalink
Remove compiler warning for reserved identifier in future JDKs
Browse files Browse the repository at this point in the history
  • Loading branch information
wendigo authored and ebyhr committed Oct 7, 2021
1 parent 5bb9420 commit 81e90ce
Show file tree
Hide file tree
Showing 10 changed files with 33 additions and 33 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -200,7 +200,7 @@ public WorkProcessor<SerializedPage> pages()
return ProcessState.blocked(blocked);
}

return ProcessState.yield();
return ProcessState.yielded();
}

return ProcessState.ofResult(page);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import static io.trino.operator.WorkProcessor.ProcessState.blocked;
import static io.trino.operator.WorkProcessor.ProcessState.finished;
import static io.trino.operator.WorkProcessor.ProcessState.ofResult;
import static io.trino.operator.WorkProcessor.ProcessState.yield;
import static io.trino.operator.WorkProcessor.ProcessState.yielded;
import static java.util.Objects.requireNonNull;

/**
Expand Down Expand Up @@ -67,7 +67,7 @@ public WorkProcessor<Page> pages()
return ofResult(result);
}

return yield();
return yielded();
});
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -360,7 +360,7 @@ else if (finished) {
}
else {
outputMemoryContext.setBytes(pageBuilder.getRetainedSizeInBytes());
return ProcessState.yield();
return ProcessState.yielded();
}
}
}
Expand Down Expand Up @@ -395,7 +395,7 @@ public ProcessState<Page> process()
return ProcessState.finished();
}
else {
return ProcessState.yield();
return ProcessState.yielded();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -295,7 +295,7 @@ public ProcessState<Page> process()
return ProcessState.finished();
}
else {
return ProcessState.yield();
return ProcessState.yielded();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -177,7 +177,7 @@ interface Transformation<T, R>
* @return the current transformation state, optionally bearing a result
* @see TransformationState#needsMoreData()
* @see TransformationState#blocked(ListenableFuture)
* @see TransformationState#yield()
* @see TransformationState#yielded()
* @see TransformationState#ofResult(Object)
* @see TransformationState#ofResult(Object, boolean)
* @see TransformationState#finished()
Expand All @@ -192,7 +192,7 @@ interface Process<T>
*
* @return the current state, optionally bearing a result
* @see ProcessState#blocked(ListenableFuture)
* @see ProcessState#yield()
* @see ProcessState#yielded()
* @see ProcessState#ofResult(Object)
* @see ProcessState#finished()
*/
Expand Down Expand Up @@ -254,7 +254,7 @@ public static <T> TransformationState<T> blocked(ListenableFuture<Void> blocked)
* Signals that transformation has yielded. {@link #process()} will be called again with the same input element.
*/
@SuppressWarnings("unchecked")
public static <T> TransformationState<T> yield()
public static <T> TransformationState<T> yielded()
{
return (TransformationState<T>) YIELD_STATE;
}
Expand Down Expand Up @@ -349,7 +349,7 @@ public static <T> ProcessState<T> blocked(ListenableFuture<Void> blocked)
* Signals that process has yielded. {@link #process()} will be called again later.
*/
@SuppressWarnings("unchecked")
public static <T> ProcessState<T> yield()
public static <T> ProcessState<T> yielded()
{
return (ProcessState<T>) YIELD_STATE;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,7 @@ else if (processor.isBlocked()) {
return ProcessState.blocked(processor.getBlockedFuture());
}
else {
return ProcessState.yield();
return ProcessState.yielded();
}

if (processorIterator.hasNext()) {
Expand Down Expand Up @@ -174,7 +174,7 @@ public ProcessState<T> process()
{
if (!lastProcessYielded && yieldSignal.getAsBoolean()) {
lastProcessYielded = true;
return ProcessState.yield();
return ProcessState.yielded();
}
lastProcessYielded = false;

Expand Down Expand Up @@ -230,7 +230,7 @@ private static <T> ProcessState<T> getNextState(WorkProcessor<T> processor)
return ProcessState.blocked(processor.getBlockedFuture());
}

return ProcessState.yield();
return ProcessState.yielded();
}

static <T, R> WorkProcessor<R> flatMap(WorkProcessor<T> processor, Function<T, WorkProcessor<R>> mapper)
Expand Down Expand Up @@ -286,7 +286,7 @@ static <T> WorkProcessor<T> flatten(WorkProcessor<WorkProcessor<T>> processor)
return TransformationState.blocked(nestedProcessor.getBlockedFuture());
}

return TransformationState.yield();
return TransformationState.yielded();
});
}

Expand All @@ -312,7 +312,7 @@ else if (processor.isBlocked()) {
return ProcessState.blocked(processor.getBlockedFuture());
}
else {
return ProcessState.yield();
return ProcessState.yielded();
}
}

Expand All @@ -331,7 +331,7 @@ else if (processor.isBlocked()) {
case BLOCKED:
return ProcessState.blocked(state.getBlocked());
case YIELD:
return ProcessState.yield();
return ProcessState.yielded();
case RESULT:
return ProcessState.ofResult(state.getResult());
case FINISHED:
Expand All @@ -353,7 +353,7 @@ private static class ProcessWorkProcessor<T>
@Nullable
WorkProcessor.Process<T> process;
// set initial state to yield as it will cause processor computations to progress
ProcessState<T> state = ProcessState.yield();
ProcessState<T> state = ProcessState.yielded();

ProcessWorkProcessor(WorkProcessor.Process<T> process)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ public WorkProcessor<Page> pages()
return ProcessState.blocked(blocked);
}

return ProcessState.yield();
return ProcessState.yielded();
}

return ProcessState.ofResult(page);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@
import static io.trino.operator.WorkProcessor.TransformationState.finished;
import static io.trino.operator.WorkProcessor.TransformationState.needsMoreData;
import static io.trino.operator.WorkProcessor.TransformationState.ofResult;
import static io.trino.operator.WorkProcessor.TransformationState.yield;
import static io.trino.operator.WorkProcessor.TransformationState.yielded;
import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.FULL_OUTER;
import static io.trino.operator.join.LookupJoinOperatorFactory.JoinType.PROBE_OUTER;
import static io.trino.operator.join.PartitionedLookupSourceFactory.NO_SPILL_EPOCH;
Expand Down Expand Up @@ -205,7 +205,7 @@ else if (!spillInProgress.isDone()) {
return ofResult(buildOutputPage(), false);
}

return yield();
return yielded();
}

if (!pageBuilder.isEmpty() || finishing) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@
import static io.trino.operator.PageUtils.recordMaterializedBytes;
import static io.trino.operator.WorkProcessor.ProcessState.finished;
import static io.trino.operator.WorkProcessor.ProcessState.ofResult;
import static io.trino.operator.WorkProcessor.ProcessState.yield;
import static io.trino.operator.WorkProcessor.ProcessState.yielded;
import static io.trino.operator.project.SelectedPositions.positionsRange;
import static io.trino.spi.block.DictionaryId.randomDictionaryId;
import static java.util.Objects.requireNonNull;
Expand Down Expand Up @@ -213,7 +213,7 @@ public ProcessState<Page> process()
lastComputeYielded = true;
lastComputeBatchSize = batchSize;
updateRetainedSize();
return yield();
return yielded();
}

if (result.isPageTooLarge()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ public void testIterator()
public void testIteratorFailsWhenWorkProcessorHasYielded()
{
// iterator should fail if underlying work has yielded
WorkProcessor<Integer> processor = processorFrom(ImmutableList.of(ProcessState.yield()));
WorkProcessor<Integer> processor = processorFrom(ImmutableList.of(ProcessState.yielded()));
Iterator<Integer> iterator = processor.iterator();
//noinspection ResultOfMethodCallIgnored
assertThatThrownBy(iterator::hasNext)
Expand All @@ -90,7 +90,7 @@ public void testMergeSorted()
List<ProcessState<Integer>> firstStream = ImmutableList.of(
ProcessState.ofResult(1),
ProcessState.ofResult(3),
ProcessState.yield(),
ProcessState.yielded(),
ProcessState.ofResult(5),
ProcessState.finished());

Expand Down Expand Up @@ -139,7 +139,7 @@ public void testMergeSortedEmptyStreams()
SettableFuture<Void> firstFuture = SettableFuture.create();
List<ProcessState<Integer>> firstStream = ImmutableList.of(
ProcessState.blocked(firstFuture),
ProcessState.yield(),
ProcessState.yielded(),
ProcessState.finished());

SettableFuture<Void> secondFuture = SettableFuture.create();
Expand Down Expand Up @@ -240,7 +240,7 @@ public void testProcessStateMonitor()

List<ProcessState<Integer>> baseScenario = ImmutableList.of(
ProcessState.ofResult(1),
ProcessState.yield(),
ProcessState.yielded(),
ProcessState.blocked(future),
ProcessState.finished());

Expand All @@ -266,7 +266,7 @@ public void testFinished()

List<ProcessState<Integer>> scenario = ImmutableList.of(
ProcessState.ofResult(1),
ProcessState.yield(),
ProcessState.yielded(),
ProcessState.blocked(future),
ProcessState.ofResult(2));

Expand Down Expand Up @@ -326,15 +326,15 @@ public void testFlatTransform()
ProcessState.ofResult(1.0),
ProcessState.blocked(baseFuture),
ProcessState.ofResult(2.0),
ProcessState.yield(),
ProcessState.yielded(),
ProcessState.ofResult(3.0),
ProcessState.ofResult(4.0),
ProcessState.finished());

SettableFuture<Void> mappedFuture1 = SettableFuture.create();
List<ProcessState<Integer>> mappedScenario1 = ImmutableList.of(
ProcessState.ofResult(1),
ProcessState.yield(),
ProcessState.yielded(),
ProcessState.blocked(mappedFuture1),
ProcessState.ofResult(2),
ProcessState.finished());
Expand Down Expand Up @@ -411,7 +411,7 @@ public void testTransform()
SettableFuture<Void> baseFuture = SettableFuture.create();
List<ProcessState<Integer>> baseScenario = ImmutableList.of(
ProcessState.ofResult(1),
ProcessState.yield(),
ProcessState.yielded(),
ProcessState.blocked(baseFuture),
ProcessState.ofResult(2),
ProcessState.ofResult(3),
Expand All @@ -422,7 +422,7 @@ public void testTransform()
Transform.of(Optional.of(1), TransformationState.needsMoreData()),
Transform.of(Optional.of(2), TransformationState.ofResult("foo")),
Transform.of(Optional.of(3), TransformationState.blocked(transformationFuture)),
Transform.of(Optional.of(3), TransformationState.yield()),
Transform.of(Optional.of(3), TransformationState.yielded()),
Transform.of(Optional.of(3), TransformationState.ofResult("bar", false)),
Transform.of(Optional.of(3), TransformationState.ofResult("zoo", true)),
Transform.of(Optional.empty(), TransformationState.ofResult("car", false)),
Expand Down Expand Up @@ -474,10 +474,10 @@ public void testCreateFrom()
{
SettableFuture<Void> future = SettableFuture.create();
List<ProcessState<Integer>> scenario = ImmutableList.of(
ProcessState.yield(),
ProcessState.yielded(),
ProcessState.ofResult(1),
ProcessState.blocked(future),
ProcessState.yield(),
ProcessState.yielded(),
ProcessState.ofResult(2),
ProcessState.finished());
WorkProcessor<Integer> processor = processorFrom(scenario);
Expand Down

0 comments on commit 81e90ce

Please sign in to comment.