Skip to content

Commit cf3d6ed

Browse files
authored
[ErrorProne] Enable UnnecessaryParentheses validation and fix all violations (#37913)
* Enable UnnecessaryParentheses validation and fix all violations * Enable UnnecessaryParentheses validation and fix all violations * fix avro format function
1 parent 79c52af commit cf3d6ed

File tree

49 files changed

+75
-80
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

49 files changed

+75
-80
lines changed

buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1551,7 +1551,6 @@ class BeamModulePlugin implements Plugin<Project> {
15511551
"ThreadPriorityCheck",
15521552
"UndefinedEquals",
15531553
"UnescapedEntity",
1554-
"UnnecessaryParentheses",
15551554
"UnrecognisedJavadocTag",
15561555
// errorprone 3.2.0+ checks
15571556
"DirectInvocationOnMock",

it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryIOLT.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -244,7 +244,7 @@ public void testWriteAndRead() throws IOException {
244244
.withAvroFormatFunction(
245245
new AvroFormatFn(
246246
configuration.numColumns,
247-
!("STORAGE_WRITE_API".equalsIgnoreCase(configuration.writeMethod))));
247+
!"STORAGE_WRITE_API".equalsIgnoreCase(configuration.writeMethod)));
248248
break;
249249
case JSON:
250250
writeIO =

runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -302,7 +302,7 @@ private BundleOutputManager(Map<TupleTag<?>, UncommittedBundle<?>> bundles) {
302302
@Override
303303
public <OutputT> void output(TupleTag<OutputT> tag, WindowedValue<OutputT> output) {
304304
checkArgument(bundles.containsKey(tag), "Unknown output tag %s", tag);
305-
(bundles.get(tag)).add((WindowedValue) output);
305+
bundles.get(tag).add((WindowedValue) output);
306306
}
307307
}
308308
}

runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -128,7 +128,7 @@ public void start() {
128128
pipelineOptions.as(FlinkPipelineOptions.class).getAutoWatermarkInterval();
129129
if (watermarkInterval == null) {
130130
watermarkInterval =
131-
(pipelineOptions.as(FlinkPipelineOptions.class).getMaxBundleTimeMills()) / 5L;
131+
pipelineOptions.as(FlinkPipelineOptions.class).getMaxBundleTimeMills() / 5L;
132132
watermarkInterval =
133133
(watermarkInterval > MIN_WATERMARK_EMIT_INTERVAL_MS)
134134
? watermarkInterval

runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -287,7 +287,7 @@ private void testTriggerCombinerLiftingDisabled(Trigger trigger) throws Exceptio
287287

288288
@Test
289289
public void testRepeatedCountTriggerDisablesCombinerLifting() throws IOException, Exception {
290-
testTriggerCombinerLiftingDisabled(Repeatedly.forever((AfterPane.elementCountAtLeast(1))));
290+
testTriggerCombinerLiftingDisabled(Repeatedly.forever(AfterPane.elementCountAtLeast(1)));
291291
}
292292

293293
@Test

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -478,7 +478,7 @@ private synchronized void recordActiveMessageInProcessingTimesMap() {
478478
return;
479479
}
480480
int processingTime =
481-
(int) (this.activeMessageMetadata.stopwatch().elapsed(TimeUnit.MILLISECONDS));
481+
(int) this.activeMessageMetadata.stopwatch().elapsed(TimeUnit.MILLISECONDS);
482482
this.processingTimesByStep.compute(
483483
this.activeMessageMetadata.userStepName(),
484484
(k, v) -> {

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntry.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -96,10 +96,10 @@ public boolean equals(@Nullable Object o) {
9696
}
9797
if (o instanceof ShuffleEntry) {
9898
ShuffleEntry that = (ShuffleEntry) o;
99-
return (Objects.equals(this.position, that.position))
100-
&& (Objects.equals(this.key, that.key))
101-
&& (Objects.equals(this.secondaryKey, that.secondaryKey))
102-
&& (Objects.equals(this.value, that.value));
99+
return Objects.equals(this.position, that.position)
100+
&& Objects.equals(this.key, that.key)
101+
&& Objects.equals(this.secondaryKey, that.secondaryKey)
102+
&& Objects.equals(this.value, that.value);
103103
}
104104
return false;
105105
}

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/getdata/ApplianceGetDataClient.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -104,7 +104,7 @@ private void issueReadBatch(ReadBatch batch) {
104104
} catch (InterruptedException e) {
105105
// We don't expect this thread to be interrupted. To simplify handling, we just fall through
106106
// to issuing the call.
107-
assert (false);
107+
assert false;
108108
Thread.currentThread().interrupt();
109109
} catch (ExecutionException e) {
110110
// startRead is a SettableFuture so this should never occur.
@@ -185,7 +185,7 @@ private void issueReadBatch(ReadBatch batch) {
185185
synchronized (this) {
186186
ReadBatch batch;
187187
if (activeReadThreads < MAX_ACTIVE_READS) {
188-
assert (pendingReadBatches.isEmpty());
188+
assert pendingReadBatches.isEmpty();
189189
activeReadThreads += 1;
190190
// fall through to below synchronized block
191191
} else if (pendingReadBatches.isEmpty()

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMultimap.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -548,7 +548,7 @@ private Map<Object, Triple<K, Boolean, ConcatIterables<V>>> mergedCachedEntries(
548548
keyStateMap
549549
.entrySet()
550550
.removeIf(
551-
(entry -> {
551+
entry -> {
552552
Object structuralKey = entry.getKey();
553553
KeyState keyState = entry.getValue();
554554
if (complete && keyState.existence == KeyExistence.KNOWN_EXIST) {
@@ -574,7 +574,7 @@ private Map<Object, Triple<K, Boolean, ConcatIterables<V>>> mergedCachedEntries(
574574
return (keyState.existence == KeyExistence.KNOWN_NONEXISTENT
575575
&& !keyState.removedLocally)
576576
|| keyState.existence == KeyExistence.UNKNOWN_EXISTENCE;
577-
}));
577+
});
578578
return cachedEntries;
579579
}
580580

runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactoryTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -222,7 +222,7 @@ private <K, InputT, AccumT, OutputT> ParDoFn createCombineValuesFn(
222222
public void testCombineValuesFnAll() throws Exception {
223223
TestReceiver receiver = new TestReceiver();
224224

225-
Combine.CombineFn<Integer, CountSum, String> combiner = (new MeanInts());
225+
Combine.CombineFn<Integer, CountSum, String> combiner = new MeanInts();
226226

227227
ParDoFn combineParDoFn =
228228
createCombineValuesFn(

0 commit comments

Comments
 (0)