Checkpointing disabled (NOT RECOMMENDED!): no consistency guarantee.
41 | *
42 | * @param Type of input to sink.
43 | */
44 | public class BigQueryExactlyOnceSink extends BigQueryBaseSink
45 | implements TwoPhaseCommittingStatefulSink {
46 |
47 | BigQueryExactlyOnceSink(BigQuerySinkConfig sinkConfig) {
48 | super(sinkConfig);
49 | traceId = BigQueryServicesImpl.generateTraceId(UUID.randomUUID().toString());
50 | }
51 |
52 | @Override
53 | public PrecommittingStatefulSinkWriter
54 | createWriter(InitContext context) {
55 | checkParallelism(context.getNumberOfParallelSubtasks());
56 | return new BigQueryBufferedWriter<>(
57 | tablePath,
58 | connectOptions,
59 | schemaProvider,
60 | serializer,
61 | createTableOptions(),
62 | fatalizeSerializer,
63 | maxParallelism,
64 | traceId,
65 | context);
66 | }
67 |
68 | @Override
69 | public PrecommittingStatefulSinkWriter
70 | restoreWriter(InitContext context, Collection recoveredState) {
71 | if (recoveredState == null || recoveredState.isEmpty()) {
72 | return createWriter(context);
73 | }
74 | // If multiple states are found, restore one with the latest checkpoint.
75 | BigQueryWriterState stateToRestore =
76 | recoveredState.stream()
77 | .max(Comparator.comparingLong(state -> state.getCheckpointId()))
78 | .get();
79 | return new BigQueryBufferedWriter<>(
80 | stateToRestore.getStreamName(),
81 | stateToRestore.getStreamOffset(),
82 | tablePath,
83 | stateToRestore.getTotalRecordsSeen(),
84 | stateToRestore.getTotalRecordsWritten(),
85 | stateToRestore.getTotalRecordsCommitted(),
86 | connectOptions,
87 | schemaProvider,
88 | serializer,
89 | createTableOptions(),
90 | fatalizeSerializer,
91 | maxParallelism,
92 | traceId,
93 | context);
94 | }
95 |
96 | @Override
97 | public Committer createCommitter() {
98 | return new BigQueryCommitter(connectOptions);
99 | }
100 |
101 | @Override
102 | public SimpleVersionedSerializer getCommittableSerializer() {
103 | return new BigQueryCommittableSerializer();
104 | }
105 |
106 | @Override
107 | public SimpleVersionedSerializer getWriterStateSerializer() {
108 | return new BigQueryWriterStateSerializer();
109 | }
110 | }
111 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/BigQuerySink.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2024 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink;
18 |
19 | import org.apache.flink.api.connector.sink2.Sink;
20 | import org.apache.flink.connector.base.DeliveryGuarantee;
21 |
22 | import org.slf4j.Logger;
23 | import org.slf4j.LoggerFactory;
24 |
25 | /**
26 | * Class wrapping BigQuery sinks with appropriate configurations.
27 | *
28 | *
With {@link DeliveryGuarantee#AT_LEAST_ONCE}, the Sink added to Flink job will be {@link
29 | * BigQueryDefaultSink}.
30 | *
31 | *
With {@link DeliveryGuarantee#EXACTLY_ONCE}, the Sink added to Flink job will be {@link
32 | * BigQueryExactlyOnceSink}.
33 | *
34 | *
Eventual data consistency at destination is also dependent on checkpointing mode. Look at
35 | * {@link BigQueryDefaultSink} and {@link BigQueryExactlyOnceSink} for write consistencies offered
36 | * across combinations of {@link CheckpointingMode} and sink's {@link DeliveryGuarantee}. It is
37 | * recommended that checkpointing is enabled to avoid unexpected behavior.
38 | */
39 | public class BigQuerySink {
40 |
41 | private static final Logger LOG = LoggerFactory.getLogger(BigQuerySink.class);
42 |
43 | public static Sink get(BigQuerySinkConfig sinkConfig) {
44 | if (sinkConfig.getDeliveryGuarantee() == DeliveryGuarantee.AT_LEAST_ONCE) {
45 | return new BigQueryDefaultSink<>(sinkConfig);
46 | }
47 | if (sinkConfig.getDeliveryGuarantee() == DeliveryGuarantee.EXACTLY_ONCE) {
48 | return new BigQueryExactlyOnceSink<>(sinkConfig);
49 | }
50 | LOG.error(
51 | "BigQuery sink does not support {} delivery guarantee. Use AT_LEAST_ONCE or EXACTLY_ONCE.",
52 | sinkConfig.getDeliveryGuarantee());
53 | throw new UnsupportedOperationException(
54 | String.format("%s is not supported", sinkConfig.getDeliveryGuarantee()));
55 | }
56 | }
57 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/TwoPhaseCommittingStatefulSink.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2024 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink;
18 |
19 | import org.apache.flink.annotation.Internal;
20 | import org.apache.flink.api.connector.sink2.Sink;
21 | import org.apache.flink.api.connector.sink2.StatefulSink;
22 | import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink;
23 |
24 | import java.io.IOException;
25 | import java.util.Collection;
26 |
27 | /**
28 | * A combination of {@link TwoPhaseCommittingSink} and {@link StatefulSink}.
29 | *
30 | *
Interface for a sink that supports TPC protocol and statefulness.
31 | *
32 | * @param Type of the sink's input.
33 | * @param Type of the sink writer's state.
34 | * @param Type of the committables.
35 | */
36 | @Internal
37 | public interface TwoPhaseCommittingStatefulSink
38 | extends TwoPhaseCommittingSink, StatefulSink {
39 |
40 | @Override
41 | PrecommittingStatefulSinkWriter createWriter(
42 | Sink.InitContext context) throws IOException;
43 |
44 | @Override
45 | PrecommittingStatefulSinkWriter restoreWriter(
46 | Sink.InitContext context, Collection recoveredState) throws IOException;
47 |
48 | /**
49 | * A combination of {@link PrecommittingSinkWriter} and {@link StatefulSinkWriter}.
50 | *
51 | *
Interface for a writer that supports TPC protocol and statefulness.
52 | *
53 | * @param Type of the sink's input.
54 | * @param Type of the sink writer's state.
55 | * @param Type of the committables.
56 | */
57 | interface PrecommittingStatefulSinkWriter
58 | extends TwoPhaseCommittingSink.PrecommittingSinkWriter,
59 | StatefulSink.StatefulSinkWriter {}
60 | }
61 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/committer/BigQueryCommittable.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2024 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink.committer;
18 |
19 | import com.google.cloud.flink.bigquery.sink.state.BigQueryStreamState;
20 |
21 | /**
22 | * Information required for a commit operation, passed from {@link BigQueryBufferedWriter} to {@link
23 | * BigQueryCommitter}.
24 | */
25 | public class BigQueryCommittable extends BigQueryStreamState {
26 |
27 | private final long producerId;
28 |
29 | public BigQueryCommittable(long producerId, String streamName, long streamOffset) {
30 | super(streamName, streamOffset);
31 | this.producerId = producerId;
32 | }
33 |
34 | public long getProducerId() {
35 | return producerId;
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/committer/BigQueryCommittableSerializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2024 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink.committer;
18 |
19 | import org.apache.flink.core.io.SimpleVersionedSerializer;
20 |
21 | import java.io.ByteArrayInputStream;
22 | import java.io.ByteArrayOutputStream;
23 | import java.io.DataInputStream;
24 | import java.io.DataOutputStream;
25 | import java.io.IOException;
26 |
27 | /** Serializer and deserializer for {@link BigQueryCommittable}. */
28 | public class BigQueryCommittableSerializer
29 | implements SimpleVersionedSerializer {
30 |
31 | @Override
32 | public int getVersion() {
33 | return 1;
34 | }
35 |
36 | @Override
37 | public byte[] serialize(BigQueryCommittable committable) throws IOException {
38 | try (final ByteArrayOutputStream baos = new ByteArrayOutputStream();
39 | final DataOutputStream out = new DataOutputStream(baos)) {
40 | out.writeLong(committable.getProducerId());
41 | out.writeUTF(committable.getStreamName());
42 | out.writeLong(committable.getStreamOffset());
43 | out.flush();
44 | return baos.toByteArray();
45 | }
46 | }
47 |
48 | @Override
49 | public BigQueryCommittable deserialize(int version, byte[] serialized) throws IOException {
50 | try (final ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
51 | final DataInputStream in = new DataInputStream(bais)) {
52 | final Long producerId = in.readLong();
53 | final String streamName = in.readUTF();
54 | final long streamOffset = in.readLong();
55 | BigQueryCommittable committable =
56 | new BigQueryCommittable(producerId, streamName, streamOffset);
57 | return committable;
58 | }
59 | }
60 | }
61 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/committer/BigQueryCommitter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2024 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink.committer;
18 |
19 | import org.apache.flink.api.connector.sink2.Committer;
20 |
21 | import com.google.api.gax.rpc.ApiException;
22 | import com.google.cloud.bigquery.storage.v1.FlushRowsResponse;
23 | import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions;
24 | import com.google.cloud.flink.bigquery.common.exceptions.BigQueryConnectorException;
25 | import com.google.cloud.flink.bigquery.services.BigQueryServices;
26 | import com.google.cloud.flink.bigquery.services.BigQueryServicesFactory;
27 | import org.slf4j.Logger;
28 | import org.slf4j.LoggerFactory;
29 |
30 | import java.io.Closeable;
31 | import java.io.IOException;
32 | import java.util.Collection;
33 |
34 | /**
35 | * Committer implementation for {@link BigQueryExactlyOnceSink}.
36 | *
37 | *
The committer is responsible for committing records buffered in BigQuery write stream to
38 | * BigQuery table.
39 | */
40 | public class BigQueryCommitter implements Committer, Closeable {
41 |
42 | private static final Logger LOG = LoggerFactory.getLogger(BigQueryCommitter.class);
43 |
44 | private final BigQueryConnectOptions connectOptions;
45 |
46 | public BigQueryCommitter(BigQueryConnectOptions connectOptions) {
47 | this.connectOptions = connectOptions;
48 | }
49 |
50 | @Override
51 | public void commit(Collection> commitRequests) {
52 | if (commitRequests.isEmpty()) {
53 | LOG.info("No committable found. Nothing to commit!");
54 | return;
55 | }
56 | try (BigQueryServices.StorageWriteClient writeClient =
57 | BigQueryServicesFactory.instance(connectOptions).storageWrite()) {
58 | for (CommitRequest commitRequest : commitRequests) {
59 | BigQueryCommittable committable = commitRequest.getCommittable();
60 | long producerId = committable.getProducerId();
61 | String streamName = committable.getStreamName();
62 | long streamOffset = committable.getStreamOffset();
63 | LOG.info("Committing records appended by producer {}", producerId);
64 | LOG.debug(
65 | "Invoking flushRows API on stream {} till offset {}",
66 | streamName,
67 | streamOffset);
68 | FlushRowsResponse response = writeClient.flushRows(streamName, streamOffset);
69 | if (response.getOffset() != streamOffset) {
70 | LOG.error(
71 | "BigQuery FlushRows API failed. Returned offset {}, expected {}",
72 | response.getOffset(),
73 | streamOffset);
74 | throw new BigQueryConnectorException(
75 | String.format("Commit operation failed for producer %d", producerId));
76 | }
77 | }
78 | } catch (IOException | ApiException e) {
79 | throw new BigQueryConnectorException("Commit operation failed", e);
80 | }
81 | }
82 |
83 | @Override
84 | public void close() {
85 | // No op.
86 | }
87 | }
88 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/exceptions/BigQuerySerializationException.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2024 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink.exceptions;
18 |
19 | /** This class wraps errors found during serialization of Flink records to BigQuery protos. */
20 | public class BigQuerySerializationException extends Exception {
21 |
22 | public BigQuerySerializationException(String message) {
23 | super(message);
24 | }
25 |
26 | public BigQuerySerializationException(String message, Throwable error) {
27 | super(message, error);
28 | }
29 | }
30 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/serializer/BigQueryProtoSerializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2024 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink.serializer;
18 |
19 | import com.google.cloud.flink.bigquery.sink.exceptions.BigQuerySerializationException;
20 | import com.google.protobuf.ByteString;
21 | import org.apache.avro.Schema;
22 |
23 | import java.io.Serializable;
24 |
25 | /**
26 | * Base class for defining a Flink record to BigQuery proto serializer.
27 | *
28 | *
One BigQueryProtoSerializer should correspond to a single BigQuery table.
29 | *
30 | * @param Type of records to be written to BigQuery.
31 | */
32 | public abstract class BigQueryProtoSerializer implements Serializable {
33 |
34 | /**
35 | * Convert Flink record to proto ByteString compatible with BigQuery table.
36 | *
37 | * @param record Record to serialize.
38 | * @return ByteString.
39 | * @throws BigQuerySerializationException If serialization failed.
40 | */
41 | public abstract ByteString serialize(IN record) throws BigQuerySerializationException;
42 |
43 | /**
44 | * Initializes the serializer with a BigQuery table schema. This will be called once for every
45 | * serializer instance before its first serialize call.
46 | *
47 | * @param schemaProvider BigQuery table's schema information.
48 | */
49 | public void init(BigQuerySchemaProvider schemaProvider) {}
50 |
51 | /**
52 | * Derives Avro {@link Schema} describing the data record. This is primarily used by the sink to
53 | * infer schema for creating new destination BigQuery table if one doesn't already exist.
54 | *
55 | * @param record Record to check for schema
56 | * @return Schema.
57 | */
58 | public abstract Schema getAvroSchema(IN record);
59 | }
60 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/serializer/BigQuerySchemaProvider.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2024 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink.serializer;
18 |
19 | import com.google.protobuf.DescriptorProtos.DescriptorProto;
20 | import com.google.protobuf.Descriptors.Descriptor;
21 | import org.apache.avro.Schema;
22 |
23 | import java.io.Serializable;
24 |
25 | /**
26 | * Interface to derive {@link Descriptor} for Generic Record serialization, along with access to
27 | * Avro {@link Schema} and {@link DescriptorProto}.
28 | */
29 | public interface BigQuerySchemaProvider extends Serializable {
30 |
31 | /**
32 | * Returns a {@link DescriptorProto} object essential for obtaining Proto Rows Builder and
33 | * Descriptor instances.
34 | *
35 | * @return DescriptorProto
36 | */
37 | DescriptorProto getDescriptorProto();
38 |
39 | /**
40 | * Returns a {@link Descriptor} object essential for obtaining Dynamic Message instances.
41 | *
42 | * @return Descriptor
43 | */
44 | Descriptor getDescriptor();
45 |
46 | /**
47 | * Returns a {@link Schema} object required for obtaining Descriptor and DescriptorProto
48 | * instances.
49 | *
50 | * @return AvroSchema
51 | */
52 | Schema getAvroSchema();
53 |
54 | /**
55 | * Returns true if BigQuery table's schema is unknown, else false. Schema can be unknown due to
56 | * reasons like table does not exist before Flink job.
57 | *
58 | * @return boolean
59 | */
60 | boolean schemaUnknown();
61 | }
62 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/state/BigQueryStreamState.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2024 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink.state;
18 |
19 | /** State representation of a BigQuery write stream. */
20 | public abstract class BigQueryStreamState {
21 |
22 | protected final String streamName;
23 | protected final long streamOffset;
24 |
25 | public BigQueryStreamState(String streamName, long streamOffset) {
26 | this.streamName = streamName;
27 | this.streamOffset = streamOffset;
28 | }
29 |
30 | public String getStreamName() {
31 | return streamName;
32 | }
33 |
34 | public long getStreamOffset() {
35 | return streamOffset;
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/throttle/BigQueryWriterThrottler.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2025 The Apache Software Foundation.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink.throttle;
18 |
19 | import org.slf4j.Logger;
20 | import org.slf4j.LoggerFactory;
21 |
22 | import java.util.concurrent.TimeUnit;
23 |
24 | /**
25 | * Throttler implementation for BigQuery sink writers.
26 | *
27 | *
BigQuery APIs used by this sink's writers are subject BigQuery imposed quotas and limits.
28 | *
29 | *
The createTable API is used if destination BigQuery table does not already exist, and this API
30 | * allows 10 QPS.
31 | *
32 | *
The {@link BigQueryBufferedWriter} invokes BigQuery's CreateWriteStream API before its initial
33 | * write to a BigQuery table. This API expects a low QPS (~3) for best performance in steady state,
34 | * since write stream creation is an expensive operation.
35 | *
36 | *
This throttler allocates writers into buckets which correspond to a specific "wait" duration
37 | * before invoking above BigQuery APIs. Given the distributed nature of Flink deployments, we aim to
38 | * achieve 3 QPS on a best effort basis.
39 | */
40 | public class BigQueryWriterThrottler implements Throttler {
41 |
42 | private static final Logger LOG = LoggerFactory.getLogger(BigQueryWriterThrottler.class);
43 | private final int writerId;
44 | private final int maxBuckets;
45 |
46 | public BigQueryWriterThrottler(int writerId, int maxParallelism) {
47 | this.writerId = writerId;
48 | this.maxBuckets = maxParallelism / 3;
49 | }
50 |
51 | @Override
52 | public void throttle() {
53 | int waitSeconds = writerId % maxBuckets;
54 | LOG.debug("Throttling writer {} for {} second", writerId, waitSeconds);
55 | try {
56 | // Sleep does nothing if input is 0 or less.
57 | TimeUnit.SECONDS.sleep(waitSeconds);
58 | } catch (InterruptedException e) {
59 | LOG.warn("Throttle attempt interrupted in subtask {}", writerId);
60 | Thread.currentThread().interrupt();
61 | }
62 | }
63 | }
64 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/throttle/Throttler.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 The Apache Software Foundation.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink.throttle;
18 |
19 | /** Limits the rate at which an operation can be performed. */
20 | public interface Throttler {
21 |
22 | /** Limits the rate by waiting if necessary. */
23 | void throttle();
24 | }
25 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/writer/BigQueryWriterState.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2024 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink.writer;
18 |
19 | import com.google.cloud.flink.bigquery.sink.state.BigQueryStreamState;
20 |
21 | /** State representation of a {@link BigQueryBufferedWriter}. */
22 | public class BigQueryWriterState extends BigQueryStreamState {
23 |
24 | // Used for Flink metrics.
25 | private final long totalRecordsSeen;
26 | private final long totalRecordsWritten;
27 | private final long totalRecordsCommitted;
28 | private final long checkpointId;
29 |
30 | public BigQueryWriterState(
31 | String streamName,
32 | long streamOffset,
33 | long totalRecordsSeen,
34 | long totalRecordsWritten,
35 | long totalRecordsCommitted,
36 | long checkpointId) {
37 | super(streamName, streamOffset);
38 | this.totalRecordsSeen = totalRecordsSeen;
39 | this.totalRecordsWritten = totalRecordsWritten;
40 | this.totalRecordsCommitted = totalRecordsCommitted;
41 | this.checkpointId = checkpointId;
42 | }
43 |
44 | public long getTotalRecordsSeen() {
45 | return totalRecordsSeen;
46 | }
47 |
48 | public long getTotalRecordsWritten() {
49 | return totalRecordsWritten;
50 | }
51 |
52 | public long getTotalRecordsCommitted() {
53 | return totalRecordsCommitted;
54 | }
55 |
56 | public long getCheckpointId() {
57 | return checkpointId;
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/writer/BigQueryWriterStateSerializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2023 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink.writer;
18 |
19 | import org.apache.flink.core.io.SimpleVersionedSerializer;
20 |
21 | import java.io.ByteArrayInputStream;
22 | import java.io.ByteArrayOutputStream;
23 | import java.io.DataInputStream;
24 | import java.io.DataOutputStream;
25 | import java.io.IOException;
26 |
27 | /** Serializer and deserializer for {@link BigQueryWriterState}. */
28 | public class BigQueryWriterStateSerializer
29 | implements SimpleVersionedSerializer {
30 |
31 | @Override
32 | public int getVersion() {
33 | return 1;
34 | }
35 |
36 | @Override
37 | public byte[] serialize(BigQueryWriterState state) throws IOException {
38 | try (final ByteArrayOutputStream baos = new ByteArrayOutputStream();
39 | final DataOutputStream out = new DataOutputStream(baos)) {
40 | out.writeUTF(state.getStreamName());
41 | out.writeLong(state.getStreamOffset());
42 | out.writeLong(state.getTotalRecordsSeen());
43 | out.writeLong(state.getTotalRecordsWritten());
44 | out.writeLong(state.getTotalRecordsCommitted());
45 | out.writeLong(state.getCheckpointId());
46 | out.flush();
47 | return baos.toByteArray();
48 | }
49 | }
50 |
51 | @Override
52 | public BigQueryWriterState deserialize(int version, byte[] serialized) throws IOException {
53 | try (final ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
54 | final DataInputStream in = new DataInputStream(bais)) {
55 | final String streamName = in.readUTF();
56 | final long streamOffset = in.readLong();
57 | final long totalRecordsSeen = in.readLong();
58 | final long totalRecordsWritten = in.readLong();
59 | final long totalRecordsCommitted = in.readLong();
60 | final long checkpointId = in.readLong();
61 | return new BigQueryWriterState(
62 | streamName,
63 | streamOffset,
64 | totalRecordsSeen,
65 | totalRecordsWritten,
66 | totalRecordsCommitted,
67 | checkpointId);
68 | }
69 | }
70 | }
71 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/sink/writer/CreateTableOptions.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 The Apache Software Foundation.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink.writer;
18 |
19 | import com.google.cloud.bigquery.TimePartitioning;
20 |
21 | import java.util.List;
22 |
23 | /** Options for creating new BigQuery table. */
24 | public class CreateTableOptions {
25 |
26 | private final boolean enableTableCreation;
27 | private final String partitionField;
28 | private final TimePartitioning.Type partitionType;
29 | private final long partitionExpirationMillis;
30 | private final List clusteredFields;
31 | private final String region;
32 |
33 | public CreateTableOptions(
34 | boolean enableTableCreation,
35 | String partitionField,
36 | TimePartitioning.Type partitionType,
37 | Long partitionExpirationMillis,
38 | List clusteredFields,
39 | String region) {
40 | this.enableTableCreation = enableTableCreation;
41 | this.partitionField = partitionField;
42 | this.partitionType = partitionType;
43 | if (partitionExpirationMillis == null) {
44 | this.partitionExpirationMillis = 0;
45 | } else {
46 | this.partitionExpirationMillis = partitionExpirationMillis;
47 | }
48 | this.clusteredFields = clusteredFields;
49 | this.region = region;
50 | }
51 |
52 | public boolean enableTableCreation() {
53 | return enableTableCreation;
54 | }
55 |
56 | public String getPartitionField() {
57 | return partitionField;
58 | }
59 |
60 | public TimePartitioning.Type getPartitionType() {
61 | return partitionType;
62 | }
63 |
64 | public long getPartitionExpirationMillis() {
65 | return partitionExpirationMillis;
66 | }
67 |
68 | public List getClusteredFields() {
69 | return clusteredFields;
70 | }
71 |
72 | public String getRegion() {
73 | return region;
74 | }
75 | }
76 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/emitter/BigQueryRecordEmitter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2023 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.source.emitter;
18 |
19 | import org.apache.flink.annotation.Internal;
20 | import org.apache.flink.api.connector.source.SourceOutput;
21 | import org.apache.flink.connector.base.source.reader.RecordEmitter;
22 | import org.apache.flink.util.Collector;
23 |
24 | import com.google.cloud.flink.bigquery.source.reader.BigQuerySourceReader;
25 | import com.google.cloud.flink.bigquery.source.reader.deserializer.BigQueryDeserializationSchema;
26 | import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplitState;
27 | import org.apache.avro.generic.GenericRecord;
28 |
29 | /**
30 | * The {@link RecordEmitter} implementation for {@link BigQuerySourceReader} .We would always update
31 | * the last consumed message id in this emitter.
32 | *
33 | * @param the emitted type.
34 | */
35 | @Internal
36 | public class BigQueryRecordEmitter
37 | implements RecordEmitter {
38 |
39 | private final BigQueryDeserializationSchema deserializationSchema;
40 | private final SourceOutputWrapper sourceOutputWrapper;
41 |
42 | public BigQueryRecordEmitter(
43 | BigQueryDeserializationSchema deserializationSchema) {
44 | this.deserializationSchema = deserializationSchema;
45 | this.sourceOutputWrapper = new SourceOutputWrapper<>();
46 | }
47 |
48 | @Override
49 | public void emitRecord(
50 | GenericRecord record, SourceOutput output, BigQuerySourceSplitState splitState)
51 | throws Exception {
52 | // Update current offset.
53 | splitState.updateOffset();
54 | // Sink the record to source output.
55 | sourceOutputWrapper.setSourceOutput(output);
56 | deserializationSchema.deserialize(record, sourceOutputWrapper);
57 | }
58 |
59 | private static class SourceOutputWrapper implements Collector {
60 | private SourceOutput sourceOutput;
61 |
62 | @Override
63 | public void collect(T record) {
64 | sourceOutput.collect(record);
65 | }
66 |
67 | @Override
68 | public void close() {}
69 |
70 | private void setSourceOutput(SourceOutput sourceOutput) {
71 | this.sourceOutput = sourceOutput;
72 | }
73 | }
74 | }
75 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumState.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2023 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.source.enumerator;
18 |
19 | import org.apache.flink.annotation.PublicEvolving;
20 |
21 | import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplit;
22 |
23 | import java.util.ArrayList;
24 | import java.util.HashMap;
25 | import java.util.List;
26 | import java.util.Map;
27 | import java.util.Objects;
28 |
29 | /** The state representation for the BigQuery source enumerator. */
30 | @PublicEvolving
31 | public class BigQuerySourceEnumState {
32 |
33 | private final List lastSeenPartitions;
34 | private final List remaniningTableStreams;
35 | private final List completedTableStreams;
36 | private final List remainingSourceSplits;
37 | private final Map assignedSourceSplits;
38 | private final Boolean initialized;
39 |
40 | public BigQuerySourceEnumState(
41 | List lastSeenPartitions,
42 | List remaniningTableStreams,
43 | List completedTableStreams,
44 | List remainingSourceSplits,
45 | Map assignedSourceSplits,
46 | Boolean initialized) {
47 | this.lastSeenPartitions = new ArrayList<>(lastSeenPartitions);
48 | this.remaniningTableStreams = new ArrayList<>(remaniningTableStreams);
49 | this.completedTableStreams = new ArrayList<>(completedTableStreams);
50 | this.remainingSourceSplits = new ArrayList<>(remainingSourceSplits);
51 | this.assignedSourceSplits = new HashMap<>(assignedSourceSplits);
52 | this.initialized = initialized;
53 | }
54 |
55 | public List getLastSeenPartitions() {
56 | return this.lastSeenPartitions;
57 | }
58 |
59 | public List getRemaniningTableStreams() {
60 | return remaniningTableStreams;
61 | }
62 |
63 | public List getCompletedTableStreams() {
64 | return completedTableStreams;
65 | }
66 |
67 | public List getRemainingSourceSplits() {
68 | return remainingSourceSplits;
69 | }
70 |
71 | public Map getAssignedSourceSplits() {
72 | return assignedSourceSplits;
73 | }
74 |
75 | public Boolean isInitialized() {
76 | return initialized;
77 | }
78 |
79 | public static BigQuerySourceEnumState initialState() {
80 | return new BigQuerySourceEnumState(
81 | new ArrayList<>(),
82 | new ArrayList<>(),
83 | new ArrayList<>(),
84 | new ArrayList<>(),
85 | new HashMap<>(),
86 | false);
87 | }
88 |
89 | @Override
90 | public int hashCode() {
91 | return Objects.hash(
92 | this.lastSeenPartitions,
93 | this.remaniningTableStreams,
94 | this.completedTableStreams,
95 | this.remainingSourceSplits,
96 | this.assignedSourceSplits,
97 | this.initialized);
98 | }
99 |
100 | @Override
101 | public boolean equals(Object obj) {
102 | if (this == obj) {
103 | return true;
104 | }
105 | if (obj == null) {
106 | return false;
107 | }
108 | if (getClass() != obj.getClass()) {
109 | return false;
110 | }
111 | final BigQuerySourceEnumState other = (BigQuerySourceEnumState) obj;
112 | return Objects.equals(this.lastSeenPartitions, other.lastSeenPartitions)
113 | && Objects.equals(this.remaniningTableStreams, other.remaniningTableStreams)
114 | && Objects.equals(this.completedTableStreams, other.completedTableStreams)
115 | && Objects.equals(this.remainingSourceSplits, other.remainingSourceSplits)
116 | && Objects.equals(this.assignedSourceSplits, other.assignedSourceSplits)
117 | && Objects.equals(this.initialized, other.initialized);
118 | }
119 |
120 | @Override
121 | public String toString() {
122 | return String.format(
123 | "BigQuerySourceEnumState{"
124 | + "lastSeenPartitions=%s"
125 | + ", remaniningTableStreams=%s"
126 | + ", completedTableStreams=%s"
127 | + ", remainingSourceSplits=%s"
128 | + ", assignedSourceSplits=%s"
129 | + ", initialized=%s}",
130 | lastSeenPartitions,
131 | remaniningTableStreams,
132 | completedTableStreams,
133 | remainingSourceSplits,
134 | assignedSourceSplits,
135 | initialized);
136 | }
137 | }
138 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/BigQuerySourceReader.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2023 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.source.reader;
18 |
19 | import org.apache.flink.annotation.Internal;
20 | import org.apache.flink.api.connector.source.SourceReaderContext;
21 | import org.apache.flink.configuration.Configuration;
22 | import org.apache.flink.connector.base.source.reader.RecordEmitter;
23 | import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
24 | import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase;
25 | import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
26 | import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
27 |
28 | import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplit;
29 | import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplitState;
30 | import org.apache.avro.generic.GenericRecord;
31 | import org.slf4j.Logger;
32 | import org.slf4j.LoggerFactory;
33 |
34 | import java.util.Map;
35 | import java.util.function.Supplier;
36 |
37 | /**
38 | * The common BigQuery source reader for both ordered & unordered message consuming.
39 | *
40 | * @param The output message type for Flink.
41 | */
42 | @Internal
43 | public class BigQuerySourceReader
44 | extends SingleThreadMultiplexSourceReaderBase<
45 | GenericRecord, OUT, BigQuerySourceSplit, BigQuerySourceSplitState> {
46 | private static final Logger LOG = LoggerFactory.getLogger(BigQuerySourceReader.class);
47 |
48 | public BigQuerySourceReader(
49 | FutureCompletingBlockingQueue> elementsQueue,
50 | Supplier> splitReaderSupplier,
51 | RecordEmitter recordEmitter,
52 | Configuration config,
53 | SourceReaderContext context) {
54 | super(elementsQueue, splitReaderSupplier, recordEmitter, config, context);
55 | }
56 |
57 | public BigQuerySourceReader(
58 | FutureCompletingBlockingQueue> elementsQueue,
59 | Supplier> splitReaderSupplier,
60 | RecordEmitter recordEmitter,
61 | SourceReaderContext context) {
62 | super(elementsQueue, splitReaderSupplier, recordEmitter, new Configuration(), context);
63 | }
64 |
65 | @Override
66 | public void start() {
67 | if (getNumberOfCurrentlyAssignedSplits() == 0) {
68 | context.sendSplitRequest();
69 | }
70 | }
71 |
72 | @Override
73 | protected void onSplitFinished(Map finishedSplitIds) {
74 | for (BigQuerySourceSplitState splitState : finishedSplitIds.values()) {
75 | BigQuerySourceSplit sourceSplit = splitState.toBigQuerySourceSplit();
76 | LOG.info("Read for split {} is completed.", sourceSplit.splitId());
77 | }
78 | context.sendSplitRequest();
79 | }
80 |
81 | @Override
82 | protected BigQuerySourceSplitState initializedState(BigQuerySourceSplit split) {
83 | return new BigQuerySourceSplitState(split);
84 | }
85 |
86 | @Override
87 | protected BigQuerySourceSplit toSplitType(String string, BigQuerySourceSplitState sst) {
88 | return sst.toBigQuerySourceSplit();
89 | }
90 | }
91 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/BigQuerySourceReaderContext.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2023 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.source.reader;
18 |
19 | import org.apache.flink.annotation.Internal;
20 | import org.apache.flink.api.connector.source.SourceEvent;
21 | import org.apache.flink.api.connector.source.SourceReaderContext;
22 | import org.apache.flink.configuration.Configuration;
23 | import org.apache.flink.metrics.groups.SourceReaderMetricGroup;
24 | import org.apache.flink.util.UserCodeClassLoader;
25 |
26 | import java.util.concurrent.atomic.AtomicLong;
27 |
28 | /** A {@link SourceReaderContext} proxy that adds limit and counts for state management. */
29 | @Internal
30 | public class BigQuerySourceReaderContext implements SourceReaderContext {
31 |
32 | private final SourceReaderContext readerContext;
33 | private final AtomicLong readCount = new AtomicLong(0);
34 | private final int limit;
35 |
36 | public BigQuerySourceReaderContext(SourceReaderContext readerContext, int limit) {
37 | this.readerContext = readerContext;
38 | this.limit = limit;
39 | }
40 |
41 | @Override
42 | public SourceReaderMetricGroup metricGroup() {
43 | return readerContext.metricGroup();
44 | }
45 |
46 | @Override
47 | public Configuration getConfiguration() {
48 | return readerContext.getConfiguration();
49 | }
50 |
51 | @Override
52 | public String getLocalHostName() {
53 | return readerContext.getLocalHostName();
54 | }
55 |
56 | @Override
57 | public int getIndexOfSubtask() {
58 | return readerContext.getIndexOfSubtask();
59 | }
60 |
61 | @Override
62 | public void sendSplitRequest() {
63 | readerContext.sendSplitRequest();
64 | }
65 |
66 | @Override
67 | public void sendSourceEventToCoordinator(SourceEvent sourceEvent) {
68 | readerContext.sendSourceEventToCoordinator(sourceEvent);
69 | }
70 |
71 | @Override
72 | public UserCodeClassLoader getUserCodeClassLoader() {
73 | return readerContext.getUserCodeClassLoader();
74 | }
75 |
76 | public Long updateReadCount(Long newReads) {
77 | return readCount.addAndGet(newReads);
78 | }
79 |
80 | public Long currentReadCount() {
81 | return readCount.get();
82 | }
83 |
84 | public boolean isLimitPushedDown() {
85 | return limit > 0;
86 | }
87 |
88 | public boolean willExceedLimit(int newReads) {
89 | return limit > 0 && (readCount.get() + newReads) >= limit;
90 | }
91 | }
92 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/AvroDeserializationSchema.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2023 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.source.reader.deserializer;
18 |
19 | import org.apache.flink.annotation.Internal;
20 | import org.apache.flink.api.common.typeinfo.TypeInformation;
21 | import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo;
22 |
23 | import org.apache.avro.Schema;
24 | import org.apache.avro.generic.GenericRecord;
25 |
26 | /**
27 | * A simple Identity de-serialization for pipelines that just want {@link GenericRecord} as response
28 | * from BigQuery.
29 | */
30 | @Internal
31 | public class AvroDeserializationSchema
32 | implements BigQueryDeserializationSchema {
33 |
34 | private final String avroSchemaString;
35 |
36 | public AvroDeserializationSchema(String avroSchemaString) {
37 | this.avroSchemaString = avroSchemaString;
38 | }
39 |
40 | @Override
41 | public GenericRecord deserialize(GenericRecord record) {
42 | return record;
43 | }
44 |
45 | @Override
46 | public TypeInformation getProducedType() {
47 | return new GenericRecordAvroTypeInfo(new Schema.Parser().parse(avroSchemaString));
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/AvroToRowDataDeserializationSchema.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2023 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.source.reader.deserializer;
18 |
19 | import org.apache.flink.annotation.Internal;
20 | import org.apache.flink.api.common.typeinfo.TypeInformation;
21 | import org.apache.flink.table.data.GenericRowData;
22 | import org.apache.flink.table.data.RowData;
23 | import org.apache.flink.table.types.logical.RowType;
24 |
25 | import com.google.cloud.flink.bigquery.common.exceptions.BigQueryConnectorException;
26 | import org.apache.avro.generic.GenericRecord;
27 | import org.slf4j.Logger;
28 | import org.slf4j.LoggerFactory;
29 |
30 | /** Simple implementation for the Deserialization schema (from Avro GenericRecord to RowData). */
31 | @Internal
32 | public class AvroToRowDataDeserializationSchema
33 | implements BigQueryDeserializationSchema {
34 | private final AvroToRowDataConverters.AvroToRowDataConverter converter;
35 | private final TypeInformation typeInfo;
36 | private static final Logger LOG =
37 | LoggerFactory.getLogger(AvroToRowDataDeserializationSchema.class);
38 |
39 | public AvroToRowDataDeserializationSchema(RowType rowType, TypeInformation typeInfo) {
40 | this.converter = AvroToRowDataConverters.createRowConverter(rowType);
41 | this.typeInfo = typeInfo;
42 | }
43 |
44 | @Override
45 | public RowData deserialize(GenericRecord record) throws BigQueryConnectorException {
46 | try {
47 | return (GenericRowData) converter.convert(record);
48 | } catch (Exception e) {
49 | LOG.error(
50 | String.format(
51 | "Error in converting Avro Generic Record %s to Row Data.%nError: %s.%nCause:%s ",
52 | record.toString(), e.getMessage(), e.getCause()));
53 | throw new BigQueryConnectorException("Error in converting to Row Data", e);
54 | }
55 | }
56 |
57 | @Override
58 | public TypeInformation getProducedType() {
59 | return typeInfo;
60 | }
61 | }
62 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/BigQueryDeserializationSchema.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2023 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.source.reader.deserializer;
18 |
19 | import org.apache.flink.annotation.PublicEvolving;
20 | import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
21 | import org.apache.flink.util.Collector;
22 |
23 | import com.google.cloud.flink.bigquery.common.exceptions.BigQueryConnectorException;
24 | import org.slf4j.Logger;
25 | import org.slf4j.LoggerFactory;
26 |
27 | import java.io.Serializable;
28 |
29 | /**
30 | * A schema bridge for de-serializing the BigQuery's return types ({@code GenericRecord} or {@link
31 | * ArrowRecord}) into a flink managed instance.
32 | *
33 | * @param The input type to de-serialize.
34 | * @param The output record type for to sink for downstream processing.
35 | */
36 | @PublicEvolving
37 | public interface BigQueryDeserializationSchema
38 | extends Serializable, ResultTypeQueryable {
39 |
40 | Logger LOG = LoggerFactory.getLogger(BigQueryDeserializationSchema.class);
41 |
42 | /**
43 | * De-serializes the IN type record.
44 | *
45 | * @param record The BSON document to de-serialize.
46 | * @return The de-serialized message as an object (null if the message cannot be de-serialized).
47 | * @throws BigQueryConnectorException In case of problems while de-serializing.
48 | */
49 | OUT deserialize(IN record) throws BigQueryConnectorException;
50 |
51 | /**
52 | * De-serializes the IN type record.
53 | *
54 | *
Can output multiple records through the {@link Collector}. Note that number and size of
55 | * the produced records should be relatively small. Depending on the source implementation
56 | * records can be buffered in memory or collecting records might delay emitting checkpoint
57 | * barrier.
58 | *
59 | * @param record The IN document to de-serialize.
60 | * @param out The collector to put the resulting messages.
61 | */
62 | default void deserialize(IN record, Collector out) throws BigQueryConnectorException {
63 | OUT deserialize = deserialize(record);
64 | if (deserialize == null) {
65 | return;
66 | }
67 | try {
68 | out.collect(deserialize);
69 | } catch (Exception e) {
70 | LOG.error(
71 | String.format(
72 | "Failed to forward the deserialized record %s to the next operator.%nError %s%nCause %s",
73 | deserialize, e.getMessage(), e.getCause()));
74 | throw new BigQueryConnectorException(
75 | "Failed to forward the deserialized record to the next operator.", e);
76 | }
77 | }
78 | }
79 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplit.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2023 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.source.split;
18 |
19 | import org.apache.flink.annotation.PublicEvolving;
20 | import org.apache.flink.api.connector.source.SourceSplit;
21 |
22 | import java.io.Serializable;
23 | import java.util.Objects;
24 |
25 | /** A {@link SourceSplit} implementation for a BigQuery Read API stream. */
26 | @PublicEvolving
27 | public class BigQuerySourceSplit implements SourceSplit, Serializable {
28 |
29 | private final String streamName;
30 | private final Long offset;
31 |
32 | public BigQuerySourceSplit(String streamName) {
33 | this.streamName = streamName;
34 | this.offset = 0L;
35 | }
36 |
37 | public BigQuerySourceSplit(String streamName, Long offset) {
38 | this.streamName = streamName;
39 | this.offset = offset;
40 | }
41 |
42 | @Override
43 | public String splitId() {
44 | return streamName;
45 | }
46 |
47 | public String getStreamName() {
48 | return streamName;
49 | }
50 |
51 | public Long getOffset() {
52 | return offset;
53 | }
54 |
55 | @Override
56 | public int hashCode() {
57 | return Objects.hash(this.streamName, this.offset);
58 | }
59 |
60 | @Override
61 | public boolean equals(Object obj) {
62 | if (this == obj) {
63 | return true;
64 | }
65 | if (obj == null) {
66 | return false;
67 | }
68 | if (getClass() != obj.getClass()) {
69 | return false;
70 | }
71 | final BigQuerySourceSplit other = (BigQuerySourceSplit) obj;
72 | if (!Objects.equals(this.streamName, other.streamName)) {
73 | return false;
74 | }
75 | return Objects.equals(this.offset, other.offset);
76 | }
77 |
78 | @Override
79 | public String toString() {
80 | return "BigQuerySourceSplit{" + "streamName=" + streamName + ", offset=" + offset + '}';
81 | }
82 | }
83 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2023 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.source.split;
18 |
19 | import org.apache.flink.annotation.Internal;
20 | import org.apache.flink.core.io.SimpleVersionedSerializer;
21 |
22 | import java.io.ByteArrayInputStream;
23 | import java.io.ByteArrayOutputStream;
24 | import java.io.DataInputStream;
25 | import java.io.DataOutputStream;
26 | import java.io.IOException;
27 |
28 | /** The {@link SimpleVersionedSerializer serializer} for {@link BigQuerySourceSplit}. */
29 | @Internal
30 | public class BigQuerySourceSplitSerializer
31 | implements SimpleVersionedSerializer {
32 |
33 | public static final BigQuerySourceSplitSerializer INSTANCE =
34 | new BigQuerySourceSplitSerializer();
35 | // This version should be bumped after modifying the source split or the enum states.
36 | public static final int VERSION = 0;
37 |
38 | private BigQuerySourceSplitSerializer() {
39 | // singleton instance
40 | }
41 |
42 | @Override
43 | public int getVersion() {
44 | return VERSION;
45 | }
46 |
47 | @Override
48 | public byte[] serialize(BigQuerySourceSplit obj) throws IOException {
49 | // VERSION 0 serialization
50 | try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
51 | DataOutputStream out = new DataOutputStream(baos)) {
52 | serializeBigQuerySourceSplit(out, obj);
53 | out.flush();
54 | return baos.toByteArray();
55 | }
56 | }
57 |
58 | @Override
59 | public BigQuerySourceSplit deserialize(int version, byte[] serialized) throws IOException {
60 | if (getVersion() != version) {
61 | throw new IllegalArgumentException(
62 | String.format(
63 | "The provided serializer version (%d) is not expected (expected : %s).",
64 | version, VERSION));
65 | }
66 | // VERSION 0 deserialization
67 | try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
68 | DataInputStream in = new DataInputStream(bais)) {
69 | return deserializeBigQuerySourceSplit(version, in);
70 | }
71 | }
72 |
73 | public void serializeBigQuerySourceSplit(DataOutputStream out, BigQuerySourceSplit split)
74 | throws IOException {
75 | out.writeUTF(split.getStreamName());
76 | out.writeLong(split.getOffset());
77 | }
78 |
79 | public BigQuerySourceSplit deserializeBigQuerySourceSplit(int version, DataInputStream in)
80 | throws IOException {
81 | switch (version) {
82 | case VERSION:
83 | String streamName = in.readUTF();
84 | long offset = in.readLong();
85 | return new BigQuerySourceSplit(streamName, offset);
86 | default:
87 | throw new IOException("Unknown version: " + version);
88 | }
89 | }
90 | }
91 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitState.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2023 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.source.split;
18 |
19 | import org.apache.flink.annotation.Internal;
20 |
21 | import java.util.Objects;
22 |
23 | /** BigQuery source split state for {@link BigQuerySourceSplit}. */
24 | @Internal
25 | public class BigQuerySourceSplitState {
26 | private final BigQuerySourceSplit split;
27 | private Long offset;
28 |
29 | public BigQuerySourceSplitState(BigQuerySourceSplit split) {
30 | this.split = split;
31 | offset = split.getOffset();
32 | }
33 |
34 | public BigQuerySourceSplit toBigQuerySourceSplit() {
35 | return new BigQuerySourceSplit(split.getStreamName(), offset);
36 | }
37 |
38 | public void updateOffset() {
39 | offset++;
40 | }
41 |
42 | @Override
43 | public String toString() {
44 | return "BigQuerySourceSplitState{" + "split=" + split + ", offset=" + offset + '}';
45 | }
46 |
47 | @Override
48 | public int hashCode() {
49 | return Objects.hash(this.split, this.offset);
50 | }
51 |
52 | @Override
53 | public boolean equals(Object obj) {
54 | if (this == obj) {
55 | return true;
56 | }
57 | if (obj == null) {
58 | return false;
59 | }
60 | if (getClass() != obj.getClass()) {
61 | return false;
62 | }
63 | final BigQuerySourceSplitState other = (BigQuerySourceSplitState) obj;
64 | if (!Objects.equals(this.split, other.split)) {
65 | return false;
66 | }
67 | return Objects.equals(this.offset, other.offset);
68 | }
69 | }
70 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/assigner/BoundedSplitAssigner.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2023 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.source.split.assigner;
18 |
19 | import org.apache.flink.annotation.Internal;
20 | import org.apache.flink.util.Preconditions;
21 |
22 | import com.google.cloud.bigquery.storage.v1.DataFormat;
23 | import com.google.cloud.bigquery.storage.v1.ReadSession;
24 | import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions;
25 | import com.google.cloud.flink.bigquery.source.enumerator.BigQuerySourceEnumState;
26 | import com.google.cloud.flink.bigquery.source.split.SplitDiscoverer;
27 |
28 | /**
29 | * A bounded implementation for a split assigner based on the BigQuery {@link ReadSession} streams.
30 | */
31 | @Internal
32 | public class BoundedSplitAssigner extends BigQuerySourceSplitAssigner {
33 |
34 | BoundedSplitAssigner(BigQueryReadOptions readOptions, BigQuerySourceEnumState sourceEnumState) {
35 | super(readOptions, sourceEnumState);
36 | }
37 |
38 | @Override
39 | public void discoverSplits() {
40 |
41 | this.remainingTableStreams.addAll(
42 | SplitDiscoverer.discoverSplits(
43 | this.readOptions.getBigQueryConnectOptions(),
44 | DataFormat.AVRO,
45 | this.readOptions.getColumnNames(),
46 | this.readOptions.getRowRestriction(),
47 | this.readOptions.getSnapshotTimestampInMillis(),
48 | this.readOptions.getMaxStreamCount()));
49 | }
50 |
51 | @Override
52 | public boolean noMoreSplits() {
53 | Preconditions.checkState(
54 | initialized, "The noMoreSplits method was called but not initialized.");
55 | return remainingTableStreams.isEmpty() && remainingSourceSplits.isEmpty();
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | com.google.cloud.flink.bigquery.table.BigQueryDynamicTableFactory
17 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/main/resources/connector.properties:
--------------------------------------------------------------------------------
1 | connector=BigQuery
2 |
--------------------------------------------------------------------------------
/flink-1.17-connector-bigquery/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/sink/BigQuerySinkTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2024 Google Inc.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not
5 | * use this file except in compliance with the License. You may obtain a copy of
6 | * the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations under
14 | * the License.
15 | */
16 |
17 | package com.google.cloud.flink.bigquery.sink;
18 |
19 | import org.apache.flink.api.common.restartstrategy.RestartStrategies;
20 | import org.apache.flink.api.common.restartstrategy.RestartStrategies.RestartStrategyConfiguration;
21 | import org.apache.flink.api.common.time.Time;
22 | import org.apache.flink.connector.base.DeliveryGuarantee;
23 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
24 |
25 | import com.google.cloud.flink.bigquery.fakes.StorageClientFaker;
26 | import com.google.cloud.flink.bigquery.sink.serializer.FakeBigQuerySerializer;
27 | import com.google.cloud.flink.bigquery.sink.serializer.TestBigQuerySchemas;
28 | import com.google.protobuf.ByteString;
29 | import org.junit.After;
30 | import org.junit.Before;
31 | import org.junit.Test;
32 |
33 | import static org.junit.Assert.assertTrue;
34 |
35 | /** Tests for {@link BigQuerySink}. */
36 | public class BigQuerySinkTest {
37 |
38 | private StreamExecutionEnvironment env;
39 |
40 | private static final RestartStrategyConfiguration NO_RESTART_STRATEGY =
41 | RestartStrategies.noRestart();
42 | private static final RestartStrategyConfiguration INVALID_FIXED_DELAY_RESTART_STRATEGY =
43 | RestartStrategies.fixedDelayRestart(20, Time.seconds(5));
44 |
45 | @Before
46 | public void setUp() {
47 | env = new StreamExecutionEnvironment();
48 | }
49 |
50 | @After
51 | public void tearDown() throws Exception {
52 | env.close();
53 | }
54 |
55 | @Test
56 | public void testGet_withAtLeastOnce() {
57 | env.setRestartStrategy(NO_RESTART_STRATEGY);
58 | BigQuerySinkConfig