Skip to content

Commit 2154315

Browse files
committed
fix spotbugs and checkstyle
1 parent 610efd3 commit 2154315

File tree

4 files changed

+57
-58
lines changed

4 files changed

+57
-58
lines changed

sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableBiFunctions.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,13 +22,13 @@
2222
/** Useful {@link SerializableFunction} overrides. */
2323
public class SerializableBiFunctions {
2424
/** Always returns the first argument. */
25-
public static <T, U, R> SerializableBiFunction<T, U, T> Select1st(
25+
public static <T, U, R> SerializableBiFunction<T, U, T> select1st(
2626
SerializableBiFunction<@Nullable T, @Nullable U, R> biFunction) {
2727
return (t, u) -> t;
2828
}
2929

3030
/** Always returns the second argument. */
31-
public static <T, U, R> SerializableBiFunction<T, U, U> Select2nd(
31+
public static <T, U, R> SerializableBiFunction<T, U, U> select2nd(
3232
SerializableBiFunction<@Nullable T, @Nullable U, R> biFunction) {
3333
return (t, u) -> u;
3434
}

sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java

Lines changed: 38 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@
2121
import static java.time.format.DateTimeFormatter.ISO_LOCAL_TIME;
2222
import static java.util.stream.Collectors.toList;
2323
import static java.util.stream.Collectors.toMap;
24-
import static org.apache.beam.sdk.io.gcp.bigquery.TableRowToStorageApiProto.DATETIME_SPACE_FORMATTER;
2524
import static org.apache.beam.sdk.values.Row.toRow;
2625

2726
import com.google.api.services.bigquery.model.TableFieldSchema;
@@ -35,6 +34,7 @@
3534
import java.time.LocalDate;
3635
import java.time.LocalDateTime;
3736
import java.time.LocalTime;
37+
import java.time.ZoneOffset;
3838
import java.time.format.DateTimeParseException;
3939
import java.util.ArrayList;
4040
import java.util.Collections;
@@ -171,11 +171,46 @@ public abstract static class Builder {
171171
}
172172

173173
private static final String BIGQUERY_TIME_PATTERN = "HH:mm:ss[.SSSSSS]";
174-
private static final java.time.format.DateTimeFormatter BIGQUERY_TIME_FORMATTER =
174+
static final java.time.format.DateTimeFormatter BIGQUERY_TIME_FORMATTER =
175175
java.time.format.DateTimeFormatter.ofPattern(BIGQUERY_TIME_PATTERN);
176-
private static final java.time.format.DateTimeFormatter BIGQUERY_DATETIME_FORMATTER =
176+
static final java.time.format.DateTimeFormatter BIGQUERY_DATETIME_FORMATTER =
177177
java.time.format.DateTimeFormatter.ofPattern("uuuu-MM-dd'T'" + BIGQUERY_TIME_PATTERN);
178178

179+
// Custom formatter that accepts "2022-05-09 18:04:59.123456"
180+
// The old dremel parser accepts this format, and so does insertall. We need to accept it
181+
// for backwards compatibility, and it is based on UTC time.
182+
static final java.time.format.DateTimeFormatter DATETIME_SPACE_FORMATTER =
183+
new java.time.format.DateTimeFormatterBuilder()
184+
.append(java.time.format.DateTimeFormatter.ISO_LOCAL_DATE)
185+
.optionalStart()
186+
.appendLiteral(' ')
187+
.optionalEnd()
188+
.optionalStart()
189+
.appendLiteral('T')
190+
.optionalEnd()
191+
.append(java.time.format.DateTimeFormatter.ISO_LOCAL_TIME)
192+
.toFormatter()
193+
.withZone(ZoneOffset.UTC);
194+
195+
static final java.time.format.DateTimeFormatter TIMESTAMP_FORMATTER =
196+
new java.time.format.DateTimeFormatterBuilder()
197+
// 'yyyy-MM-dd(T| )HH:mm:ss.SSSSSSSSS'
198+
.append(DATETIME_SPACE_FORMATTER)
199+
// 'yyyy-MM-dd(T| )HH:mm:ss.SSSSSSSSS(+HH:mm:ss|Z)'
200+
.optionalStart()
201+
.appendOffsetId()
202+
.optionalEnd()
203+
.optionalStart()
204+
.appendOffset("+HH:mm", "+00:00")
205+
.optionalEnd()
206+
// 'yyyy-MM-dd(T| )HH:mm:ss.SSSSSSSSS [time_zone]', time_zone -> UTC, Asia/Kolkata, etc
207+
// if both an offset and a time zone are provided, the offset takes precedence
208+
.optionalStart()
209+
.appendLiteral(' ')
210+
.parseCaseSensitive()
211+
.appendZoneRegionId()
212+
.toFormatter();
213+
179214
private static final DateTimeFormatter BIGQUERY_TIMESTAMP_PRINTER;
180215

181216
/**

sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java

Lines changed: 14 additions & 50 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,8 @@
1818
package org.apache.beam.sdk.io.gcp.bigquery;
1919

2020
import static java.util.stream.Collectors.toList;
21+
import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.DATETIME_SPACE_FORMATTER;
22+
import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.TIMESTAMP_FORMATTER;
2123

2224
import com.google.api.services.bigquery.model.TableCell;
2325
import com.google.api.services.bigquery.model.TableRow;
@@ -50,8 +52,6 @@
5052
import java.time.LocalDateTime;
5153
import java.time.LocalTime;
5254
import java.time.ZoneOffset;
53-
import java.time.format.DateTimeFormatter;
54-
import java.time.format.DateTimeFormatterBuilder;
5555
import java.time.format.DateTimeParseException;
5656
import java.util.AbstractMap;
5757
import java.util.ArrayList;
@@ -81,42 +81,6 @@
8181
* with the Storage write API.
8282
*/
8383
public class TableRowToStorageApiProto {
84-
85-
// Custom formatter that accepts "2022-05-09 18:04:59.123456"
86-
// The old dremel parser accepts this format, and so does insertall. We need to accept it
87-
// for backwards compatibility, and it is based on UTC time.
88-
static final DateTimeFormatter DATETIME_SPACE_FORMATTER =
89-
new DateTimeFormatterBuilder()
90-
.append(DateTimeFormatter.ISO_LOCAL_DATE)
91-
.optionalStart()
92-
.appendLiteral(' ')
93-
.optionalEnd()
94-
.optionalStart()
95-
.appendLiteral('T')
96-
.optionalEnd()
97-
.append(DateTimeFormatter.ISO_LOCAL_TIME)
98-
.toFormatter()
99-
.withZone(ZoneOffset.UTC);
100-
101-
static final DateTimeFormatter TIMESTAMP_FORMATTER =
102-
new DateTimeFormatterBuilder()
103-
// 'yyyy-MM-dd(T| )HH:mm:ss.SSSSSSSSS'
104-
.append(DATETIME_SPACE_FORMATTER)
105-
// 'yyyy-MM-dd(T| )HH:mm:ss.SSSSSSSSS(+HH:mm:ss|Z)'
106-
.optionalStart()
107-
.appendOffsetId()
108-
.optionalEnd()
109-
.optionalStart()
110-
.appendOffset("+HH:mm", "+00:00")
111-
.optionalEnd()
112-
// 'yyyy-MM-dd(T| )HH:mm:ss.SSSSSSSSS [time_zone]', time_zone -> UTC, Asia/Kolkata, etc
113-
// if both an offset and a time zone are provided, the offset takes precedence
114-
.optionalStart()
115-
.appendLiteral(' ')
116-
.parseCaseSensitive()
117-
.appendZoneRegionId()
118-
.toFormatter();
119-
12084
abstract static class SchemaConversionException extends Exception {
12185
SchemaConversionException(String msg) {
12286
super(msg);
@@ -220,8 +184,8 @@ private static String getPrettyFieldName(String fullName) {
220184
.build();
221185

222186
@FunctionalInterface
223-
public interface ThrowingBiFunction<T, U, R> {
224-
R apply(T t, U u) throws SchemaConversionException;
187+
public interface ThrowingBiFunction<FirstInputT, SecondInputT, OutputT> {
188+
OutputT apply(FirstInputT t, SecondInputT u) throws SchemaConversionException;
225189
}
226190

227191
// Map of functions to convert json values into the value expected in the Vortex proto object.
@@ -1333,15 +1297,15 @@ public static TableRow tableRowFromMessageUseSetF(
13331297

13341298
// Our process for generating descriptors modifies the names of nested descriptors for wrapper
13351299
// types, so we record them here.
1336-
private static String FLOAT_VALUE_DESCRIPTOR_NAME = "google_protobuf_FloatValue";
1337-
private static String DOUBLE_VALUE_DESCRIPTOR_NAME = "google_protobuf_DoubleValue";
1338-
private static String BOOL_VALUE_DESCRIPTOR_NAME = "google_protobuf_BoolValue";
1339-
private static String INT32_VALUE_DESCRIPTOR_NAME = "google_protobuf_Int32Value";
1340-
private static String INT64_VALUE_DESCRIPTOR_NAME = "google_protobuf_Int64Value";
1341-
private static String UINT32_VALUE_DESCRIPTOR_NAME = "google_protobuf_UInt32Value";
1342-
private static String UINT64_VALUE_DESCRIPTOR_NAME = "google_protobuf_UInt64Value";
1343-
private static String BYTES_VALUE_DESCRIPTOR_NAME = "google_protobuf_BytesValue";
1344-
private static String TIMESTAMP_VALUE_DESCRIPTOR_NAME = "google_protobuf_Timestamp";
1300+
private static final String FLOAT_VALUE_DESCRIPTOR_NAME = "google_protobuf_FloatValue";
1301+
private static final String DOUBLE_VALUE_DESCRIPTOR_NAME = "google_protobuf_DoubleValue";
1302+
private static final String BOOL_VALUE_DESCRIPTOR_NAME = "google_protobuf_BoolValue";
1303+
private static final String INT32_VALUE_DESCRIPTOR_NAME = "google_protobuf_Int32Value";
1304+
private static final String INT64_VALUE_DESCRIPTOR_NAME = "google_protobuf_Int64Value";
1305+
private static final String UINT32_VALUE_DESCRIPTOR_NAME = "google_protobuf_UInt32Value";
1306+
private static final String UINT64_VALUE_DESCRIPTOR_NAME = "google_protobuf_UInt64Value";
1307+
private static final String BYTES_VALUE_DESCRIPTOR_NAME = "google_protobuf_BytesValue";
1308+
private static final String TIMESTAMP_VALUE_DESCRIPTOR_NAME = "google_protobuf_Timestamp";
13451309

13461310
// Translate a proto message value into a json value. If useSetF==false, this will fail with
13471311
// Optional.empty() if
@@ -1551,7 +1515,7 @@ public static Object jsonValueFromMessageValue(
15511515
if (isProtoFieldTypeInteger(fieldDescriptor.getType())) {
15521516
long packedDateTime = Long.valueOf(fieldValue.toString());
15531517
return CivilTimeEncoder.decodePacked64DatetimeMicrosAsJavaTime(packedDateTime)
1554-
.format(DATETIME_SPACE_FORMATTER);
1518+
.format(BigQueryUtils.BIGQUERY_DATETIME_FORMATTER);
15551519
} else {
15561520
return fieldValue.toString();
15571521
}

sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoTest.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,8 @@
1717
*/
1818
package org.apache.beam.sdk.io.gcp.bigquery;
1919

20-
import static org.apache.beam.sdk.io.gcp.bigquery.TableRowToStorageApiProto.DATETIME_SPACE_FORMATTER;
20+
import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.DATETIME_SPACE_FORMATTER;
21+
import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.TIMESTAMP_FORMATTER;
2122
import static org.apache.beam.sdk.io.gcp.bigquery.TableRowToStorageApiProto.TYPE_MAP_PROTO_CONVERTERS;
2223
import static org.junit.Assert.assertEquals;
2324
import static org.junit.Assert.assertFalse;
@@ -1414,8 +1415,7 @@ private TableRow normalizeTableRowF(
14141415
long epochSeconds = timestampLongValue / 1_000_000L;
14151416
long nanoAdjustment = (timestampLongValue % 1_000_000L) * 1_000L;
14161417
Instant instant = Instant.ofEpochSecond(epochSeconds, nanoAdjustment);
1417-
return LocalDateTime.ofInstant(instant, ZoneOffset.UTC)
1418-
.format(TableRowToStorageApiProto.TIMESTAMP_FORMATTER);
1418+
return LocalDateTime.ofInstant(instant, ZoneOffset.UTC).format(TIMESTAMP_FORMATTER);
14191419
case DATE:
14201420
int daysInt = (int) convertedValue;
14211421
return LocalDate.ofEpochDay(daysInt).toString();

0 commit comments

Comments
 (0)