This is an automated email from the ASF dual-hosted git repository.
kenn pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/master by this push:
new 9829d6d3cf9 Merge pull request #37792: [ErrorProne] Fix
AutoValueBoxedValues warnings across the codebase
9829d6d3cf9 is described below
commit 9829d6d3cf9ba246bcac70850a600b01df00431e
Author: Radosław Stankiewicz <[email protected]>
AuthorDate: Wed Mar 11 15:36:12 2026 +0100
Merge pull request #37792: [ErrorProne] Fix AutoValueBoxedValues warnings
across the codebase
---
.../org/apache/beam/gradle/BeamModulePlugin.groovy | 1 -
.../datatokenization/utils/CsvConverters.java | 6 +-
.../clickhouse/conditions/ClickHouseRowsCheck.java | 4 +-
.../datadog/conditions/DatadogLogEntriesCheck.java | 4 +-
.../gcp/bigquery/conditions/BigQueryRowsCheck.java | 4 +-
.../gcp/pubsub/conditions/PubsubMessagesCheck.java | 4 +-
.../beam/it/gcp/bigquery/BigQueryStreamingLT.java | 8 +-
.../apache/beam/it/gcp/bigtable/BigTableIOLT.java | 8 +-
.../java/org/apache/beam/it/kafka/KafkaIOLT.java | 8 +-
.../mongodb/conditions/MongoDBDocumentsCheck.java | 4 +-
.../it/splunk/conditions/SplunkEventsCheck.java | 4 +-
.../runners/fnexecution/wire/CommonCoderTest.java | 2 +-
.../GenerateSequenceSchemaTransformProvider.java | 12 +-
.../java/org/apache/beam/sdk/schemas/Schema.java | 10 +-
.../apache/beam/sdk/schemas/transforms/Group.java | 4 +-
.../beam/sdk/schemas/AutoValueSchemaTest.java | 4 +-
.../apache/beam/sdk/schemas/SchemaCoderTest.java | 6 +-
.../beam/sdk/schemas/transforms/SelectTest.java | 8 +-
.../TypedSchemaTransformProviderTest.java | 4 +-
.../beam/sdk/transforms/ParDoSchemaTest.java | 8 +-
.../org/apache/beam/sdk/transforms/ToJsonTest.java | 6 +-
.../expansion/service/ExpansionServiceTest.java | 2 +-
.../extensions/avro/schemas/utils/AvroUtils.java | 2 +-
.../beam/sdk/extensions/ml/DLPDeidentifyText.java | 4 +-
.../beam/sdk/extensions/ml/DLPInspectText.java | 4 +-
.../beam/sdk/extensions/ml/DLPReidentifyText.java | 4 +-
.../ml/RecommendationAIImportCatalogItems.java | 6 +-
.../ml/RecommendationAIImportUserEvents.java | 6 +-
.../extensions/sql/impl/parser/SqlAlterTable.java | 2 +-
.../sql/impl/parser/SqlCreateExternalTable.java | 2 +-
.../beam/sdk/extensions/timeseries/FillGaps.java | 6 +-
.../zetasketch/ApproximateCountDistinct.java | 8 +-
.../sdk/io/aws2/dynamodb/testing/DynamoDBIOIT.java | 2 +-
.../sdk/io/aws2/kinesis/KinesisIOWriteTest.java | 10 +-
.../apache/beam/sdk/io/cdap/TestRowDBWritable.java | 6 +-
.../beam/sdk/io/common/SchemaAwareJavaBeans.java | 36 ++---
.../org/apache/beam/sdk/io/common/TestRow.java | 10 +-
.../beam/sdk/io/datadog/DatadogEventPublisher.java | 10 +-
.../DebeziumReadSchemaTransformProvider.java | 4 +-
.../beam/sdk/io/elasticsearch/ElasticsearchIO.java | 2 +-
.../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 44 +++---
.../sdk/io/gcp/bigquery/BigQueryIOTranslation.java | 40 ++---
.../beam/sdk/io/gcp/bigquery/WritePartition.java | 2 +-
.../beam/sdk/io/gcp/bigquery/WriteTables.java | 2 +-
.../apache/beam/sdk/io/gcp/spanner/SpannerIO.java | 8 +-
.../beam/sdk/io/gcp/testing/BigqueryMatcher.java | 4 +-
.../beam/sdk/io/gcp/bigtable/BigtableIOTest.java | 16 +-
.../sdk/io/hadoop/format/TestRowDBWritable.java | 6 +-
.../org/apache/beam/sdk/io/hbase/HBaseIOIT.java | 2 +-
.../java/org/apache/beam/sdk/io/jdbc/JdbcIO.java | 4 +-
.../beam/sdk/io/kafka/KafkaSourceDescriptor.java | 4 +-
.../org/apache/beam/io/requestresponse/Call.java | 53 ++-----
.../apache/beam/io/requestresponse/Monitoring.java | 170 ++++++---------------
.../io/requestresponse/RequestResponseIOTest.java | 8 +-
.../apache/beam/sdk/io/singlestore/TestHelper.java | 2 +-
.../org/apache/beam/sdk/io/solace/data/Solace.java | 4 +-
.../beam/sdk/io/splunk/HttpEventPublisher.java | 4 +-
.../testing/TestSchemaTransformProvider.java | 6 +-
58 files changed, 253 insertions(+), 371 deletions(-)
diff --git
a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
index 9382527fa36..7f8d6ea965b 100644
--- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
+++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
@@ -1578,7 +1578,6 @@ class BeamModulePlugin implements Plugin<Project> {
"UnnecessaryLongToIntConversion",
"UnusedVariable",
// intended suppressions emerged in newer protobuf versions
- "AutoValueBoxedValues",
// For backward compatibility. Public method checked in before
this check impl
// Possible use in interface subclasses
"ClassInitializationDeadlock",
diff --git
a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java
b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java
index 76eb486e21b..2fef6c59806 100644
---
a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java
+++
b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/CsvConverters.java
@@ -435,7 +435,7 @@ public class CsvConverters {
@Nullable
public abstract String delimiter();
- public abstract Boolean hasHeaders();
+ public abstract boolean hasHeaders();
public abstract String inputFileSpec();
@@ -468,7 +468,7 @@ public class CsvConverters {
public abstract Builder setDelimiter(@Nullable String delimiter);
- public abstract Builder setHasHeaders(Boolean hasHeaders);
+ public abstract Builder setHasHeaders(boolean hasHeaders);
public abstract Builder setInputFileSpec(String inputFileSpec);
@@ -486,8 +486,6 @@ public class CsvConverters {
checkArgument(readCsv.csvFormat() != null, "Csv format must not be
null.");
- checkArgument(readCsv.hasHeaders() != null, "Header information must
be provided.");
-
return readCsv;
}
}
diff --git
a/it/clickhouse/src/main/java/org/apache/beam/it/clickhouse/conditions/ClickHouseRowsCheck.java
b/it/clickhouse/src/main/java/org/apache/beam/it/clickhouse/conditions/ClickHouseRowsCheck.java
index be6587eb2ad..14fef0aa290 100644
---
a/it/clickhouse/src/main/java/org/apache/beam/it/clickhouse/conditions/ClickHouseRowsCheck.java
+++
b/it/clickhouse/src/main/java/org/apache/beam/it/clickhouse/conditions/ClickHouseRowsCheck.java
@@ -30,7 +30,7 @@ public abstract class ClickHouseRowsCheck extends
ConditionCheck {
abstract String table();
- abstract Integer minRows();
+ abstract int minRows();
abstract @Nullable Integer maxRows();
@@ -85,7 +85,7 @@ public abstract class ClickHouseRowsCheck extends
ConditionCheck {
public abstract Builder setTable(String table);
- public abstract Builder setMinRows(Integer minRows);
+ public abstract Builder setMinRows(int minRows);
public abstract Builder setMaxRows(Integer maxRows);
diff --git
a/it/datadog/src/main/java/org/apache/beam/it/datadog/conditions/DatadogLogEntriesCheck.java
b/it/datadog/src/main/java/org/apache/beam/it/datadog/conditions/DatadogLogEntriesCheck.java
index 8c77c2c80a4..7b282b6afe7 100644
---
a/it/datadog/src/main/java/org/apache/beam/it/datadog/conditions/DatadogLogEntriesCheck.java
+++
b/it/datadog/src/main/java/org/apache/beam/it/datadog/conditions/DatadogLogEntriesCheck.java
@@ -28,7 +28,7 @@ public abstract class DatadogLogEntriesCheck extends
ConditionCheck {
abstract DatadogResourceManager resourceManager();
- abstract Integer minEntries();
+ abstract int minEntries();
@Nullable
abstract Integer maxEntries();
@@ -78,7 +78,7 @@ public abstract class DatadogLogEntriesCheck extends
ConditionCheck {
public abstract Builder setResourceManager(DatadogResourceManager
resourceManager);
- public abstract Builder setMinEntries(Integer minEvents);
+ public abstract Builder setMinEntries(int minEvents);
public abstract Builder setMaxEntries(Integer maxEvents);
diff --git
a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/bigquery/conditions/BigQueryRowsCheck.java
b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/bigquery/conditions/BigQueryRowsCheck.java
index 31a31481602..28626f7edd1 100644
---
a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/bigquery/conditions/BigQueryRowsCheck.java
+++
b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/bigquery/conditions/BigQueryRowsCheck.java
@@ -31,7 +31,7 @@ public abstract class BigQueryRowsCheck extends
ConditionCheck {
abstract TableId tableId();
- abstract Integer minRows();
+ abstract int minRows();
abstract @Nullable Integer maxRows();
@@ -87,7 +87,7 @@ public abstract class BigQueryRowsCheck extends
ConditionCheck {
public abstract Builder setTableId(TableId tableId);
- public abstract Builder setMinRows(Integer minRows);
+ public abstract Builder setMinRows(int minRows);
public abstract Builder setMaxRows(Integer maxRows);
diff --git
a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/pubsub/conditions/PubsubMessagesCheck.java
b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/pubsub/conditions/PubsubMessagesCheck.java
index 3877bff7164..46fd78049b3 100644
---
a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/pubsub/conditions/PubsubMessagesCheck.java
+++
b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/pubsub/conditions/PubsubMessagesCheck.java
@@ -36,7 +36,7 @@ public abstract class PubsubMessagesCheck extends
ConditionCheck {
abstract SubscriptionName subscription();
- abstract Integer minMessages();
+ abstract int minMessages();
abstract @Nullable Integer maxMessages();
@@ -104,7 +104,7 @@ public abstract class PubsubMessagesCheck extends
ConditionCheck {
public abstract Builder setSubscription(SubscriptionName subscription);
- public abstract Builder setMinMessages(Integer minMessages);
+ public abstract Builder setMinMessages(int minMessages);
public abstract Builder setMaxMessages(Integer maxMessages);
diff --git
a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java
b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java
index d68c0f07865..6e511bd8e5c 100644
---
a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java
+++
b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java
@@ -160,18 +160,18 @@ public class BigQueryStreamingLT extends IOLoadTestBase {
@AutoValue
abstract static class TestConfiguration {
/** Rows will be generated for this many minutes. */
- abstract Integer getMinutes();
+ abstract int getMinutes();
/** Data shape: The byte-size for each field. */
- abstract Integer getByteSizePerField();
+ abstract int getByteSizePerField();
/** Data shape: The number of fields per row. */
- abstract Integer getNumFields();
+ abstract int getNumFields();
/**
* Rate of generated elements sent to the sink. Will run with a minimum of
1k rows per second.
*/
- abstract Integer getRowsPerSecond();
+ abstract int getRowsPerSecond();
abstract String getRunner();
diff --git
a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigtable/BigTableIOLT.java
b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigtable/BigTableIOLT.java
index 9770a40cb83..410c992fe2d 100644
---
a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigtable/BigTableIOLT.java
+++
b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigtable/BigTableIOLT.java
@@ -142,7 +142,7 @@ public class BigTableIOLT extends IOLoadTestBase {
region,
readInfo.jobId(),
getBeamMetricsName(PipelineMetricsType.COUNTER,
READ_ELEMENT_METRIC_NAME));
- assertEquals(configuration.getNumRows(), numRecords, 0.5);
+ assertEquals((double) configuration.getNumRows(), numRecords, 0.5);
// export metrics
MetricsConfiguration metricsConfig =
@@ -214,13 +214,13 @@ public class BigTableIOLT extends IOLoadTestBase {
/** Options for BigtableIO load test. */
@AutoValue
abstract static class Configuration {
- abstract Long getNumRows();
+ abstract long getNumRows();
- abstract Integer getPipelineTimeout();
+ abstract int getPipelineTimeout();
abstract String getRunner();
- abstract Integer getValueSizeBytes();
+ abstract int getValueSizeBytes();
static Configuration of(long numRows, int pipelineTimeout, String runner,
int valueSizeBytes) {
return new AutoValue_BigTableIOLT_Configuration.Builder()
diff --git a/it/kafka/src/test/java/org/apache/beam/it/kafka/KafkaIOLT.java
b/it/kafka/src/test/java/org/apache/beam/it/kafka/KafkaIOLT.java
index ce6ad877c37..f2a2c73ee87 100644
--- a/it/kafka/src/test/java/org/apache/beam/it/kafka/KafkaIOLT.java
+++ b/it/kafka/src/test/java/org/apache/beam/it/kafka/KafkaIOLT.java
@@ -145,7 +145,7 @@ public final class KafkaIOLT extends IOLoadTestBase {
region,
readInfo.jobId(),
getBeamMetricsName(PipelineMetricsType.COUNTER,
READ_ELEMENT_METRIC_NAME));
- assertEquals(configuration.getNumRows(), numRecords, 10.0);
+ assertEquals((double) configuration.getNumRows(), numRecords, 10.0);
} finally {
// clean up pipelines
if (pipelineLauncher.getJobStatus(project, region, writeInfo.jobId())
@@ -207,13 +207,13 @@ public final class KafkaIOLT extends IOLoadTestBase {
/** Options for Kafka IO load test. */
@AutoValue
abstract static class Configuration {
- abstract Long getNumRows();
+ abstract long getNumRows();
- abstract Integer getPipelineTimeout();
+ abstract int getPipelineTimeout();
abstract String getRunner();
- abstract Integer getRowSize();
+ abstract int getRowSize();
static Configuration of(long numRows, int pipelineTimeout, String runner) {
return new AutoValue_KafkaIOLT_Configuration.Builder()
diff --git
a/it/mongodb/src/main/java/org/apache/beam/it/mongodb/conditions/MongoDBDocumentsCheck.java
b/it/mongodb/src/main/java/org/apache/beam/it/mongodb/conditions/MongoDBDocumentsCheck.java
index 67c3624dbc7..7c2e1fcc51a 100644
---
a/it/mongodb/src/main/java/org/apache/beam/it/mongodb/conditions/MongoDBDocumentsCheck.java
+++
b/it/mongodb/src/main/java/org/apache/beam/it/mongodb/conditions/MongoDBDocumentsCheck.java
@@ -30,7 +30,7 @@ public abstract class MongoDBDocumentsCheck extends
ConditionCheck {
abstract String collectionName();
- abstract Integer minDocuments();
+ abstract int minDocuments();
abstract @Nullable Integer maxDocuments();
@@ -88,7 +88,7 @@ public abstract class MongoDBDocumentsCheck extends
ConditionCheck {
public abstract Builder setCollectionName(String collectionName);
- public abstract Builder setMinDocuments(Integer minDocuments);
+ public abstract Builder setMinDocuments(int minDocuments);
public abstract Builder setMaxDocuments(Integer maxDocuments);
diff --git
a/it/splunk/src/main/java/org/apache/beam/it/splunk/conditions/SplunkEventsCheck.java
b/it/splunk/src/main/java/org/apache/beam/it/splunk/conditions/SplunkEventsCheck.java
index c2d5f5b4f28..3d5ae2de698 100644
---
a/it/splunk/src/main/java/org/apache/beam/it/splunk/conditions/SplunkEventsCheck.java
+++
b/it/splunk/src/main/java/org/apache/beam/it/splunk/conditions/SplunkEventsCheck.java
@@ -30,7 +30,7 @@ public abstract class SplunkEventsCheck extends
ConditionCheck {
abstract @Nullable String query();
- abstract Integer minEvents();
+ abstract int minEvents();
abstract @Nullable Integer maxEvents();
@@ -85,7 +85,7 @@ public abstract class SplunkEventsCheck extends
ConditionCheck {
public abstract Builder setQuery(String query);
- public abstract Builder setMinEvents(Integer minEvents);
+ public abstract Builder setMinEvents(int minEvents);
public abstract Builder setMaxEvents(Integer maxEvents);
diff --git
a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java
b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java
index a79b0996cbb..36367d0d972 100644
---
a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java
+++
b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java
@@ -152,7 +152,7 @@ public class CommonCoderTest {
@SuppressWarnings("mutable")
abstract byte[] getPayload();
- abstract Boolean getNonDeterministic();
+ abstract boolean getNonDeterministic();
abstract Map<ByteString, ByteString> getState();
diff --git
a/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProvider.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProvider.java
index d9dfc2a90bd..83d87c5247b 100644
---
a/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProvider.java
+++
b/sdks/java/core/src/main/java/org/apache/beam/sdk/providers/GenerateSequenceSchemaTransformProvider.java
@@ -18,7 +18,6 @@
package org.apache.beam.sdk.providers;
import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;
-import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
import com.google.auto.service.AutoService;
import com.google.auto.value.AutoValue;
@@ -90,7 +89,7 @@ public class GenerateSequenceSchemaTransformProvider
@AutoValue
public abstract static class Rate {
@SchemaFieldDescription("Number of elements component of the rate.")
- public abstract Long getElements();
+ public abstract long getElements();
@SchemaFieldDescription("Number of seconds component of the rate.")
@Nullable
@@ -103,7 +102,7 @@ public class GenerateSequenceSchemaTransformProvider
@AutoValue.Builder
public abstract static class Builder {
- public abstract Builder setElements(Long elements);
+ public abstract Builder setElements(long elements);
public abstract Builder setSeconds(Long seconds);
@@ -117,7 +116,7 @@ public class GenerateSequenceSchemaTransformProvider
}
@SchemaFieldDescription("The minimum number to generate (inclusive).")
- public abstract Long getStart();
+ public abstract long getStart();
@SchemaFieldDescription(
"The maximum number to generate (exclusive). Will be an unbounded
sequence if left unspecified.")
@@ -133,7 +132,7 @@ public class GenerateSequenceSchemaTransformProvider
@AutoValue.Builder
public abstract static class Builder {
- public abstract Builder setStart(Long start);
+ public abstract Builder setStart(long start);
public abstract Builder setEnd(Long end);
@@ -143,8 +142,7 @@ public class GenerateSequenceSchemaTransformProvider
}
public void validate() {
- checkNotNull(this.getStart(), "Must specify a starting point
\"start\".");
- Long start = this.getStart();
+ long start = this.getStart();
Long end = this.getEnd();
if (end != null) {
checkArgument(end == -1 || end >= start, "Invalid range [%s, %s)",
start, end);
diff --git
a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
index c2144f71eac..ecedfd0f03b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
@@ -685,7 +685,7 @@ public class Schema implements Serializable {
public abstract TypeName getTypeName();
// Whether this type is nullable.
- public abstract Boolean getNullable();
+ public abstract boolean getNullable();
// For logical types, return the implementing class.
@@ -744,7 +744,7 @@ public class Schema implements Serializable {
abstract Builder setCollectionElementType(@Nullable FieldType
collectionElementType);
- abstract Builder setNullable(Boolean nullable);
+ abstract Builder setNullable(boolean nullable);
abstract Builder setMapKeyType(@Nullable FieldType mapKeyType);
@@ -953,7 +953,7 @@ public class Schema implements Serializable {
}
}
return Objects.equals(getTypeName(), other.getTypeName())
- && Objects.equals(getNullable(), other.getNullable())
+ && getNullable() == other.getNullable()
&& Objects.equals(getCollectionElementType(),
other.getCollectionElementType())
&& Objects.equals(getMapKeyType(), other.getMapKeyType())
&& Objects.equals(getMapValueType(), other.getMapValueType())
@@ -984,7 +984,7 @@ public class Schema implements Serializable {
return false;
}
}
- if (!Objects.equals(getNullable(), other.getNullable())) {
+ if (getNullable() != other.getNullable()) {
return false;
}
if (!Objects.equals(getMetadata(), other.getMetadata())) {
@@ -1009,7 +1009,7 @@ public class Schema implements Serializable {
/** Check whether two types are equivalent. */
public boolean equivalent(FieldType other, EquivalenceNullablePolicy
nullablePolicy) {
if (nullablePolicy == EquivalenceNullablePolicy.SAME
- && !other.getNullable().equals(getNullable())) {
+ && other.getNullable() != getNullable()) {
return false;
} else if (nullablePolicy == EquivalenceNullablePolicy.WEAKEN) {
if (getNullable() && !other.getNullable()) {
diff --git
a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Group.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Group.java
index c77c8dcd20a..8d314430b9c 100644
---
a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Group.java
+++
b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Group.java
@@ -998,7 +998,7 @@ public class Group {
abstract @Nullable Fanout getFanout();
- abstract Boolean getFewKeys();
+ abstract boolean getFewKeys();
abstract ByFields<InputT> getByFields();
@@ -1014,7 +1014,7 @@ public class Group {
abstract static class Builder<InputT> {
public abstract Builder<InputT> setFanout(@Nullable Fanout value);
- public abstract Builder<InputT> setFewKeys(Boolean fewKeys);
+ public abstract Builder<InputT> setFewKeys(boolean fewKeys);
abstract Builder<InputT> setByFields(ByFields<InputT> byFields);
diff --git
a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java
b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java
index d7a5c386224..f4700212511 100644
---
a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java
+++
b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java
@@ -728,7 +728,7 @@ public class AutoValueSchemaTest {
abstract String getStr();
@SchemaFieldNumber("0")
- abstract Long getLng();
+ abstract long getLng();
}
private static final Schema FIELD_NUMBER_SCHEMA =
@@ -866,7 +866,7 @@ public class AutoValueSchemaTest {
"This field is a long in the row. Interestingly enough, longs are e"
+ "ncoded as int64 by Beam, while ints are encoded as int32. "
+ "Sign semantics are another thing")
- abstract Long getLng();
+ abstract long getLng();
}
private static final Schema FIELD_DESCRIPTION_SCHEMA =
diff --git
a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java
b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java
index cc57d382af0..75a43874341 100644
---
a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java
+++
b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java
@@ -82,13 +82,13 @@ public class SchemaCoderTest {
public abstract static class SimpleAutoValue {
public abstract String getString();
- public abstract Integer getInt32();
+ public abstract int getInt32();
- public abstract Long getInt64();
+ public abstract long getInt64();
public abstract DateTime getDatetime();
- public static SimpleAutoValue of(String string, Integer int32, Long int64,
DateTime datetime) {
+ public static SimpleAutoValue of(String string, int int32, long int64,
DateTime datetime) {
return new AutoValue_SchemaCoderTest_SimpleAutoValue(string, int32,
int64, datetime);
}
}
diff --git
a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/SelectTest.java
b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/SelectTest.java
index ba8ef98115c..eea4ab03dfc 100644
---
a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/SelectTest.java
+++
b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/SelectTest.java
@@ -58,9 +58,9 @@ public class SelectTest {
abstract static class Schema1 {
abstract String getField1();
- abstract Integer getField2();
+ abstract int getField2();
- abstract Double getField3();
+ abstract double getField3();
static Schema1 create() {
return new AutoValue_SelectTest_Schema1("field1", 42, 3.14);
@@ -73,7 +73,7 @@ public class SelectTest {
abstract static class Schema1Selected {
abstract String getField1();
- abstract Double getField3();
+ abstract double getField3();
static Schema1Selected create() {
return new AutoValue_SelectTest_Schema1Selected("field1", 3.14);
@@ -88,7 +88,7 @@ public class SelectTest {
abstract static class Schema1SelectedRenamed {
abstract String getFieldOne();
- abstract Double getFieldThree();
+ abstract double getFieldThree();
static Schema1SelectedRenamed create() {
return new AutoValue_SelectTest_Schema1SelectedRenamed("field1", 3.14);
diff --git
a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java
b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java
index 2eef0e30f80..275172d971c 100644
---
a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java
+++
b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java
@@ -45,7 +45,7 @@ public class TypedSchemaTransformProviderTest {
abstract static class Configuration {
abstract String getStringField();
- abstract Integer getIntegerField();
+ abstract int getIntegerField();
static Configuration create(String stringField, int integerField) {
return new AutoValue_TypedSchemaTransformProviderTest_Configuration(
@@ -140,7 +140,7 @@ public class TypedSchemaTransformProviderTest {
for (Configuration config : Arrays.asList(outputConfig,
minimalOutputConfig)) {
assertEquals("field1", config.getStringField());
- assertEquals(13, config.getIntegerField().intValue());
+ assertEquals(13, config.getIntegerField());
}
assertEquals("Description of fake provider", provider.description());
}
diff --git
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoSchemaTest.java
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoSchemaTest.java
index ea1999ef6b3..07538a40d1f 100644
---
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoSchemaTest.java
+++
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoSchemaTest.java
@@ -402,7 +402,7 @@ public class ParDoSchemaTest implements Serializable {
abstract static class Inferred {
abstract String getStringField();
- abstract Integer getIntegerField();
+ abstract int getIntegerField();
}
@Test
@@ -448,7 +448,7 @@ public class ParDoSchemaTest implements Serializable {
@DefaultSchema(AutoValueSchema.class)
@AutoValue
abstract static class Inferred2 {
- abstract Integer getIntegerField();
+ abstract int getIntegerField();
abstract String getStringField();
}
@@ -519,7 +519,7 @@ public class ParDoSchemaTest implements Serializable {
@DefaultSchema(AutoValueSchema.class)
@AutoValue
abstract static class ForExtraction {
- abstract Integer getIntegerField();
+ abstract int getIntegerField();
abstract String getStringField();
@@ -828,7 +828,7 @@ public class ParDoSchemaTest implements Serializable {
@DefaultSchema(AutoValueSchema.class)
@AutoValue
abstract static class TestStateSchemaValue2 {
- abstract Integer getInteger();
+ abstract int getInteger();
}
@DefaultSchema(AutoValueSchema.class)
diff --git
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ToJsonTest.java
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ToJsonTest.java
index 76a29a98a7f..7ca828e4d2b 100644
---
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ToJsonTest.java
+++
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ToJsonTest.java
@@ -44,15 +44,15 @@ public class ToJsonTest implements Serializable {
@DefaultSchema(AutoValueSchema.class)
@AutoValue
abstract static class Person {
- public static Person of(String name, Integer height, Boolean
knowsJavascript) {
+ public static Person of(String name, int height, boolean knowsJavascript) {
return new AutoValue_ToJsonTest_Person(name, height, knowsJavascript);
}
public abstract String getName();
- public abstract Integer getHeight();
+ public abstract int getHeight();
- public abstract Boolean getKnowsJavascript();
+ public abstract boolean getKnowsJavascript();
}
@Test
diff --git
a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java
b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java
index d7bfc5f1677..6c8332561c9 100644
---
a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java
+++
b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java
@@ -493,7 +493,7 @@ public class ExpansionServiceTest {
@DefaultSchema(AutoValueSchema.class)
@AutoValue
abstract static class TestConfigSimpleSchema {
- abstract Long getFoo();
+ abstract long getFoo();
abstract String getBar();
diff --git
a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java
b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java
index cee0b62bfe1..6f4c29305ee 100644
---
a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java
+++
b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java
@@ -1261,7 +1261,7 @@ public class AvroUtils {
private static @Nullable Object genericFromBeamField(
FieldType fieldType, org.apache.avro.Schema avroSchema, @Nullable Object
value) {
TypeWithNullability typeWithNullability = new
TypeWithNullability(avroSchema);
- if (!fieldType.getNullable().equals(typeWithNullability.nullable)) {
+ if (fieldType.getNullable() != typeWithNullability.nullable) {
throw new IllegalArgumentException(
"FieldType "
+ fieldType
diff --git
a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
index 39e0b4836a0..d3d72a7902e 100644
---
a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
+++
b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
@@ -88,7 +88,7 @@ public abstract class DLPDeidentifyText
public abstract @Nullable String getColumnDelimiter();
/** Returns size of input elements batch to be sent to Cloud DLP service in
one request. */
- public abstract Integer getBatchSizeBytes();
+ public abstract int getBatchSizeBytes();
/** Returns ID of Google Cloud project to be used when deidentifying data. */
public abstract String getProjectId();
@@ -121,7 +121,7 @@ public abstract class DLPDeidentifyText
*
* @param batchSize Size of input elements batch to be sent to Cloud DLP
service in one request.
*/
- public abstract Builder setBatchSizeBytes(Integer batchSize);
+ public abstract Builder setBatchSizeBytes(int batchSize);
/**
* Sets ID of Google Cloud project to be used when deidentifying data.
diff --git
a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPInspectText.java
b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPInspectText.java
index 8109fc0ea74..c6292bd80b3 100644
---
a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPInspectText.java
+++
b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPInspectText.java
@@ -77,7 +77,7 @@ public abstract class DLPInspectText
public abstract @Nullable InspectConfig getInspectConfig();
/** Returns size of input elements batch to be sent to Cloud DLP service in
one request. */
- public abstract Integer getBatchSizeBytes();
+ public abstract int getBatchSizeBytes();
/** Returns ID of Google Cloud project to be used when deidentifying data. */
public abstract String getProjectId();
@@ -110,7 +110,7 @@ public abstract class DLPInspectText
*
* @param batchSize Size of input elements batch to be sent to Cloud DLP
service in one request.
*/
- public abstract Builder setBatchSizeBytes(Integer batchSize);
+ public abstract Builder setBatchSizeBytes(int batchSize);
/**
* Sets ID of Google Cloud project to be used when deidentifying data.
diff --git
a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java
b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java
index 622b8493437..b5678d6c77b 100644
---
a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java
+++
b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java
@@ -94,7 +94,7 @@ public abstract class DLPReidentifyText
public abstract @Nullable PCollectionView<List<String>> getHeaderColumns();
/** Returns size of input elements batch to be sent to Cloud DLP service in
one request. */
- public abstract Integer getBatchSizeBytes();
+ public abstract int getBatchSizeBytes();
/** Returns ID of Google Cloud project to be used when deidentifying data. */
public abstract String getProjectId();
@@ -136,7 +136,7 @@ public abstract class DLPReidentifyText
*
* @param batchSize Size of input elements batch to be sent to Cloud DLP
service in one request.
*/
- public abstract Builder setBatchSizeBytes(Integer batchSize);
+ public abstract Builder setBatchSizeBytes(int batchSize);
/**
* Sets list of column names if the input KV value is a delimited row.
*
diff --git
a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/RecommendationAIImportCatalogItems.java
b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/RecommendationAIImportCatalogItems.java
index d84bedef885..31dc8c821a2 100644
---
a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/RecommendationAIImportCatalogItems.java
+++
b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/RecommendationAIImportCatalogItems.java
@@ -79,7 +79,7 @@ public abstract class RecommendationAIImportCatalogItems
public abstract @Nullable String catalogName();
/** Returns size of input elements batch to be sent in one request. */
- public abstract Integer batchSize();
+ public abstract int batchSize();
/**
* Returns time limit (in processing time) on how long an incomplete batch
of elements is allowed
@@ -95,7 +95,7 @@ public abstract class RecommendationAIImportCatalogItems
return this.toBuilder().setCatalogName(catalogName).build();
}
- public RecommendationAIImportCatalogItems withBatchSize(Integer batchSize) {
+ public RecommendationAIImportCatalogItems withBatchSize(int batchSize) {
return this.toBuilder().setBatchSize(batchSize).build();
}
@@ -142,7 +142,7 @@ public abstract class RecommendationAIImportCatalogItems
* @param batchSize Amount of input elements to be sent to Recommendation
AI service in one
* request.
*/
- public abstract Builder setBatchSize(Integer batchSize);
+ public abstract Builder setBatchSize(int batchSize);
/**
* Sets time limit (in processing time) on how long an incomplete batch of
elements is allowed
diff --git
a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/RecommendationAIImportUserEvents.java
b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/RecommendationAIImportUserEvents.java
index cd60486e6eb..5907e1c0d11 100644
---
a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/RecommendationAIImportUserEvents.java
+++
b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/RecommendationAIImportUserEvents.java
@@ -85,7 +85,7 @@ public abstract class RecommendationAIImportUserEvents
public abstract @Nullable String eventStore();
/** Returns size of input elements batch to be sent in one request. */
- public abstract Integer batchSize();
+ public abstract int batchSize();
/**
* Returns time limit (in processing time) on how long an incomplete batch
of elements is allowed
@@ -105,7 +105,7 @@ public abstract class RecommendationAIImportUserEvents
return this.toBuilder().setEventStore(eventStore).build();
}
- public RecommendationAIImportUserEvents withBatchSize(Integer batchSize) {
+ public RecommendationAIImportUserEvents withBatchSize(int batchSize) {
return this.toBuilder().setBatchSize(batchSize).build();
}
@@ -159,7 +159,7 @@ public abstract class RecommendationAIImportUserEvents
* @param batchSize Amount of input elements to be sent to Recommendation
AI service in one
* request.
*/
- public abstract Builder setBatchSize(Integer batchSize);
+ public abstract Builder setBatchSize(int batchSize);
/**
* Sets time limit (in processing time) on how long an incomplete batch of
elements is allowed
diff --git
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlAlterTable.java
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlAlterTable.java
index 32a86864bcb..2edf8be9aaa 100644
---
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlAlterTable.java
+++
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlAlterTable.java
@@ -224,7 +224,7 @@ public class SqlAlterTable extends SqlAlter implements
BeamSqlParser.ExecutableS
writer.identifier(column.getName(), false);
writer.keyword(CalciteUtils.toSqlTypeName(column.getType()).name());
- if (column.getType().getNullable() != null &&
!column.getType().getNullable()) {
+ if (!column.getType().getNullable()) {
writer.keyword("NOT NULL");
}
diff --git
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java
index ab644145b4f..499a12ee9e0 100644
---
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java
+++
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java
@@ -189,7 +189,7 @@ public class SqlCreateExternalTable extends SqlCreate
implements BeamSqlParser.E
writer.identifier(column.getName(), false);
writer.identifier(CalciteUtils.toSqlTypeName(column.getType()).name(),
false);
- if (column.getType().getNullable() != null &&
!column.getType().getNullable()) {
+ if (!column.getType().getNullable()) {
writer.keyword("NOT NULL");
}
diff --git
a/sdks/java/extensions/timeseries/src/main/java/org/apache/beam/sdk/extensions/timeseries/FillGaps.java
b/sdks/java/extensions/timeseries/src/main/java/org/apache/beam/sdk/extensions/timeseries/FillGaps.java
index 97dabfe93ca..9fd40852184 100644
---
a/sdks/java/extensions/timeseries/src/main/java/org/apache/beam/sdk/extensions/timeseries/FillGaps.java
+++
b/sdks/java/extensions/timeseries/src/main/java/org/apache/beam/sdk/extensions/timeseries/FillGaps.java
@@ -152,7 +152,7 @@ public abstract class FillGaps<ValueT>
abstract Duration getTimeseriesBucketDuration();
- abstract Long getMaxGapFillBuckets();
+ abstract long getMaxGapFillBuckets();
abstract Instant getStopTime();
@@ -173,7 +173,7 @@ public abstract class FillGaps<ValueT>
abstract static class Builder<ValueT> {
abstract Builder<ValueT> setTimeseriesBucketDuration(Duration value);
- abstract Builder<ValueT> setMaxGapFillBuckets(Long value);
+ abstract Builder<ValueT> setMaxGapFillBuckets(long value);
abstract Builder<ValueT> setStopTime(Instant value);
@@ -211,7 +211,7 @@ public abstract class FillGaps<ValueT>
}
/* The max gap duration that will be filled. The transform will stop filling
timeseries buckets after this duration. */
- public FillGaps<ValueT> withMaxGapFillBuckets(Long value) {
+ public FillGaps<ValueT> withMaxGapFillBuckets(long value) {
return toBuilder().setMaxGapFillBuckets(value).build();
}
diff --git
a/sdks/java/extensions/zetasketch/src/main/java/org/apache/beam/sdk/extensions/zetasketch/ApproximateCountDistinct.java
b/sdks/java/extensions/zetasketch/src/main/java/org/apache/beam/sdk/extensions/zetasketch/ApproximateCountDistinct.java
index b1272608fd9..3701f032f79 100644
---
a/sdks/java/extensions/zetasketch/src/main/java/org/apache/beam/sdk/extensions/zetasketch/ApproximateCountDistinct.java
+++
b/sdks/java/extensions/zetasketch/src/main/java/org/apache/beam/sdk/extensions/zetasketch/ApproximateCountDistinct.java
@@ -130,7 +130,7 @@ public class ApproximateCountDistinct {
return toBuilder().setMapping(Contextful.<T, Long>fn(fn)).build();
}
- public <V> Globally<V> withPercision(Integer withPercision) {
+ public <V> Globally<V> withPercision(int withPercision) {
@SuppressWarnings("unchecked")
Globally<V> globally = (Globally<V>)
toBuilder().setPrecision(withPercision).build();
return globally;
@@ -176,7 +176,7 @@ public class ApproximateCountDistinct {
public abstract static class PerKey<K, V>
extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Long>>> {
- public abstract Integer getPrecision();
+ public abstract int getPrecision();
@Nullable
public abstract Contextful<Fn<KV<K, V>, KV<K, Long>>> getMapping();
@@ -186,14 +186,14 @@ public class ApproximateCountDistinct {
@AutoValue.Builder
public abstract static class Builder<K, V> {
- public abstract Builder<K, V> setPrecision(Integer precision);
+ public abstract Builder<K, V> setPrecision(int precision);
public abstract Builder<K, V> setMapping(Contextful<Fn<KV<K, V>, KV<K,
Long>>> value);
public abstract PerKey<K, V> build();
}
- public <K2, V2> PerKey<K2, V2> withPercision(Integer withPercision) {
+ public <K2, V2> PerKey<K2, V2> withPercision(int withPercision) {
// Work around for loss of type inference when using API.
@SuppressWarnings("unchecked")
PerKey<K2, V2> perKey = (PerKey<K2, V2>)
this.toBuilder().setPrecision(withPercision).build();
diff --git
a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/testing/DynamoDBIOIT.java
b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/testing/DynamoDBIOIT.java
index 9ba35044c3f..1efe2de952f 100644
---
a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/testing/DynamoDBIOIT.java
+++
b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/testing/DynamoDBIOIT.java
@@ -154,7 +154,7 @@ public class DynamoDBIOIT {
}
private static KV<String, WriteRequest> buildWriteRequest(TestRow row) {
- AttributeValue id =
AttributeValue.builder().n(row.id().toString()).build();
+ AttributeValue id =
AttributeValue.builder().n(String.valueOf(row.id())).build();
AttributeValue name = AttributeValue.builder().s(row.name()).build();
PutRequest req = PutRequest.builder().item(ImmutableMap.of(COL_ID, id,
COL_NAME, name)).build();
return KV.of(env.options().getDynamoDBTable(),
WriteRequest.builder().putRequest(req).build());
diff --git
a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIOWriteTest.java
b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIOWriteTest.java
index 61020ff571c..34f630afeb4 100644
---
a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIOWriteTest.java
+++
b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIOWriteTest.java
@@ -241,7 +241,7 @@ public class KinesisIOWriteTest extends PutRecordsHelpers {
pipeline
.apply(Create.of(100))
.apply(ParDo.of(new GenerateTestRows()))
- .apply(kinesisWrite().withPartitioner(row -> row.id().toString()));
+ .apply(kinesisWrite().withPartitioner(row ->
String.valueOf(row.id())));
pipeline.run().waitUntilFinish();
verify(client).putRecords(argThat(hasSize(2))); // 1 aggregated record per
shard
@@ -258,7 +258,7 @@ public class KinesisIOWriteTest extends PutRecordsHelpers {
pipeline
.apply(Create.of(100))
.apply(ParDo.of(new GenerateTestRows()))
- .apply(kinesisWrite().withPartitioner(row -> row.id().toString()));
+ .apply(kinesisWrite().withPartitioner(row ->
String.valueOf(row.id())));
pipeline.run().waitUntilFinish();
resp.complete(ListShardsResponse.builder().build()); // complete list
shards after pipeline
@@ -278,7 +278,7 @@ public class KinesisIOWriteTest extends PutRecordsHelpers {
.apply(
kinesisWrite()
.withRecordAggregation(b -> b.shardRefreshInterval(ZERO)) //
disable refresh
- .withPartitioner(row -> row.id().toString()));
+ .withPartitioner(row -> String.valueOf(row.id())));
pipeline.run().waitUntilFinish();
@@ -378,7 +378,7 @@ public class KinesisIOWriteTest extends PutRecordsHelpers {
Write<TestRow> write =
kinesisWrite()
- .withPartitioner(r -> r.id().toString())
+ .withPartitioner(r -> String.valueOf(r.id()))
.withRecordAggregation(b ->
b.maxBufferedTime(millis(100)).maxBufferedTimeJitter(0.2));
DateTimeUtils.setCurrentMillisFixed(0);
@@ -420,7 +420,7 @@ public class KinesisIOWriteTest extends PutRecordsHelpers {
Write<TestRow> write =
kinesisWrite()
- .withPartitioner(r -> r.id().toString())
+ .withPartitioner(r -> String.valueOf(r.id()))
.withRecordAggregation(b -> b.shardRefreshInterval(millis(1000)));
DateTimeUtils.setCurrentMillisFixed(1);
diff --git
a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/TestRowDBWritable.java
b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/TestRowDBWritable.java
index ef17d1601dd..da07061280d 100644
---
a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/TestRowDBWritable.java
+++
b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/TestRowDBWritable.java
@@ -36,18 +36,18 @@ import org.apache.hadoop.mapreduce.lib.db.DBWritable;
@DefaultCoder(AvroCoder.class)
class TestRowDBWritable extends TestRow implements DBWritable, Writable {
- private Integer id;
+ private int id;
private String name;
public TestRowDBWritable() {}
- public TestRowDBWritable(Integer id, String name) {
+ public TestRowDBWritable(int id, String name) {
this.id = id;
this.name = name;
}
@Override
- public Integer id() {
+ public int id() {
return id;
}
diff --git
a/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/SchemaAwareJavaBeans.java
b/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/SchemaAwareJavaBeans.java
index 76535c3e17f..ee044790533 100644
---
a/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/SchemaAwareJavaBeans.java
+++
b/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/SchemaAwareJavaBeans.java
@@ -40,12 +40,12 @@ public class SchemaAwareJavaBeans {
/** Convenience method for {@link AllPrimitiveDataTypes} instantiation. */
public static AllPrimitiveDataTypes allPrimitiveDataTypes(
- Boolean aBoolean,
+ boolean aBoolean,
BigDecimal aDecimal,
- Double aDouble,
- Float aFloat,
- Integer anInteger,
- Long aLong,
+ double aDouble,
+ float aFloat,
+ int anInteger,
+ long aLong,
String aString) {
return new AutoValue_SchemaAwareJavaBeans_AllPrimitiveDataTypes.Builder()
.setABoolean(aBoolean)
@@ -85,7 +85,7 @@ public class SchemaAwareJavaBeans {
}
/** Convenience method for {@link ByteType} instantiation. */
- public static ByteType byteType(Byte aByte, List<Byte> byteList) {
+ public static ByteType byteType(byte aByte, List<Byte> byteList) {
return new AutoValue_SchemaAwareJavaBeans_ByteType.Builder()
.setByte(aByte)
.setByteList(byteList)
@@ -329,17 +329,17 @@ public class SchemaAwareJavaBeans {
@AutoValue
public abstract static class AllPrimitiveDataTypes implements Serializable {
- public abstract Boolean getABoolean();
+ public abstract boolean getABoolean();
public abstract BigDecimal getADecimal();
- public abstract Double getADouble();
+ public abstract double getADouble();
- public abstract Float getAFloat();
+ public abstract float getAFloat();
- public abstract Integer getAnInteger();
+ public abstract int getAnInteger();
- public abstract Long getALong();
+ public abstract long getALong();
public abstract String getAString();
@@ -348,17 +348,17 @@ public class SchemaAwareJavaBeans {
@AutoValue.Builder
public abstract static class Builder {
- public abstract Builder setABoolean(Boolean value);
+ public abstract Builder setABoolean(boolean value);
public abstract Builder setADecimal(BigDecimal value);
- public abstract Builder setADouble(Double value);
+ public abstract Builder setADouble(double value);
- public abstract Builder setAFloat(Float value);
+ public abstract Builder setAFloat(float value);
- public abstract Builder setAnInteger(Integer value);
+ public abstract Builder setAnInteger(int value);
- public abstract Builder setALong(Long value);
+ public abstract Builder setALong(long value);
public abstract Builder setAString(String value);
@@ -448,7 +448,7 @@ public class SchemaAwareJavaBeans {
@AutoValue
public abstract static class ByteType {
- public abstract Byte getByte();
+ public abstract byte getByte();
public abstract List<Byte> getByteList();
@@ -457,7 +457,7 @@ public class SchemaAwareJavaBeans {
@AutoValue.Builder
public abstract static class Builder {
- public abstract Builder setByte(Byte value);
+ public abstract Builder setByte(byte value);
public abstract Builder setByteList(List<Byte> value);
diff --git
a/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/TestRow.java
b/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/TestRow.java
index 125fb70cc0b..86e0bb9a933 100644
---
a/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/TestRow.java
+++
b/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/TestRow.java
@@ -30,26 +30,26 @@ import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Immuta
@AutoValue
public abstract class TestRow implements Serializable, Comparable<TestRow> {
/** Manually create a test row. */
- public static TestRow create(Integer id, String name) {
+ public static TestRow create(int id, String name) {
return new AutoValue_TestRow(id, name);
}
- public abstract Integer id();
+ public abstract int id();
public abstract String name();
@Override
public int compareTo(TestRow other) {
- return id().compareTo(other.id());
+ return Integer.compare(id(), other.id());
}
/** Creates a {@link TestRow} from the seed value. */
- public static TestRow fromSeed(Integer seed) {
+ public static TestRow fromSeed(int seed) {
return create(seed, getNameForSeed(seed));
}
/** Returns the name field value produced from the given seed. */
- public static String getNameForSeed(Integer seed) {
+ public static String getNameForSeed(int seed) {
return "Testval" + seed;
}
diff --git
a/sdks/java/io/datadog/src/main/java/org/apache/beam/sdk/io/datadog/DatadogEventPublisher.java
b/sdks/java/io/datadog/src/main/java/org/apache/beam/sdk/io/datadog/DatadogEventPublisher.java
index 00a106b2ded..b9f1d852557 100644
---
a/sdks/java/io/datadog/src/main/java/org/apache/beam/sdk/io/datadog/DatadogEventPublisher.java
+++
b/sdks/java/io/datadog/src/main/java/org/apache/beam/sdk/io/datadog/DatadogEventPublisher.java
@@ -97,7 +97,7 @@ public abstract class DatadogEventPublisher {
abstract String apiKey();
- abstract Integer maxElapsedMillis();
+ abstract int maxElapsedMillis();
/**
* Executes a POST for the list of {@link DatadogEvent} objects into
Datadog's Logs API.
@@ -228,9 +228,9 @@ public abstract class DatadogEventPublisher {
abstract String apiKey();
- abstract Builder setMaxElapsedMillis(Integer maxElapsedMillis);
+ abstract Builder setMaxElapsedMillis(int maxElapsedMillis);
- abstract Integer maxElapsedMillis();
+ abstract int maxElapsedMillis();
abstract DatadogEventPublisher autoBuild();
@@ -263,9 +263,7 @@ public abstract class DatadogEventPublisher {
* @param maxElapsedMillis max elapsed time in milliseconds for timeout.
* @return {@link Builder}
*/
- public Builder withMaxElapsedMillis(Integer maxElapsedMillis) {
- checkNotNull(
- maxElapsedMillis, "withMaxElapsedMillis(maxElapsedMillis) called
with null input.");
+ public Builder withMaxElapsedMillis(int maxElapsedMillis) {
return setMaxElapsedMillis(maxElapsedMillis);
}
diff --git
a/sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java
b/sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java
index d85bb1a7dc5..d1d5103cc49 100644
---
a/sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java
+++
b/sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java
@@ -180,7 +180,7 @@ public class DebeziumReadSchemaTransformProvider
public abstract String getHost();
- public abstract Integer getPort();
+ public abstract int getPort();
public abstract String getTable();
@@ -201,7 +201,7 @@ public class DebeziumReadSchemaTransformProvider
public abstract Builder setHost(String host);
- public abstract Builder setPort(Integer port);
+ public abstract Builder setPort(int port);
public abstract Builder setDatabase(String database);
diff --git
a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
index ba4ac276994..f3d6122f2a0 100644
---
a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
+++
b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
@@ -2051,7 +2051,7 @@ public class ElasticsearchIO {
public abstract @Nullable String getBulkDirective();
- public abstract Boolean getHasError();
+ public abstract boolean getHasError();
public abstract @Nullable String getResponseItemJson();
diff --git
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
index 960e3e2747f..f9a053122f6 100644
---
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
+++
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
@@ -1207,7 +1207,7 @@ public class BigQueryIO {
abstract Builder<T> setUseLegacySql(Boolean useLegacySql);
- abstract Builder<T> setWithTemplateCompatibility(Boolean
useTemplateCompatibility);
+ abstract Builder<T> setWithTemplateCompatibility(boolean
useTemplateCompatibility);
abstract Builder<T> setBigQueryServices(BigQueryServices
bigQueryServices);
@@ -1244,7 +1244,7 @@ public class BigQueryIO {
abstract Builder<T> setFromBeamRowFn(FromBeamRowFunction<T> fromRowFn);
- abstract Builder<T> setUseAvroLogicalTypes(Boolean useAvroLogicalTypes);
+ abstract Builder<T> setUseAvroLogicalTypes(boolean useAvroLogicalTypes);
abstract Builder<T> setBadRecordErrorHandler(
ErrorHandler<BadRecord, ?> badRecordErrorHandler);
@@ -1266,7 +1266,7 @@ public class BigQueryIO {
abstract @Nullable Boolean getUseLegacySql();
- abstract Boolean getWithTemplateCompatibility();
+ abstract boolean getWithTemplateCompatibility();
abstract BigQueryServices getBigQueryServices();
@@ -1301,7 +1301,7 @@ public class BigQueryIO {
abstract @Nullable FromBeamRowFunction<T> getFromBeamRowFn();
- abstract Boolean getUseAvroLogicalTypes();
+ abstract boolean getUseAvroLogicalTypes();
abstract ErrorHandler<BadRecord, ?> getBadRecordErrorHandler();
@@ -2750,11 +2750,11 @@ public class BigQueryIO {
abstract boolean getExtendedErrorInfo();
- abstract Boolean getSkipInvalidRows();
+ abstract boolean getSkipInvalidRows();
- abstract Boolean getIgnoreUnknownValues();
+ abstract boolean getIgnoreUnknownValues();
- abstract Boolean getIgnoreInsertIds();
+ abstract boolean getIgnoreInsertIds();
abstract int getMaxRetryJobs();
@@ -2764,19 +2764,19 @@ public class BigQueryIO {
abstract AppendRowsRequest.MissingValueInterpretation
getDefaultMissingValueInterpretation();
- abstract Boolean getOptimizeWrites();
+ abstract boolean getOptimizeWrites();
- abstract Boolean getUseBeamSchema();
+ abstract boolean getUseBeamSchema();
- abstract Boolean getAutoSharding();
+ abstract boolean getAutoSharding();
- abstract Boolean getPropagateSuccessful();
+ abstract boolean getPropagateSuccessful();
- abstract Boolean getAutoSchemaUpdate();
+ abstract boolean getAutoSchemaUpdate();
abstract @Nullable Class<T> getWriteProtosClass();
- abstract Boolean getDirectWriteProtos();
+ abstract boolean getDirectWriteProtos();
abstract @Nullable SerializableFunction<T, String>
getDeterministicRecordIdFn();
@@ -2865,11 +2865,11 @@ public class BigQueryIO {
abstract Builder<T> setExtendedErrorInfo(boolean extendedErrorInfo);
- abstract Builder<T> setSkipInvalidRows(Boolean skipInvalidRows);
+ abstract Builder<T> setSkipInvalidRows(boolean skipInvalidRows);
- abstract Builder<T> setIgnoreUnknownValues(Boolean ignoreUnknownValues);
+ abstract Builder<T> setIgnoreUnknownValues(boolean ignoreUnknownValues);
- abstract Builder<T> setIgnoreInsertIds(Boolean ignoreInsertIds);
+ abstract Builder<T> setIgnoreInsertIds(boolean ignoreInsertIds);
abstract Builder<T> setKmsKey(@Nullable String kmsKey);
@@ -2878,21 +2878,21 @@ public class BigQueryIO {
abstract Builder<T> setDefaultMissingValueInterpretation(
AppendRowsRequest.MissingValueInterpretation
missingValueInterpretation);
- abstract Builder<T> setOptimizeWrites(Boolean optimizeWrites);
+ abstract Builder<T> setOptimizeWrites(boolean optimizeWrites);
- abstract Builder<T> setUseBeamSchema(Boolean useBeamSchema);
+ abstract Builder<T> setUseBeamSchema(boolean useBeamSchema);
- abstract Builder<T> setAutoSharding(Boolean autoSharding);
+ abstract Builder<T> setAutoSharding(boolean autoSharding);
abstract Builder<T> setMaxRetryJobs(int maxRetryJobs);
- abstract Builder<T> setPropagateSuccessful(Boolean propagateSuccessful);
+ abstract Builder<T> setPropagateSuccessful(boolean propagateSuccessful);
- abstract Builder<T> setAutoSchemaUpdate(Boolean autoSchemaUpdate);
+ abstract Builder<T> setAutoSchemaUpdate(boolean autoSchemaUpdate);
abstract Builder<T> setWriteProtosClass(@Nullable Class<T> clazz);
- abstract Builder<T> setDirectWriteProtos(Boolean direct);
+ abstract Builder<T> setDirectWriteProtos(boolean direct);
abstract Builder<T> setDeterministicRecordIdFn(
SerializableFunction<T, String> toUniqueIdFunction);
diff --git
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java
index c2e891145ac..4c546f4963a 100644
---
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java
+++
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTranslation.java
@@ -193,9 +193,7 @@ public class BigQueryIOTranslation {
if (transform.getFromBeamRowFn() != null) {
fieldValues.put("from_beam_row_fn",
toByteArray(transform.getFromBeamRowFn()));
}
- if (transform.getUseAvroLogicalTypes() != null) {
- fieldValues.put("use_avro_logical_types",
transform.getUseAvroLogicalTypes());
- }
+ fieldValues.put("use_avro_logical_types",
transform.getUseAvroLogicalTypes());
if (transform.getDirectReadPicosTimestampPrecision() != null) {
fieldValues.put(
"direct_read_picos_timestamp_precision",
@@ -565,19 +563,11 @@ public class BigQueryIOTranslation {
fieldValues.put("custom_gcs_temp_location",
transform.getCustomGcsTempLocation().get());
}
fieldValues.put("extended_error_info", transform.getExtendedErrorInfo());
- if (transform.getSkipInvalidRows() != null) {
- fieldValues.put("skip_invalid_rows", transform.getSkipInvalidRows());
- }
- if (transform.getIgnoreUnknownValues() != null) {
- fieldValues.put("ignore_unknown_values",
transform.getIgnoreUnknownValues());
- }
- if (transform.getIgnoreInsertIds() != null) {
- fieldValues.put("ignore_insert_ids", transform.getIgnoreInsertIds());
- }
+ fieldValues.put("skip_invalid_rows", transform.getSkipInvalidRows());
+ fieldValues.put("ignore_unknown_values",
transform.getIgnoreUnknownValues());
+ fieldValues.put("ignore_insert_ids", transform.getIgnoreInsertIds());
fieldValues.put("max_retry_jobs", transform.getMaxRetryJobs());
- if (transform.getPropagateSuccessful() != null) {
- fieldValues.put("propagate_successful",
transform.getPropagateSuccessful());
- }
+ fieldValues.put("propagate_successful",
transform.getPropagateSuccessful());
if (transform.getKmsKey() != null) {
fieldValues.put("kms_key", transform.getKmsKey());
}
@@ -589,24 +579,14 @@ public class BigQueryIOTranslation {
"default_missing_value_interpretation",
toByteArray(transform.getDefaultMissingValueInterpretation()));
}
- if (transform.getOptimizeWrites() != null) {
- fieldValues.put("optimize_writes", transform.getOptimizeWrites());
- }
- if (transform.getUseBeamSchema() != null) {
- fieldValues.put("use_beam_schema", transform.getUseBeamSchema());
- }
- if (transform.getAutoSharding() != null) {
- fieldValues.put("auto_sharding", transform.getAutoSharding());
- }
- if (transform.getAutoSchemaUpdate() != null) {
- fieldValues.put("auto_schema_update", transform.getAutoSchemaUpdate());
- }
+ fieldValues.put("optimize_writes", transform.getOptimizeWrites());
+ fieldValues.put("use_beam_schema", transform.getUseBeamSchema());
+ fieldValues.put("auto_sharding", transform.getAutoSharding());
+ fieldValues.put("auto_schema_update", transform.getAutoSchemaUpdate());
if (transform.getWriteProtosClass() != null) {
fieldValues.put("write_protos_class",
toByteArray(transform.getWriteProtosClass()));
}
- if (transform.getDirectWriteProtos() != null) {
- fieldValues.put("direct_write_protos",
transform.getDirectWriteProtos());
- }
+ fieldValues.put("direct_write_protos", transform.getDirectWriteProtos());
if (transform.getDeterministicRecordIdFn() != null) {
fieldValues.put(
"deterministic_record_id_fn",
toByteArray(transform.getDeterministicRecordIdFn()));
diff --git
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java
index 6fde170e379..cdebf1172cb 100644
---
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java
+++
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java
@@ -48,7 +48,7 @@ class WritePartition<DestinationT>
abstract static class Result {
public abstract List<String> getFilenames();
- abstract Boolean isFirstPane();
+ abstract boolean isFirstPane();
}
static class ResultCoder extends AtomicCoder<Result> {
diff --git
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java
index 288b94ce081..c952522202f 100644
---
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java
+++
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java
@@ -101,7 +101,7 @@ class WriteTables<DestinationT extends @NonNull Object>
abstract static class Result {
abstract String getTableName();
- abstract Boolean isFirstPane();
+ abstract boolean isFirstPane();
}
static class ResultCoder extends AtomicCoder<WriteTables.Result> {
diff --git
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
index 3a69d1177f4..3494ca22be4 100644
---
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
+++
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
@@ -562,7 +562,7 @@ public class SpannerIO {
abstract Builder setTimestampBound(TimestampBound timestampBound);
- abstract Builder setBatching(Boolean batching);
+ abstract Builder setBatching(boolean batching);
abstract ReadAll build();
}
@@ -694,7 +694,7 @@ public class SpannerIO {
return withSpannerConfig(config.withRpcPriority(RpcPriority.HIGH));
}
- abstract Boolean getBatching();
+ abstract boolean getBatching();
@Override
public PCollection<Struct> expand(PCollection<ReadOperation> input) {
@@ -776,7 +776,7 @@ public class SpannerIO {
abstract @Nullable PartitionOptions getPartitionOptions();
- abstract Boolean getBatching();
+ abstract boolean getBatching();
abstract @Nullable TypeDescriptor<Struct> getTypeDescriptor();
@@ -799,7 +799,7 @@ public class SpannerIO {
abstract Builder setPartitionOptions(PartitionOptions partitionOptions);
- abstract Builder setBatching(Boolean batching);
+ abstract Builder setBatching(boolean batching);
abstract Builder setTypeDescriptor(TypeDescriptor<Struct>
typeDescriptor);
diff --git
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcher.java
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcher.java
index cbd7a75f7c4..f2a86e57474 100644
---
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcher.java
+++
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcher.java
@@ -90,7 +90,7 @@ public class BigqueryMatcher extends
TypeSafeMatcher<TableAndQuery>
@AutoValue
public abstract static class TableAndQuery {
public static TableAndQuery create(
- String applicationName, String projectId, String query, Boolean
usingStandardSql) {
+ String applicationName, String projectId, String query, boolean
usingStandardSql) {
return new AutoValue_BigqueryMatcher_TableAndQuery(
applicationName, projectId, query, usingStandardSql);
}
@@ -101,7 +101,7 @@ public class BigqueryMatcher extends
TypeSafeMatcher<TableAndQuery>
public abstract String getQuery();
- public abstract Boolean getUsingStandardSql();
+ public abstract boolean getUsingStandardSql();
}
@Override
diff --git
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
index a654f63deea..8d9a0f1bcfa 100644
---
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
+++
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
@@ -2047,13 +2047,13 @@ public class BigtableIOTest {
/** Error injection options for FakeBigtableService and FakeBigtableReader.
*/
@AutoValue
abstract static class FailureOptions implements Serializable {
- abstract Boolean getFailAtStart();
+ abstract boolean getFailAtStart();
- abstract Boolean getFailAtAdvance();
+ abstract boolean getFailAtAdvance();
- abstract Boolean getFailAtSplit();
+ abstract boolean getFailAtSplit();
- abstract Boolean getFailAtWriteRecord();
+ abstract boolean getFailAtWriteRecord();
static Builder builder() {
return new AutoValue_BigtableIOTest_FailureOptions.Builder()
@@ -2065,13 +2065,13 @@ public class BigtableIOTest {
@AutoValue.Builder
abstract static class Builder {
- abstract Builder setFailAtStart(Boolean failAtStart);
+ abstract Builder setFailAtStart(boolean failAtStart);
- abstract Builder setFailAtAdvance(Boolean failAtAdvance);
+ abstract Builder setFailAtAdvance(boolean failAtAdvance);
- abstract Builder setFailAtSplit(Boolean failAtSplit);
+ abstract Builder setFailAtSplit(boolean failAtSplit);
- abstract Builder setFailAtWriteRecord(Boolean failAtWriteRecord);
+ abstract Builder setFailAtWriteRecord(boolean failAtWriteRecord);
abstract FailureOptions build();
}
diff --git
a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/TestRowDBWritable.java
b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/TestRowDBWritable.java
index f8ef1e71d15..2e993f4eb1d 100644
---
a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/TestRowDBWritable.java
+++
b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/TestRowDBWritable.java
@@ -36,18 +36,18 @@ import org.apache.hadoop.mapreduce.lib.db.DBWritable;
@DefaultCoder(AvroCoder.class)
class TestRowDBWritable extends TestRow implements DBWritable, Writable {
- private Integer id;
+ private int id;
private String name;
public TestRowDBWritable() {}
- public TestRowDBWritable(Integer id, String name) {
+ public TestRowDBWritable(int id, String name) {
this.id = id;
this.name = name;
}
@Override
- public Integer id() {
+ public int id() {
return id;
}
diff --git
a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOIT.java
b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOIT.java
index 3b7787f7242..28da351ec46 100644
---
a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOIT.java
+++
b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOIT.java
@@ -157,7 +157,7 @@ public class HBaseIOIT {
@ProcessElement
public void processElement(ProcessContext c) {
c.output(
- new Put(c.element().id().toString().getBytes(StandardCharsets.UTF_8))
+ new
Put(String.valueOf(c.element().id()).getBytes(StandardCharsets.UTF_8))
.addColumn(COLUMN_FAMILY, COLUMN_HASH,
Bytes.toBytes(c.element().name())));
}
}
diff --git
a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
index e6db4d82712..c674ce84eed 100644
--- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
+++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
@@ -2740,7 +2740,7 @@ public class JdbcIO {
abstract @Nullable Long getMaxBatchBufferingDuration();
@Pure
- abstract Boolean getReturnResults();
+ abstract boolean getReturnResults();
@Pure
static Builder builder() {
@@ -2771,7 +2771,7 @@ public class JdbcIO {
abstract Builder<T, V> setMaxBatchBufferingDuration(
@Nullable Long maxBatchBufferingDuration);
- abstract Builder<T, V> setReturnResults(Boolean returnResults);
+ abstract Builder<T, V> setReturnResults(boolean returnResults);
abstract WriteFnSpec<T, V> build();
}
diff --git
a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaSourceDescriptor.java
b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaSourceDescriptor.java
index 67ee7a65783..b4bf707f24d 100644
---
a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaSourceDescriptor.java
+++
b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaSourceDescriptor.java
@@ -46,7 +46,7 @@ public abstract class KafkaSourceDescriptor implements
Serializable {
@SchemaFieldName("partition")
@SchemaFieldNumber("1")
@Pure
- abstract Integer getPartition();
+ abstract int getPartition();
@SchemaFieldName("start_read_offset")
@SchemaFieldNumber("2")
@@ -120,7 +120,7 @@ public abstract class KafkaSourceDescriptor implements
Serializable {
// TODO(BEAM-10677): Remove this function after AutoValueSchema is fixed.
static KafkaSourceDescriptor create(
String topic,
- Integer partition,
+ int partition,
Long start_read_offset,
Instant start_read_time,
Long stop_read_offset,
diff --git
a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java
b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java
index ab73946534c..07aa66cbe2f 100644
---
a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java
+++
b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java
@@ -362,7 +362,14 @@ class Call<RequestT, ResponseT> extends
PTransform<PCollection<RequestT>, Result
abstract static class Configuration<RequestT, ResponseT> implements
Serializable {
static <RequestT, ResponseT> Builder<RequestT, ResponseT> builder() {
- return new AutoValue_Call_Configuration.Builder<RequestT, ResponseT>();
+ return new AutoValue_Call_Configuration.Builder<RequestT, ResponseT>()
+ .setShouldRepeat(false)
+ .setTimeout(RequestResponseIO.DEFAULT_TIMEOUT)
+ .setCallShouldBackoff(new NoopCallShouldBackoff<>())
+ .setSleeperSupplier((SerializableSupplier<Sleeper>) () ->
Sleeper.DEFAULT)
+ .setBackOffSupplier(new DefaultSerializableBackoffSupplier())
+ .setMonitoringConfiguration(Monitoring.builder().build())
+ .setSetupTeardown(new NoopSetupTeardown());
}
/** The user custom code that converts a {@link RequestT} into a {@link
ResponseT}. */
@@ -390,7 +397,7 @@ class Call<RequestT, ResponseT> extends
PTransform<PCollection<RequestT>, Result
* invocations, using the {@link Repeater}, in the setting of {@link
* RequestResponseIO#REPEATABLE_ERROR_TYPES}. Defaults to false.
*/
- abstract Boolean getShouldRepeat();
+ abstract boolean getShouldRepeat();
/**
* The {@link CallShouldBackoff} that determines whether the {@link DoFn}
should hold {@link
@@ -438,64 +445,36 @@ class Call<RequestT, ResponseT> extends
PTransform<PCollection<RequestT>, Result
/** See {@link Configuration#getTimeout}. */
abstract Builder<RequestT, ResponseT> setTimeout(Duration value);
- abstract Optional<Duration> getTimeout();
+ abstract Duration getTimeout();
/** See {@link Configuration#getShouldRepeat}. */
- abstract Builder<RequestT, ResponseT> setShouldRepeat(Boolean value);
+ abstract Builder<RequestT, ResponseT> setShouldRepeat(boolean value);
- abstract Optional<Boolean> getShouldRepeat();
+ abstract boolean getShouldRepeat();
/** See {@link Configuration#getCallShouldBackoff}. */
abstract Builder<RequestT, ResponseT> setCallShouldBackoff(
CallShouldBackoff<ResponseT> value);
- abstract Optional<CallShouldBackoff<ResponseT>> getCallShouldBackoff();
+ abstract CallShouldBackoff<ResponseT> getCallShouldBackoff();
/** See {@link Configuration#getSleeperSupplier}. */
abstract Builder<RequestT, ResponseT>
setSleeperSupplier(SerializableSupplier<Sleeper> value);
- abstract Optional<SerializableSupplier<Sleeper>> getSleeperSupplier();
+ abstract SerializableSupplier<Sleeper> getSleeperSupplier();
/** See {@link Configuration#getBackOffSupplier}. */
abstract Builder<RequestT, ResponseT>
setBackOffSupplier(SerializableSupplier<BackOff> value);
- abstract Optional<SerializableSupplier<BackOff>> getBackOffSupplier();
+ abstract SerializableSupplier<BackOff> getBackOffSupplier();
abstract Builder<RequestT, ResponseT>
setMonitoringConfiguration(Monitoring value);
- abstract Optional<Monitoring> getMonitoringConfiguration();
+ abstract Monitoring getMonitoringConfiguration();
abstract Configuration<RequestT, ResponseT> autoBuild();
final Configuration<RequestT, ResponseT> build() {
- if (!getSetupTeardown().isPresent()) {
- setSetupTeardown(new NoopSetupTeardown());
- }
-
- if (!getShouldRepeat().isPresent()) {
- setShouldRepeat(false);
- }
-
- if (!getTimeout().isPresent()) {
- setTimeout(RequestResponseIO.DEFAULT_TIMEOUT);
- }
-
- if (!getCallShouldBackoff().isPresent()) {
- setCallShouldBackoff(new NoopCallShouldBackoff<>());
- }
-
- if (!getSleeperSupplier().isPresent()) {
- setSleeperSupplier((SerializableSupplier<Sleeper>) () ->
Sleeper.DEFAULT);
- }
-
- if (!getBackOffSupplier().isPresent()) {
- setBackOffSupplier(new DefaultSerializableBackoffSupplier());
- }
-
- if (!getMonitoringConfiguration().isPresent()) {
- setMonitoringConfiguration(Monitoring.builder().build());
- }
-
return autoBuild();
}
}
diff --git
a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Monitoring.java
b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Monitoring.java
index 77f964b4c93..fcaad4f326e 100644
---
a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Monitoring.java
+++
b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Monitoring.java
@@ -20,7 +20,6 @@ package org.apache.beam.io.requestresponse;
import com.google.auto.value.AutoValue;
import java.io.Serializable;
import java.util.Arrays;
-import java.util.Optional;
import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.Metric;
import org.apache.beam.sdk.transforms.DoFn;
@@ -136,58 +135,74 @@ public abstract class Monitoring implements Serializable {
}
public static Builder builder() {
- return new AutoValue_Monitoring.Builder();
+ return new AutoValue_Monitoring.Builder()
+ .setCountRequests(false)
+ .setCountResponses(false)
+ .setCountCalls(false)
+ .setCountFailures(false)
+ .setCountSetup(false)
+ .setCountTeardown(false)
+ .setCountBackoffs(false)
+ .setCountSleeps(false)
+ .setCountShouldBackoff(false)
+ .setCountCacheReadRequests(false)
+ .setCountCacheReadNulls(false)
+ .setCountCacheReadNonNulls(false)
+ .setCountCacheReadFailures(false)
+ .setCountCacheWriteRequests(false)
+ .setCountCacheWriteSuccesses(false)
+ .setCountCacheWriteFailures(false);
}
/** Count incoming request elements processed by {@link Call}'s {@link
DoFn}. */
- public abstract Boolean getCountRequests();
+ public abstract boolean getCountRequests();
/**
* Count outgoing responses resulting from {@link Call}'s successful {@link
Caller} invocation.
*/
- public abstract Boolean getCountResponses();
+ public abstract boolean getCountResponses();
/** Count invocations of {@link Caller#call}. */
- public abstract Boolean getCountCalls();
+ public abstract boolean getCountCalls();
/** Count failures resulting from {@link Call}'s successful {@link Caller}
invocation. */
- public abstract Boolean getCountFailures();
+ public abstract boolean getCountFailures();
/** Count invocations of {@link SetupTeardown#setup}. */
- public abstract Boolean getCountSetup();
+ public abstract boolean getCountSetup();
/** Count invocations of {@link SetupTeardown#teardown}. */
- public abstract Boolean getCountTeardown();
+ public abstract boolean getCountTeardown();
/** Count invocations of {@link BackOff#nextBackOffMillis}. */
- public abstract Boolean getCountBackoffs();
+ public abstract boolean getCountBackoffs();
/** Count invocations of {@link Sleeper#sleep}. */
- public abstract Boolean getCountSleeps();
+ public abstract boolean getCountSleeps();
/** Count when {@link CallShouldBackoff#isTrue} is found true. */
- public abstract Boolean getCountShouldBackoff();
+ public abstract boolean getCountShouldBackoff();
/** Count number of attempts to read from the {@link Cache}. */
- public abstract Boolean getCountCacheReadRequests();
+ public abstract boolean getCountCacheReadRequests();
/** Count associated null values resulting from {@link Cache} reads. */
- public abstract Boolean getCountCacheReadNulls();
+ public abstract boolean getCountCacheReadNulls();
/** Count associated non-null values resulting from {@link Cache} reads. */
- public abstract Boolean getCountCacheReadNonNulls();
+ public abstract boolean getCountCacheReadNonNulls();
/** Count {@link Cache} read failures. */
- public abstract Boolean getCountCacheReadFailures();
+ public abstract boolean getCountCacheReadFailures();
/** Count number of attempts to write to the {@link Cache}. */
- public abstract Boolean getCountCacheWriteRequests();
+ public abstract boolean getCountCacheWriteRequests();
/** Count {@link Cache} write successes. */
- public abstract Boolean getCountCacheWriteSuccesses();
+ public abstract boolean getCountCacheWriteSuccesses();
/** Count {@link Cache} write failures. */
- public abstract Boolean getCountCacheWriteFailures();
+ public abstract boolean getCountCacheWriteFailures();
/**
* Turns on all monitoring. The purpose of this method is, when used with
{@link #toBuilder} and
@@ -240,123 +255,38 @@ public abstract class Monitoring implements Serializable
{
@AutoValue.Builder
public abstract static class Builder {
- public abstract Builder setCountRequests(Boolean value);
+ public abstract Builder setCountRequests(boolean value);
- public abstract Builder setCountResponses(Boolean value);
+ public abstract Builder setCountResponses(boolean value);
- public abstract Builder setCountCalls(Boolean value);
+ public abstract Builder setCountCalls(boolean value);
- public abstract Builder setCountFailures(Boolean value);
+ public abstract Builder setCountFailures(boolean value);
- public abstract Builder setCountSetup(Boolean value);
+ public abstract Builder setCountSetup(boolean value);
- public abstract Builder setCountTeardown(Boolean value);
+ public abstract Builder setCountTeardown(boolean value);
- public abstract Builder setCountBackoffs(Boolean value);
+ public abstract Builder setCountBackoffs(boolean value);
- public abstract Builder setCountSleeps(Boolean value);
+ public abstract Builder setCountSleeps(boolean value);
- public abstract Builder setCountShouldBackoff(Boolean value);
+ public abstract Builder setCountShouldBackoff(boolean value);
- public abstract Builder setCountCacheReadRequests(Boolean value);
+ public abstract Builder setCountCacheReadRequests(boolean value);
- public abstract Builder setCountCacheReadNulls(Boolean value);
+ public abstract Builder setCountCacheReadNulls(boolean value);
- public abstract Builder setCountCacheReadNonNulls(Boolean value);
+ public abstract Builder setCountCacheReadNonNulls(boolean value);
- public abstract Builder setCountCacheReadFailures(Boolean value);
+ public abstract Builder setCountCacheReadFailures(boolean value);
- public abstract Builder setCountCacheWriteRequests(Boolean value);
+ public abstract Builder setCountCacheWriteRequests(boolean value);
- public abstract Builder setCountCacheWriteSuccesses(Boolean value);
+ public abstract Builder setCountCacheWriteSuccesses(boolean value);
- public abstract Builder setCountCacheWriteFailures(Boolean value);
+ public abstract Builder setCountCacheWriteFailures(boolean value);
- abstract Optional<Boolean> getCountRequests();
-
- abstract Optional<Boolean> getCountResponses();
-
- abstract Optional<Boolean> getCountCalls();
-
- abstract Optional<Boolean> getCountFailures();
-
- abstract Optional<Boolean> getCountSetup();
-
- abstract Optional<Boolean> getCountTeardown();
-
- abstract Optional<Boolean> getCountBackoffs();
-
- abstract Optional<Boolean> getCountSleeps();
-
- abstract Optional<Boolean> getCountShouldBackoff();
-
- abstract Optional<Boolean> getCountCacheReadRequests();
-
- abstract Optional<Boolean> getCountCacheReadNulls();
-
- abstract Optional<Boolean> getCountCacheReadNonNulls();
-
- abstract Optional<Boolean> getCountCacheReadFailures();
-
- abstract Optional<Boolean> getCountCacheWriteRequests();
-
- abstract Optional<Boolean> getCountCacheWriteSuccesses();
-
- abstract Optional<Boolean> getCountCacheWriteFailures();
-
- abstract Monitoring autoBuild();
-
- public final Monitoring build() {
- if (!getCountRequests().isPresent()) {
- setCountRequests(false);
- }
- if (!getCountResponses().isPresent()) {
- setCountResponses(false);
- }
- if (!getCountCalls().isPresent()) {
- setCountCalls(false);
- }
- if (!getCountFailures().isPresent()) {
- setCountFailures(false);
- }
- if (!getCountSetup().isPresent()) {
- setCountSetup(false);
- }
- if (!getCountTeardown().isPresent()) {
- setCountTeardown(false);
- }
- if (!getCountBackoffs().isPresent()) {
- setCountBackoffs(false);
- }
- if (!getCountSleeps().isPresent()) {
- setCountSleeps(false);
- }
- if (!getCountShouldBackoff().isPresent()) {
- setCountShouldBackoff(false);
- }
- if (!getCountCacheReadRequests().isPresent()) {
- setCountCacheReadRequests(false);
- }
- if (!getCountCacheReadNulls().isPresent()) {
- setCountCacheReadNulls(false);
- }
- if (!getCountCacheReadNonNulls().isPresent()) {
- setCountCacheReadNonNulls(false);
- }
- if (!getCountCacheReadFailures().isPresent()) {
- setCountCacheReadFailures(false);
- }
- if (!getCountCacheWriteRequests().isPresent()) {
- setCountCacheWriteRequests(false);
- }
- if (!getCountCacheWriteSuccesses().isPresent()) {
- setCountCacheWriteSuccesses(false);
- }
- if (!getCountCacheWriteFailures().isPresent()) {
- setCountCacheWriteFailures(false);
- }
-
- return autoBuild();
- }
+ public abstract Monitoring build();
}
}
diff --git
a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RequestResponseIOTest.java
b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RequestResponseIOTest.java
index 5a199225f39..d851e2039eb 100644
---
a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RequestResponseIOTest.java
+++
b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/requestresponse/RequestResponseIOTest.java
@@ -390,14 +390,14 @@ public class RequestResponseIOTest {
abstract String getAString();
- abstract Long getALong();
+ abstract long getALong();
@AutoValue.Builder
abstract static class Builder {
abstract Builder setAString(String value);
- abstract Builder setALong(Long value);
+ abstract Builder setALong(long value);
abstract Request build();
}
@@ -411,14 +411,14 @@ public class RequestResponseIOTest {
abstract String getAString();
- abstract Long getALong();
+ abstract long getALong();
@AutoValue.Builder
abstract static class Builder {
abstract Builder setAString(String value);
- abstract Builder setALong(Long value);
+ abstract Builder setALong(long value);
abstract Response build();
}
diff --git
a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/TestHelper.java
b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/TestHelper.java
index e04785db4ee..de55865e4ad 100644
---
a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/TestHelper.java
+++
b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/TestHelper.java
@@ -39,7 +39,7 @@ public class TestHelper {
@Override
public List<String> mapRow(TestRow element) {
List<String> res = new ArrayList<>();
- res.add(element.id().toString());
+ res.add(String.valueOf(element.id()));
res.add(element.name());
return res;
}
diff --git
a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java
b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java
index 21274237f46..e6cd35b63b4 100644
---
a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java
+++
b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java
@@ -318,7 +318,7 @@ public class Solace {
/** Whether the message was published or not. */
@SchemaFieldNumber("1")
- public abstract Boolean getPublished();
+ public abstract boolean getPublished();
/**
* The publishing latency in nanoseconds. This is the difference between
the time the message
@@ -340,7 +340,7 @@ public class Solace {
public abstract static class Builder {
public abstract Builder setMessageId(String messageId);
- public abstract Builder setPublished(Boolean published);
+ public abstract Builder setPublished(boolean published);
public abstract Builder setLatencyNanos(Long latencyNanos);
diff --git
a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/HttpEventPublisher.java
b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/HttpEventPublisher.java
index d6fded956a2..cd7899ce8f4 100644
---
a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/HttpEventPublisher.java
+++
b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/HttpEventPublisher.java
@@ -113,9 +113,9 @@ abstract class HttpEventPublisher {
@SuppressWarnings("mutable")
abstract byte @Nullable [] rootCaCertificate();
- abstract Boolean disableCertificateValidation();
+ abstract @Nullable Boolean disableCertificateValidation();
- abstract Boolean enableGzipHttpCompression();
+ abstract @Nullable Boolean enableGzipHttpCompression();
/**
* Executes a POST for the list of {@link SplunkEvent} objects into Splunk's
Http Event Collector
diff --git
a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java
b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java
index 2771ecd0164..2c9c65191fb 100644
---
a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java
+++
b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/testing/TestSchemaTransformProvider.java
@@ -50,13 +50,13 @@ public class TestSchemaTransformProvider
public abstract String getExtraString();
@SchemaFieldDescription("Integer to add to each row element.")
- public abstract Integer getExtraInteger();
+ public abstract int getExtraInteger();
@AutoValue.Builder
public abstract static class Builder {
public abstract Builder setExtraString(String extraString);
- public abstract Builder setExtraInteger(Integer extraInteger);
+ public abstract Builder setExtraInteger(int extraInteger);
public abstract Config build();
}
@@ -65,7 +65,7 @@ public class TestSchemaTransformProvider
@Override
public SchemaTransform from(Config config) {
String extraString = config.getExtraString();
- Integer extraInteger = config.getExtraInteger();
+ int extraInteger = config.getExtraInteger();
return new SchemaTransform() {
@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {