From eb9dc8fa0d13cf8d5e1249b87733f222157118bd Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 27 Jun 2023 13:31:05 -0700 Subject: [PATCH 01/34] cleaning up the main branch for new version. --- pom.xml | 687 --------------- .../bigquery/ArrowDeserializationSchema.java | 143 --- .../flink/bigquery/ArrowFormatFactory.java | 81 -- .../ArrowRowDataDeserializationSchema.java | 163 ---- .../flink/bigquery/BigQueryArrowFormat.java | 52 -- .../bigquery/BigQueryDynamicTableFactory.java | 258 ------ .../bigquery/BigQueryDynamicTableSource.java | 82 -- .../flink/bigquery/BigQueryReadSession.java | 77 -- .../bigquery/BigQuerySourceFunction.java | 141 --- .../flink/bigquery/FlinkBigQueryConfig.java | 812 ------------------ .../bigquery/FlinkBigQueryException.java | 27 - .../FlinkBigQueryProxyAndHttpConfig.java | 241 ------ .../arrow/util/ArrowSchemaConverter.java | 70 -- .../arrow/util/ArrowToRowDataConverters.java | 328 ------- .../util/LogicalTypeToArrowTypeConverter.java | 168 ---- ...QueryDirectWriterCommitMessageContext.java | 49 -- ...inkBigQueryConnectorUserAgentProvider.java | 101 --- .../bigquery/common/FlinkBigQueryUtil.java | 39 - .../common/UserAgentHeaderProvider.java | 54 -- .../common/WriterCommitMessageContext.java | 20 - .../org.apache.flink.table.factories.Factory | 2 - .../flink-bigquery-connector.properties | 1 - ...yDirectWriterCommitMessageContextTest.java | 61 -- .../FlinkBigQueryProxyAndHttpConfigTest.java | 341 -------- .../bigquery/UserAgentHeaderProviderTest.java | 35 - .../flink/bigquery/integration/Constants.java | 50 -- .../FlinkBigQueryIntegrationTestBase.java | 66 -- .../FlinkReadByFormatIntegrationTest.java | 151 ---- .../FlinkReadFromQueryIntegrationTest.java | 132 --- .../integration/FlinkReadIntegrationTest.java | 310 ------- .../integration/IntegrationTestUtils.java | 158 ---- 31 files changed, 4900 deletions(-) delete mode 100644 pom.xml delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/ArrowDeserializationSchema.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/ArrowFormatFactory.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/ArrowRowDataDeserializationSchema.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/BigQueryArrowFormat.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/BigQueryDynamicTableFactory.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/BigQueryDynamicTableSource.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/BigQueryReadSession.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/BigQuerySourceFunction.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/FlinkBigQueryConfig.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/FlinkBigQueryException.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/FlinkBigQueryProxyAndHttpConfig.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/arrow/util/ArrowSchemaConverter.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/arrow/util/ArrowToRowDataConverters.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/arrow/util/LogicalTypeToArrowTypeConverter.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/common/BigQueryDirectWriterCommitMessageContext.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/common/FlinkBigQueryConnectorUserAgentProvider.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/common/FlinkBigQueryUtil.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/common/UserAgentHeaderProvider.java delete mode 100644 src/main/java/com/google/cloud/flink/bigquery/common/WriterCommitMessageContext.java delete mode 100644 src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory delete mode 100644 src/main/resources/flink-bigquery-connector.properties delete mode 100644 src/test/java/com/google/cloud/flink/bigquery/BigQueryDirectWriterCommitMessageContextTest.java delete mode 100644 src/test/java/com/google/cloud/flink/bigquery/FlinkBigQueryProxyAndHttpConfigTest.java delete mode 100644 src/test/java/com/google/cloud/flink/bigquery/UserAgentHeaderProviderTest.java delete mode 100644 src/test/java/com/google/cloud/flink/bigquery/integration/Constants.java delete mode 100644 src/test/java/com/google/cloud/flink/bigquery/integration/FlinkBigQueryIntegrationTestBase.java delete mode 100644 src/test/java/com/google/cloud/flink/bigquery/integration/FlinkReadByFormatIntegrationTest.java delete mode 100644 src/test/java/com/google/cloud/flink/bigquery/integration/FlinkReadFromQueryIntegrationTest.java delete mode 100644 src/test/java/com/google/cloud/flink/bigquery/integration/FlinkReadIntegrationTest.java delete mode 100644 src/test/java/com/google/cloud/flink/bigquery/integration/IntegrationTestUtils.java diff --git a/pom.xml b/pom.xml deleted file mode 100644 index 33f1d973..00000000 --- a/pom.xml +++ /dev/null @@ -1,687 +0,0 @@ - - 4.0.0 - com.google.cloud.flink - flink-bigquery-connector - ${revision} - Flink BigQuery Connector - Flink BigQuery Connector - https://github.com/GoogleCloudDataproc/flink-bigquery-connector - - - - Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt - repo - - - - - - Google Inc. - http://www.google.com - - - - - - scm:git:git@github.com:GoogleCloudDataproc/flink-bigquery-connector.git - - - scm:git:git@github.com:GoogleCloudDataproc/flink-bigquery-connector.git - - git@github.com:GoogleCloudDataproc/flink-bigquery-connector.git - - - - - ossrh - https://oss.sonatype.org/content/repositories/snapshots - - - ossrh - https://oss.sonatype.org/service/local/staging/deploy/maven2/ - - - - - 0.0.1-SNAPSHOT - true - - 1.8.2 - 6.0.1 - 0.23.2 - 1.11.0 - 1.12.0 - 2.3.1 - 2.6.1 - 1.1.3 - 1.45.1 - 31.1-jre - 3.3.0 - 2.13.2 - 2.13.4.1 - 3.1 - 4.13.1 - 4.1.75.Final - 3.20.0 - 2.12 - 1.7.15 - 8 - 8 - - - - - - - com.fasterxml.jackson - jackson-bom - ${jackson.version} - import - pom - - - com.fasterxml.jackson.core - jackson-core - ${jackson.version} - compile - - - com.fasterxml.jackson.core - jackson-databind - - ${jackson-databind.version} - compile - - - com.google.cloud.spark - bigquery-connector-common - ${bigquery-connector-common.version} - - - com.google.protobuf - protobuf-bom - ${protobuf.version} - pom - import - - - io.grpc - grpc-bom - ${grpc.version} - pom - import - - - io.netty - netty-bom - ${netty.version} - pom - import - - - org.apache.arrow - arrow-memory - ${arrow.version} - pom - - - - - - - com.google.cloud.spark - bigquery-connector-common - - - org.apache.flink - flink-core - ${flink.version} - provided - - - org.apache.flink - flink-table-common - ${flink.version} - provided - - - io.grpc - grpc-core - - - org.slf4j - slf4j-api - ${slf4j.version} - provided - - - - com.google.truth - truth - ${google-truth.version} - test - - - org.apache.flink - flink-core - ${flink.version} - test - test-jar - - - org.apache.flink - flink-test-utils_2.12 - ${flink.version} - test - - - org.apache.flink - flink-connector-test-utils - ${flink.connector.test.version} - test - - - org.apache.flink - flink-table-api-java-bridge_${scala.version} - ${flink.version} - provided - - - org.apache.flink - flink-table-planner-blink_${scala.version} - ${flink.version} - provided - - - org.apache.flink - flink-python_${scala.version} - ${flink.version} - provided - - - org.apache.arrow - arrow-memory-netty - ${arrow.version} - runtime - - - org.apache.flink - flink-clients_${scala.version} - ${flink.version} - provided - - - org.apache.hadoop - hadoop-client - ${hadoop-client.version} - - - com.google.guava - guava - ${guava.version} - - - junit - junit - ${junit.version} - test - - - com.github.jsqlparser - jsqlparser - ${jsqlparser.version} - - - com.fasterxml.jackson.core - jackson-databind - - - org.apache.flink - flink-table-api-java-bridge_${scala.version} - ${flink.version} - provided - - - com.google.protobuf - protobuf-java - - - org.apache.commons - commons-lang3 - 3.12.0 - - - - - - - com.diffplug.spotless - spotless-maven-plugin - 2.21.0 - - - com.github.spotbugs - spotbugs-maven-plugin - 4.5.3.0 - - - org.apache.maven.plugins - maven-compiler-plugin - 3.10.0 - - - org.apache.maven.plugins - maven-enforcer-plugin - 3.3.0 - - - org.apache.maven.plugins - maven-failsafe-plugin - 3.0.0-M5 - - - org.apache.maven.plugins - maven-jar-plugin - 3.2.2 - - - org.apache.maven.plugins - maven-resources-plugin - 3.2.0 - - - org.apache.maven.plugins - maven-shade-plugin - 3.2.4 - - - org.apache.maven.plugins - maven-surefire-plugin - 3.0.0-M5 - - - org.codehaus.mojo - build-helper-maven-plugin - 3.0.0 - - - org.apache.maven.plugins - maven-assembly-plugin - 3.1.1 - - - - jar-with-dependencies - - - - - - make-assembly - package - - single - - - - - - org.jacoco - jacoco-maven-plugin - 0.8.7 - - - - prepare-agent - - - - - - - - - org.apache.maven.plugins - maven-compiler-plugin - - 1.8 - 1.8 - - - - org.apache.maven.plugins - maven-shade-plugin - - false - - - android - com.google.cloud.flink.bigquery.repackaged.android - - - avro.shaded - com.google.cloud.flink.bigquery.repackaged.avro.shaded - - - com.fasterxml - com.google.cloud.flink.bigquery.repackaged.com.fasterxml - - - - com.github - com.google.cloud.flink.bigquery.repackaged.com.github - - - com.google - com.google.cloud.flink.bigquery.repackaged.com.google - - com.google.cloud.bigquery.connector.common.** - com.google.cloud.flink.bigquery.** - - - - com.thoughtworks.paranamer - com.google.cloud.flink.bigquery.repackaged.com.thoughtworks.paranamer - - - - com.typesafe - com.google.cloud.flink.bigquery.repackaged.com.typesafe - - - io.grpc - com.google.cloud.flink.bigquery.repackaged.io.grpc - - - io.netty - com.google.cloud.flink.bigquery.repackaged.io.netty - - - io.opencensus - com.google.cloud.flink.bigquery.repackaged.io.opencensus - - - - io.perfmark - com.google.cloud.flink.bigquery.repackaged.io.perfmark - - - org.apache.arrow - com.google.cloud.flink.bigquery.repackaged.org.apache.arrow - - - - org.apache.beam - com.google.cloud.flink.bigquery.repackaged.org.apache.beam - - - - org.apache.commons - com.google.cloud.flink.bigquery.repackaged.org.apache.commons - - - - org.apache.http - com.google.cloud.flink.bigquery.repackaged.org.apache.http - - - - org.checkerframework - com.google.cloud.flink.bigquery.repackaged.org.checkerframework - - - - org.codehaus.mojo - com.google.cloud.flink.bigquery.repackaged.org.codehaus.mojo - - - - org.conscrypt - com.google.cloud.flink.bigquery.repackaged.org.conscrypt - - - - org.json - com.google.cloud.flink.bigquery.repackaged.org.json - - - org.threeten - com.google.cloud.flink.bigquery.repackaged.org.threeten - - - org.tukaani.xz - com.google.cloud.flink.bigquery.repackaged.org.tukaani.xz - - - - org.xerial.snappy - com.google.cloud.flink.bigquery.repackaged.org.xerial.snappy - - - - META-INF/native/libnetty - META-INF/native/libcom_google_cloud_spark_bigquery_repackaged_netty - - - - META-INF/native/netty - META-INF/native/com_google_cloud_spark_bigquery_repackaged_netty - - - - - - - - - *:* - - module-info.class - META-INF/*.MF - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - META-INF/maven/** - **/*.proto - - - - - - - package - - shade - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - **/*AcceptanceTest.java - **/*IntegrationTest.java - - 0 - - - - org.apache.maven.plugins - maven-jar-plugin - - - - true - - true - - - - - - - com.diffplug.spotless - spotless-maven-plugin - - - - - *.md - .gitignore - - - - - - - - fmt:off - fmt:on - - - 1.7 - - - - - - - check - - compile - - - - - com.github.spotbugs - spotbugs-maven-plugin - - true - - - - org.apache.maven.plugins - maven-assembly-plugin - 3.1.1 - - - - jar-with-dependencies - - - - - - make-assembly - package - - single - - - - - - - - - integration - - false - - - - - org.apache.maven.plugins - maven-failsafe-plugin - - 5 - false - - **/*IntegrationTest.java - - - - - integration-test - - integration-test - - - - verify - - verify - - - - - - - - - acceptance - - false - - - - - org.apache.maven.plugins - maven-failsafe-plugin - - 2 - false - - **/*AcceptanceTest.java - - - - - integration-test - - integration-test - - - - verify - - verify - - - - - - - - - coverage - - false - - - - - org.jacoco - jacoco-maven-plugin - - - - - - - - - com.github.spotbugs - spotbugs-maven-plugin - - - - diff --git a/src/main/java/com/google/cloud/flink/bigquery/ArrowDeserializationSchema.java b/src/main/java/com/google/cloud/flink/bigquery/ArrowDeserializationSchema.java deleted file mode 100644 index 833f0cf5..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/ArrowDeserializationSchema.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery; - -import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.Objects; -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.memory.RootAllocator; -import org.apache.arrow.vector.FieldVector; -import org.apache.arrow.vector.VectorLoader; -import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.arrow.vector.ipc.ReadChannel; -import org.apache.arrow.vector.ipc.message.ArrowRecordBatch; -import org.apache.arrow.vector.ipc.message.MessageSerializer; -import org.apache.arrow.vector.types.pojo.Field; -import org.apache.arrow.vector.types.pojo.Schema; -import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.data.RowData; -import org.apache.flink.util.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ArrowDeserializationSchema implements DeserializationSchema, Serializable { - - private static final long serialVersionUID = 1L; - final Logger logger = LoggerFactory.getLogger(ArrowDeserializationSchema.class); - - private BufferAllocator allocator; - private TypeInformation typeInfo; - private VectorSchemaRoot root; - private VectorLoader loader; - private List vectors = new ArrayList<>(); - - private Class recordClazz; - private String schemaJsonString; - - public ArrowDeserializationSchema( - Class recordClazz, String schemaJsonString, TypeInformation typeInfo) { - Preconditions.checkNotNull(recordClazz, "Arrow record class must not be null."); - this.typeInfo = typeInfo; - this.recordClazz = recordClazz; - this.schemaJsonString = schemaJsonString; - } - - public static ArrowDeserializationSchema forGeneric( - String schemaJsonString, TypeInformation typeInfo) { - return new ArrowDeserializationSchema<>(VectorSchemaRoot.class, schemaJsonString, typeInfo); - } - - @Override - public T deserialize(byte[] responseByteMessage) throws IOException { - ReadRowsResponse response = ReadRowsResponse.parseFrom(responseByteMessage); - byte[] arrowRecordBatchMessage = - response.getArrowRecordBatch().getSerializedRecordBatch().toByteArray(); - - if (arrowRecordBatchMessage == null) { - throw new FlinkBigQueryException("Deserializing message is empty"); - } - - initializeArrow(); - ArrowRecordBatch deserializedBatch = - MessageSerializer.deserializeRecordBatch( - new ReadChannel(new ByteArrayReadableSeekableByteChannel(arrowRecordBatchMessage)), - allocator); - loader.load(deserializedBatch); - deserializedBatch.close(); - return (T) root; - } - - private void initializeArrow() throws IOException { - Schema schema = getSchema(schemaJsonString); - - if (root != null) { - return; - } - if (allocator == null) { - this.allocator = new RootAllocator(Long.MAX_VALUE); - } - for (Field field : schema.getFields()) { - vectors.add(field.createVector(allocator)); - } - root = new VectorSchemaRoot(vectors); - this.loader = new VectorLoader(root); - } - - @Override - public boolean isEndOfStream(T nextElement) { - return nextElement == null; - } - - @Override - public TypeInformation getProducedType() { - return (TypeInformation) typeInfo; - } - - private Schema getSchema(String schemaJson) { - Schema schema = null; - try { - schema = Schema.fromJSON(schemaJson); - } catch (IOException e) { - logger.error("Error while converting to Schema from jsonString"); - throw new FlinkBigQueryException("Error while converting to Schema from jsonString", e); - } - return schema; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ArrowDeserializationSchema that = (ArrowDeserializationSchema) o; - return recordClazz.equals(that.recordClazz) - && Objects.equals(getSchema(this.schemaJsonString), getSchema(that.schemaJsonString)); - } - - @Override - public int hashCode() { - return Objects.hash(recordClazz, getSchema(this.schemaJsonString)); - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/ArrowFormatFactory.java b/src/main/java/com/google/cloud/flink/bigquery/ArrowFormatFactory.java deleted file mode 100644 index a274360d..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/ArrowFormatFactory.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery; - -import com.google.cloud.bigquery.storage.v1.ArrowSchema; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.DeserializationFormatFactory; -import org.apache.flink.table.factories.DynamicTableFactory.Context; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.factories.SerializationFormatFactory; - -public class ArrowFormatFactory - implements DeserializationFormatFactory, SerializationFormatFactory { - public static final String IDENTIFIER = "arrow"; - public String selectedFields = null; - public ArrowSchema arrowSchema; - - @Override - public DecodingFormat> createDecodingFormat( - Context context, ReadableConfig formatOptions) { - Map options = context.getCatalogTable().getOptions(); - FactoryUtil.validateFactoryOptions(this, formatOptions); - this.selectedFields = options.get("selectedFields"); - if (this.selectedFields.endsWith(",")) { - this.selectedFields = - selectedFields.substring(0, selectedFields.length() - 1).replace("selectedFields=", ""); - } - List selectedFieldList = new ArrayList(); - List arrowFieldList = Arrays.asList(options.get("arrowFields").split(",")); - if (selectedFields != null) { - selectedFieldList = Arrays.asList(selectedFields.split(",")); - } - return new BigQueryArrowFormat(selectedFieldList, arrowFieldList); - } - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public EncodingFormat> createEncodingFormat( - Context context, ReadableConfig formatOptions) { - return null; - } - - @Override - public Set> requiredOptions() { - return Collections.emptySet(); - } - - @Override - public Set> optionalOptions() { - return Collections.emptySet(); - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/ArrowRowDataDeserializationSchema.java b/src/main/java/com/google/cloud/flink/bigquery/ArrowRowDataDeserializationSchema.java deleted file mode 100644 index b88ca699..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/ArrowRowDataDeserializationSchema.java +++ /dev/null @@ -1,163 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery; - -import com.google.cloud.flink.bigquery.arrow.util.ArrowSchemaConverter; -import com.google.cloud.flink.bigquery.arrow.util.ArrowToRowDataConverters; -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.Objects; -import javax.annotation.Nullable; -import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.arrow.vector.types.pojo.Schema; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.util.Collector; - -public class ArrowRowDataDeserializationSchema - implements DeserializationSchema, Serializable { - - public static final long serialVersionUID = 1L; - public TypeInformation typeInfo; - public DeserializationSchema nestedSchema; - public ArrowToRowDataConverters.ArrowToRowDataConverter runtimeConverter; - public List rowDataList; - public List selectedFields = new ArrayList(); - public String selectedFieldString; - final List readSessionFieldNames = new ArrayList(); - public String arrowReadSessionSchema; - public String arrowSchemaJson; - - public ArrowRowDataDeserializationSchema( - RowType rowType, - TypeInformation typeInfo, - List selectedFieldList, - List arrowFieldList) { - this.typeInfo = typeInfo; - Schema arrowSchema = - ArrowSchemaConverter.convertToSchema( - getRowTypeForArrowSchema(rowType, selectedFieldList, arrowFieldList)); - arrowSchema.getFields().stream() - .forEach( - field -> { - this.readSessionFieldNames.add(field.getName()); - }); - this.arrowSchemaJson = arrowSchema.toJson().toString(); - this.nestedSchema = ArrowDeserializationSchema.forGeneric(arrowSchemaJson, typeInfo); - this.runtimeConverter = - ArrowToRowDataConverters.createRowConverter(rowType, readSessionFieldNames); - } - - private RowType getRowTypeForArrowSchema( - RowType rowType, List selectedFieldList, List arrowFieldList) { - List rowFieldNames = rowType.getFieldNames(); - List rowFields = rowType.getChildren(); - List updatedRowFields = new ArrayList(); - List updatedRowFieldNames = new ArrayList(); - for (int i = 0; i < rowFieldNames.size(); i++) { - if (selectedFieldList.get(i).equals(arrowFieldList.get(i))) { - updatedRowFieldNames.add(rowFieldNames.get(i)); - updatedRowFields.add(rowFields.get(i)); - } else { - String arrowFieldsName = arrowFieldList.get(i); - int rowTypeFieldIndex = selectedFieldList.indexOf(arrowFieldsName); - updatedRowFieldNames.add(rowFieldNames.get(rowTypeFieldIndex)); - updatedRowFields.add(rowFields.get(rowTypeFieldIndex)); - } - } - RowType updatedRowType = - RowType.of( - updatedRowFields.toArray(new LogicalType[0]), - updatedRowFieldNames.toArray(new String[0])); - return updatedRowType; - } - - @Override - public TypeInformation getProducedType() { - return typeInfo; - } - - @Override - public void open(InitializationContext context) throws Exception { - this.nestedSchema.open(context); - } - - @Override - public void deserialize(@Nullable byte[] responseByteMessage, Collector out) - throws IOException { - if (responseByteMessage == null) { - throw new FlinkBigQueryException("Deserializing message is empty"); - } - VectorSchemaRoot root = null; - try { - root = nestedSchema.deserialize(responseByteMessage); - List rowdatalist = (List) runtimeConverter.convert(root); - for (int i = 0; i < rowdatalist.size(); i++) { - out.collect(rowdatalist.get(i)); - } - - } catch (Exception ex) { - throw new FlinkBigQueryException("Error while deserializing Arrow type", ex); - } finally { - if (root != null) { - root.close(); - } - } - } - - @Override - public RowData deserialize(@Nullable byte[] message) throws IOException { - if (message == null) { - return null; - } - RowData rowData; - try { - VectorSchemaRoot root = nestedSchema.deserialize(message); - rowData = (RowData) runtimeConverter.convert(root); - } catch (Exception ex) { - throw new FlinkBigQueryException("Error while deserializing Arrow type", ex); - } - return rowData; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ArrowRowDataDeserializationSchema that = (ArrowRowDataDeserializationSchema) o; - return nestedSchema.equals(that.nestedSchema) && typeInfo.equals(that.typeInfo); - } - - @Override - public int hashCode() { - return Objects.hash(nestedSchema, typeInfo); - } - - @Override - public boolean isEndOfStream(RowData nextElement) { - return nextElement == null; - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/BigQueryArrowFormat.java b/src/main/java/com/google/cloud/flink/bigquery/BigQueryArrowFormat.java deleted file mode 100644 index a6177013..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/BigQueryArrowFormat.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery; - -import java.util.List; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.source.DynamicTableSource.Context; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; - -public class BigQueryArrowFormat implements DecodingFormat> { - private List selectedFieldList; - private List arrowFieldList; - - public BigQueryArrowFormat(List selectedFieldList, List arrowFieldList) { - this.selectedFieldList = selectedFieldList; - this.arrowFieldList = arrowFieldList; - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); - } - - @Override - public DeserializationSchema createRuntimeDecoder( - Context context, DataType producedDataType) { - - final RowType rowType = (RowType) producedDataType.getLogicalType(); - final TypeInformation rowDataTypeInfo = - (TypeInformation) context.createTypeInformation(producedDataType); - return new ArrowRowDataDeserializationSchema( - rowType, rowDataTypeInfo, selectedFieldList, arrowFieldList); - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/BigQueryDynamicTableFactory.java b/src/main/java/com/google/cloud/flink/bigquery/BigQueryDynamicTableFactory.java deleted file mode 100644 index 765ffb98..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/BigQueryDynamicTableFactory.java +++ /dev/null @@ -1,258 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery; - -import com.google.auth.Credentials; -import com.google.cloud.bigquery.connector.common.BigQueryClientFactory; -import com.google.cloud.bigquery.connector.common.BigQueryCredentialsSupplier; -import com.google.cloud.bigquery.storage.v1.ReadSession; -import com.google.cloud.bigquery.storage.v1.ReadStream; -import com.google.cloud.flink.bigquery.common.FlinkBigQueryConnectorUserAgentProvider; -import com.google.cloud.flink.bigquery.common.UserAgentHeaderProvider; -import com.google.common.collect.ImmutableMap; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; -import net.sf.jsqlparser.JSQLParserException; -import org.apache.arrow.vector.ipc.ReadChannel; -import org.apache.arrow.vector.ipc.message.MessageSerializer; -import org.apache.arrow.vector.types.pojo.Field; -import org.apache.arrow.vector.types.pojo.Schema; -import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.runtime.util.EnvironmentInformation; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.DynamicTableSourceFactory; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.types.DataType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public final class BigQueryDynamicTableFactory implements DynamicTableSourceFactory { - - private static final Logger log = LoggerFactory.getLogger(BigQueryDynamicTableFactory.class); - private FlinkBigQueryConfig bqconfig; - private BigQueryClientFactory bigQueryReadClientFactory; - public static final ConfigOption TABLE = - ConfigOptions.key("table").stringType().noDefaultValue(); - public static final ConfigOption QUERY = - ConfigOptions.key("query").stringType().noDefaultValue(); - public static final ConfigOption FILTER = - ConfigOptions.key("filter").stringType().defaultValue(""); - public static final ConfigOption PARALLELISM = - ConfigOptions.key("parallelism").intType().defaultValue(1); - public static final ConfigOption MAX_PARALLELISM = - ConfigOptions.key("maxParallelism").intType().defaultValue(1); - public static final ConfigOption SELECTED_FIELDS = - ConfigOptions.key("selectedFields").stringType().noDefaultValue(); - public static final ConfigOption DEFAULT_PARALLELISM = - ConfigOptions.key("defaultParallelism").intType().defaultValue(1); - public static final ConfigOption CREDENTIAL_KEY_FILE = - ConfigOptions.key("credentialsFile").stringType().noDefaultValue(); - public static final ConfigOption ACCESS_TOKEN = - ConfigOptions.key("gcpAccessToken").stringType().defaultValue(""); - public static final ConfigOption CREDENTIALS_KEY = - ConfigOptions.key("credentials").stringType().defaultValue(""); - public static final ConfigOption PROXY_URI = - ConfigOptions.key("proxyUri").stringType().defaultValue(""); - public static final ConfigOption PROXY_USERNAME = - ConfigOptions.key("proxyUsername").stringType().defaultValue(""); - public static final ConfigOption PROXY_PASSWORD = - ConfigOptions.key("proxyPassword").stringType().defaultValue(""); - public static final ConfigOption BQ_ENCODED_CREATER_READSESSION_REQUEST = - ConfigOptions.key("bqEncodedCreateReadSessionRequest").stringType().noDefaultValue(); - public static final ConfigOption BQ_BACKGROUND_THREADS_PER_STREAM = - ConfigOptions.key("bqBackgroundThreadsPerStream").stringType().noDefaultValue(); - public static final ConfigOption MATERIALIZATION_PROJECT = - ConfigOptions.key("materializationProject").stringType().noDefaultValue(); - public static final ConfigOption MATERIALIZATION_DATASET = - ConfigOptions.key("materializationDataset").stringType().noDefaultValue(); - public static final ConfigOption ARROW_COMPRESSION_CODEC = - ConfigOptions.key("arrowCompressionCodec").stringType().noDefaultValue(); - public static final ConfigOption PARTITION_FIELD = - ConfigOptions.key("partitionField").stringType().defaultValue(""); - public static final ConfigOption PARTITION_TYPE = - ConfigOptions.key("partitionType").stringType().defaultValue(""); - public static final ConfigOption PARTITION_EXPIRATION_MS = - ConfigOptions.key("partitionExpirationMs").stringType().defaultValue(""); - public static final ConfigOption PARTITION_REQUIRE_FILTER = - ConfigOptions.key("partitionRequireFilter").stringType().defaultValue(""); - public static ConfigOption READ_SESSION_ARROW_SCHEMA_FIELDS; - - private String flinkVersion = EnvironmentInformation.getVersion(); - private String arrowFields = ""; - - @Override - public String factoryIdentifier() { - return "bigquery"; - } - - @Override - public Set> requiredOptions() { - final Set> options = new HashSet<>(); - options.add(TABLE); - options.add(SELECTED_FIELDS); - options.add(QUERY); - options.add(MATERIALIZATION_PROJECT); - options.add(MATERIALIZATION_DATASET); - return options; - } - - @Override - public Set> optionalOptions() { - final Set> options = new HashSet<>(); - options.add(FactoryUtil.FORMAT); - options.add(CREDENTIAL_KEY_FILE); - options.add(ACCESS_TOKEN); - options.add(CREDENTIALS_KEY); - options.add(FILTER); - options.add(DEFAULT_PARALLELISM); - options.add(PROXY_URI); - options.add(PROXY_USERNAME); - options.add(PROXY_PASSWORD); - options.add(BQ_ENCODED_CREATER_READSESSION_REQUEST); - options.add(BQ_BACKGROUND_THREADS_PER_STREAM); - options.add(PARALLELISM); - options.add(MAX_PARALLELISM); - options.add(ARROW_COMPRESSION_CODEC); - options.add(PARTITION_FIELD); - options.add(PARTITION_TYPE); - options.add(PARTITION_EXPIRATION_MS); - options.add(PARTITION_REQUIRE_FILTER); - return options; - } - - DecodingFormat> decodingFormat; - - @Override - public DynamicTableSource createDynamicTableSource(Context context) { - final FactoryUtil.TableFactoryHelper helper = - FactoryUtil.createTableFactoryHelper(this, context); - final ReadableConfig options = helper.getOptions(); - try { - helper.validate(); - } catch (Exception ex) { - String exceptionString = ensureExpectedException(ex.toString(), options); - if (exceptionString != null) { - throw new IllegalArgumentException(exceptionString); - } - } - ArrayList readStreams = getReadStreamNames(options); - context.getCatalogTable().getOptions().put("arrowFields", arrowFields); - context.getCatalogTable().getOptions().put("selectedFields", bqconfig.getSelectedFields()); - decodingFormat = helper.discoverDecodingFormat(ArrowFormatFactory.class, FactoryUtil.FORMAT); - - final DataType producedDataType = context.getCatalogTable().getSchema().toPhysicalRowDataType(); - return new BigQueryDynamicTableSource( - decodingFormat, producedDataType, readStreams, bigQueryReadClientFactory); - } - - private ArrayList getReadStreamNames(ReadableConfig options) { - bigQueryReadClientFactory = null; - UserAgentHeaderProvider userAgentHeaderProvider; - BigQueryCredentialsSupplier bigQueryCredentialsSupplier; - ArrayList readStreamNames = new ArrayList(); - try { - - ImmutableMap defaultOptions = - ImmutableMap.of("flinkVersion", EnvironmentInformation.getVersion()); - - bqconfig = - FlinkBigQueryConfig.from( - requiredOptions(), - optionalOptions(), - options, - defaultOptions, - new org.apache.hadoop.conf.Configuration(), - options.get(DEFAULT_PARALLELISM), - new org.apache.flink.configuration.Configuration(), - flinkVersion, - Optional.empty()); - - Credentials credentials = bqconfig.createCredentials(); - bigQueryCredentialsSupplier = - new BigQueryCredentialsSupplier( - bqconfig.getAccessToken(), - bqconfig.getCredentialsKey(), - bqconfig.getCredentialsFile(), - Optional.empty(), - Optional.empty(), - Optional.empty()); - - FlinkBigQueryConnectorUserAgentProvider agentProvider = - new FlinkBigQueryConnectorUserAgentProvider(flinkVersion); - userAgentHeaderProvider = new UserAgentHeaderProvider(agentProvider.getUserAgent()); - bigQueryReadClientFactory = - new BigQueryClientFactory(bigQueryCredentialsSupplier, userAgentHeaderProvider, bqconfig); - - // Create read session - ReadSession readSession = - BigQueryReadSession.getReadsession(credentials, bqconfig, bigQueryReadClientFactory); - List readsessionList = readSession.getStreamsList(); - for (ReadStream stream : readsessionList) { - readStreamNames.add(stream.getName()); - } - - Schema arrowSchema = - MessageSerializer.deserializeSchema( - new ReadChannel( - new ByteArrayReadableSeekableByteChannel( - readSession.getArrowSchema().getSerializedSchema().toByteArray()))); - - arrowFields = - arrowSchema.getFields().stream().map(Field::getName).collect(Collectors.joining(",")); - } catch (JSQLParserException | IOException ex) { - log.error("Error while reading big query session", ex); - throw new FlinkBigQueryException("Error while reading big query session:", ex); - } - return readStreamNames; - } - - private String ensureExpectedException(String exceptionString, ReadableConfig options) { - String errorString = null; - String stringToCheck = "Missing required options are:"; - String exceptionHead = - exceptionString.substring( - 0, exceptionString.lastIndexOf(stringToCheck) + stringToCheck.length()); - ArrayList missingArgs = - new ArrayList<>( - Arrays.asList( - StringUtils.substringAfterLast(exceptionString, stringToCheck).trim().split("\n"))); - if (options.get(TABLE) != null) { - missingArgs.remove("query"); - missingArgs.remove("materializationProject"); - missingArgs.remove("materializationDataset"); - } else if (options.get(QUERY) != null) { - missingArgs.remove("table"); - missingArgs.remove("selectedFields"); - } - if (!missingArgs.isEmpty()) { - errorString = errorString + exceptionHead + "\n\n" + String.join("\n", missingArgs); - } - return errorString; - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/BigQueryDynamicTableSource.java b/src/main/java/com/google/cloud/flink/bigquery/BigQueryDynamicTableSource.java deleted file mode 100644 index 9bbc2923..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/BigQueryDynamicTableSource.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery; - -import com.google.cloud.bigquery.connector.common.BigQueryClientFactory; -import java.util.ArrayList; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.DecodingFormat; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.connector.source.ScanTableSource; -import org.apache.flink.table.connector.source.SourceFunctionProvider; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.sources.ProjectableTableSource; -import org.apache.flink.table.sources.TableSource; -import org.apache.flink.table.types.DataType; - -public final class BigQueryDynamicTableSource - implements ScanTableSource, ProjectableTableSource { - - private final DecodingFormat> decodingFormat; - private DataType producedDataType; - private ArrayList readStreamNames; - private BigQueryClientFactory bigQueryReadClientFactory; - - public BigQueryDynamicTableSource( - DecodingFormat> decodingFormat, - DataType producedDataType, - ArrayList readStreamNames, - BigQueryClientFactory bigQueryReadClientFactory) { - - this.bigQueryReadClientFactory = bigQueryReadClientFactory; - this.decodingFormat = decodingFormat; - this.producedDataType = producedDataType; - this.readStreamNames = readStreamNames; - } - - @Override - public ChangelogMode getChangelogMode() { - return decodingFormat.getChangelogMode(); - } - - @Override - public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { - // create runtime classes that are shipped to the cluster - final DeserializationSchema deserializer = - decodingFormat.createRuntimeDecoder(runtimeProviderContext, producedDataType); - final SourceFunction sourceFunction = - new BigQuerySourceFunction(deserializer, readStreamNames, bigQueryReadClientFactory); - return SourceFunctionProvider.of(sourceFunction, false); - } - - @Override - public DynamicTableSource copy() { - return new BigQueryDynamicTableSource( - decodingFormat, producedDataType, readStreamNames, bigQueryReadClientFactory); - } - - @Override - public String asSummaryString() { - return "BigQuery Table Source"; - } - - @Override - public TableSource projectFields(int[] fields) { - return null; - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/BigQueryReadSession.java b/src/main/java/com/google/cloud/flink/bigquery/BigQueryReadSession.java deleted file mode 100644 index 48ead648..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/BigQueryReadSession.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery; - -import com.google.auth.Credentials; -import com.google.cloud.bigquery.BigQuery; -import com.google.cloud.bigquery.BigQueryOptions; -import com.google.cloud.bigquery.TableId; -import com.google.cloud.bigquery.TableInfo; -import com.google.cloud.bigquery.connector.common.BigQueryClient; -import com.google.cloud.bigquery.connector.common.BigQueryClientFactory; -import com.google.cloud.bigquery.connector.common.ReadSessionCreator; -import com.google.cloud.bigquery.connector.common.ReadSessionCreatorConfig; -import com.google.cloud.bigquery.connector.common.ReadSessionResponse; -import com.google.cloud.bigquery.storage.v1.ReadSession; -import com.google.common.collect.ImmutableList; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.Arrays; -import java.util.Optional; -import net.sf.jsqlparser.JSQLParserException; - -public class BigQueryReadSession { - - public static ReadSession getReadsession( - Credentials credentials, - FlinkBigQueryConfig bqconfig, - BigQueryClientFactory bigQueryReadClientFactory) - throws FileNotFoundException, IOException, JSQLParserException { - - final BigQuery bigquery = - BigQueryOptions.newBuilder().setCredentials(credentials).build().getService(); - Optional materializationProject = - bqconfig.getQuery().isPresent() - ? Optional.of(bqconfig.getParentProjectId()) - : Optional.empty(); - Optional materializationDataset = - bqconfig.getQuery().isPresent() ? bqconfig.getMaterializationDataset() : Optional.empty(); - - BigQueryClient bigQueryClient = - new BigQueryClient(bigquery, materializationProject, materializationDataset); - ReadSessionCreatorConfig readSessionCreatorConfig = bqconfig.toReadSessionCreatorConfig(); - ReadSessionCreator readSessionCreator = - new ReadSessionCreator(readSessionCreatorConfig, bigQueryClient, bigQueryReadClientFactory); - - TableId tabId = null; - if (bqconfig.getQuery().isPresent()) { - - int expirationTimeInMinutes = bqconfig.getMaterializationExpirationTimeInMinutes(); - TableInfo tableInfo = - bigQueryClient.materializeQueryToTable( - bqconfig.getQuery().get(), expirationTimeInMinutes); - tabId = tableInfo.getTableId(); - } - - TableId tableId = bqconfig.getQuery().isPresent() ? tabId : bqconfig.getTableId(); - ImmutableList selectedFields = - ImmutableList.copyOf(Arrays.asList(bqconfig.getSelectedFields().split(","))); - Optional filter = - bqconfig.getFilter().isPresent() ? bqconfig.getFilter() : Optional.empty(); - ReadSessionResponse response = readSessionCreator.create(tableId, selectedFields, filter); - return response.getReadSession(); - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/BigQuerySourceFunction.java b/src/main/java/com/google/cloud/flink/bigquery/BigQuerySourceFunction.java deleted file mode 100644 index 92908165..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/BigQuerySourceFunction.java +++ /dev/null @@ -1,141 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * numOfStreamsations under the License. - */ -package com.google.cloud.flink.bigquery; - -import com.google.cloud.bigquery.connector.common.BigQueryClientFactory; -import com.google.cloud.bigquery.connector.common.ReadRowsHelper; -import com.google.cloud.bigquery.connector.common.ReadRowsHelper.Options; -import com.google.cloud.bigquery.storage.v1.ReadRowsRequest; -import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Optional; -import org.apache.flink.api.common.functions.util.ListCollector; -import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; -import org.apache.flink.table.data.RowData; -import org.apache.flink.util.Collector; -import org.apache.flink.util.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public final class BigQuerySourceFunction extends RichParallelSourceFunction - implements ResultTypeQueryable { - - boolean running = true; - private static final long serialVersionUID = 1; - private static final Logger log = LoggerFactory.getLogger(BigQuerySourceFunction.class); - private int numOfStreams; - private int executerIndex; - private DeserializationSchema deserializer; - private ArrayList readSessionStreamList = new ArrayList<>(); - private BigQueryClientFactory bigQueryReadClientFactory; - private List streamNames = new ArrayList(); - private int numOfExecutors; - - public BigQuerySourceFunction() {} - - public BigQuerySourceFunction( - DeserializationSchema deserializer, - ArrayList readSessionStreams, - BigQueryClientFactory bigQueryReadClientFactory) { - - this.deserializer = deserializer; - this.readSessionStreamList = readSessionStreams; - this.numOfStreams = readSessionStreamList.size(); - this.bigQueryReadClientFactory = bigQueryReadClientFactory; - } - - @Override - public TypeInformation getProducedType() { - return deserializer.getProducedType(); - } - - @Override - public void open(Configuration parameters) throws Exception { - deserializer.open(() -> getRuntimeContext().getMetricGroup().addGroup("bigQuery")); - this.executerIndex = getRuntimeContext().getIndexOfThisSubtask(); - this.numOfExecutors = getRuntimeContext().getNumberOfParallelSubtasks(); - this.numOfStreams = readSessionStreamList.size(); - this.streamNames.clear(); - for (int i = executerIndex; i < numOfStreams; i += numOfExecutors) { - if (running) { - this.streamNames.add(readSessionStreamList.get(i)); - } - } - } - - @SuppressWarnings("resource") - @Override - public void run(SourceContext ctx) throws Exception { - List outputCollector = new ArrayList<>(); - ListCollector listCollector = new ListCollector<>(outputCollector); - Options options = - new ReadRowsHelper.Options( - /* maxReadRowsRetries= */ 5, - Optional.of("endpoint"), - /* backgroundParsingThreads= */ 5, - 1); - - for (String streamName : streamNames) { - ReadRowsRequest.Builder readRowsRequest = - ReadRowsRequest.newBuilder().setReadStream(streamName); - ReadRowsHelper readRowsHelper = - new ReadRowsHelper(bigQueryReadClientFactory, readRowsRequest, options); - - Iterator readRows = readRowsHelper.readRows(); - while (readRows.hasNext()) { - ReadRowsResponse response = readRows.next(); - try { - if (response.hasArrowRecordBatch()) { - Preconditions.checkState(response.hasArrowRecordBatch()); - deserializer.deserialize(response.toByteArray(), (Collector) listCollector); - - } else if (response.hasAvroRows()) { - Preconditions.checkState(response.hasAvroRows()); - deserializer.deserialize(response.toByteArray(), (Collector) listCollector); - break; - } - } catch (IOException ex) { - log.error("Error while deserialization:", ex); - throw new FlinkBigQueryException("Error while deserialization:", ex); - } - } - readRowsHelper.close(); - } - - for (int i = 0; i < outputCollector.size(); i++) { - ctx.collect((RowData) outputCollector.get(i)); - } - outputCollector.clear(); - } - - @Override - public void close() { - running = false; - } - - @Override - public void cancel() { - // TODO Auto-generated method stub - - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/FlinkBigQueryConfig.java b/src/main/java/com/google/cloud/flink/bigquery/FlinkBigQueryConfig.java deleted file mode 100644 index 67b3e7c8..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/FlinkBigQueryConfig.java +++ /dev/null @@ -1,812 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery; - -import static com.google.cloud.bigquery.connector.common.BigQueryUtil.firstPresent; -import static com.google.cloud.bigquery.connector.common.BigQueryUtil.parseTableId; -import static java.lang.String.format; - -import com.google.api.gax.retrying.RetrySettings; -import com.google.auth.Credentials; -import com.google.cloud.bigquery.BigQueryOptions; -import com.google.cloud.bigquery.JobInfo; -import com.google.cloud.bigquery.TableId; -import com.google.cloud.bigquery.TimePartitioning; -import com.google.cloud.bigquery.connector.common.BigQueryClient; -import com.google.cloud.bigquery.connector.common.BigQueryConfig; -import com.google.cloud.bigquery.connector.common.BigQueryCredentialsSupplier; -import com.google.cloud.bigquery.connector.common.BigQueryProxyConfig; -import com.google.cloud.bigquery.connector.common.ReadSessionCreatorConfig; -import com.google.cloud.bigquery.connector.common.ReadSessionCreatorConfigBuilder; -import com.google.cloud.bigquery.storage.v1.ArrowSerializationOptions.CompressionCodec; -import com.google.cloud.bigquery.storage.v1.DataFormat; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import java.io.Serializable; -import java.time.format.DateTimeFormatter; -import java.time.format.DateTimeParseException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.OptionalInt; -import java.util.OptionalLong; -import java.util.Set; -import java.util.function.Supplier; -import java.util.regex.Pattern; -import java.util.stream.Collectors; -import net.sf.jsqlparser.JSQLParserException; -import net.sf.jsqlparser.parser.CCJSqlParserUtil; -import net.sf.jsqlparser.statement.Statement; -import net.sf.jsqlparser.statement.select.PlainSelect; -import net.sf.jsqlparser.statement.select.Select; -import net.sf.jsqlparser.statement.select.SelectItem; -import net.sf.jsqlparser.util.TablesNamesFinder; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.TableSchema; -import org.apache.hadoop.conf.Configuration; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.threeten.bp.Duration; - -public class FlinkBigQueryConfig implements BigQueryConfig, Serializable { - - private static final long serialVersionUID = 1L; - final Logger logger = LoggerFactory.getLogger(FlinkBigQueryConfig.class); - - public enum WriteMethod { - DIRECT, - INDIRECT; - - public static WriteMethod from(@Nullable String writeMethod) { - try { - return WriteMethod.valueOf(writeMethod.toUpperCase(Locale.ENGLISH)); - } catch (RuntimeException e) { - throw new IllegalArgumentException( - "WriteMethod can be only " + Arrays.toString(WriteMethod.values())); - } - } - } - - public static final String VIEWS_ENABLED_OPTION = "viewsEnabled"; - public static final String USE_AVRO_LOGICAL_TYPES_OPTION = "useAvroLogicalTypes"; - public static final String DATE_PARTITION_PARAM = "datePartition"; - public static final String VALIDATE_SPARK_AVRO_PARAM = "validateSparkAvroInternalParam"; - public static final String INTERMEDIATE_FORMAT_OPTION = "intermediateFormat"; - public static final int DEFAULT_MATERIALIZATION_EXPRIRATION_TIME_IN_MINUTES = 24 * 60; - @VisibleForTesting static final DataFormat DEFAULT_READ_DATA_FORMAT = DataFormat.ARROW; - - @VisibleForTesting - static final CompressionCodec DEFAULT_ARROW_COMPRESSION_CODEC = - CompressionCodec.COMPRESSION_UNSPECIFIED; - - static final String GCS_CONFIG_CREDENTIALS_FILE_PROPERTY = - "google.cloud.auth.service.account.json.keyfile"; - static final String GCS_CONFIG_PROJECT_ID_PROPERTY = "q-gcp-6750-pso-gs-flink-22-01"; - private static final String READ_DATA_FORMAT_OPTION = DataFormat.ARROW.toString(); - private static final ImmutableList PERMITTED_READ_DATA_FORMATS = - ImmutableList.of(DataFormat.ARROW.toString(), DataFormat.AVRO.toString()); - private static final Supplier> DEFAULT_FALLBACK = - () -> empty(); - private static final String CONF_PREFIX = "flink.datasource.bigquery."; - private static final int DEFAULT_BIGQUERY_CLIENT_CONNECT_TIMEOUT = 60 * 1000; - private static final int DEFAULT_BIGQUERY_CLIENT_READ_TIMEOUT = 60 * 1000; - private static final Pattern LOWERCASE_QUERY_PATTERN = Pattern.compile("^(select|with)\\s+.*$"); - - public static final int MIN_BUFFERED_RESPONSES_PER_STREAM = 1; - public static final int MIN_STREAMS_PER_PARTITION = 1; - private static final int DEFAULT_BIGQUERY_CLIENT_RETRIES = 10; - private static final String ARROW_COMPRESSION_CODEC_OPTION = "arrowCompressionCodec"; - private static final WriteMethod DEFAULT_WRITE_METHOD = WriteMethod.INDIRECT; - - TableId tableId; - String selectedFields; - String flinkVersion; - com.google.common.base.Optional query = empty(); - String parentProjectId; - boolean useParentProjectForMetadataOperations; - com.google.common.base.Optional credentialsKey; - com.google.common.base.Optional credentialsFile; - com.google.common.base.Optional accessToken; - com.google.common.base.Optional filter = empty(); - com.google.common.base.Optional schema = empty(); - Integer maxParallelism = null; - int defaultParallelism = 1; - com.google.common.base.Optional temporaryGcsBucket = empty(); - com.google.common.base.Optional persistentGcsBucket = empty(); - com.google.common.base.Optional persistentGcsPath = empty(); - - DataFormat readDataFormat = DEFAULT_READ_DATA_FORMAT; - boolean combinePushedDownFilters = true; - boolean viewsEnabled = false; - com.google.common.base.Optional materializationProject = empty(); - com.google.common.base.Optional materializationDataset = empty(); - com.google.common.base.Optional partitionField = empty(); - Long partitionExpirationMs = null; - com.google.common.base.Optional partitionRequireFilter = empty(); - com.google.common.base.Optional partitionType = empty(); - com.google.common.base.Optional clusteredFields = empty(); - com.google.common.base.Optional createDisposition = empty(); - boolean optimizedEmptyProjection = true; - boolean useAvroLogicalTypes = false; - ImmutableList loadSchemaUpdateOptions = ImmutableList.of(); - int materializationExpirationTimeInMinutes = DEFAULT_MATERIALIZATION_EXPRIRATION_TIME_IN_MINUTES; - int maxReadRowsRetries = 3; - boolean pushAllFilters = true; - private com.google.common.base.Optional encodedCreateReadSessionRequest = empty(); - private com.google.common.base.Optional storageReadEndpoint = empty(); - private int numBackgroundThreadsPerStream = 0; - private int numPrebufferReadRowsResponses = MIN_BUFFERED_RESPONSES_PER_STREAM; - private int numStreamsPerPartition = MIN_STREAMS_PER_PARTITION; - private FlinkBigQueryProxyAndHttpConfig flinkBigQueryProxyAndHttpConfig; - private CompressionCodec arrowCompressionCodec = DEFAULT_ARROW_COMPRESSION_CODEC; - private WriteMethod writeMethod = DEFAULT_WRITE_METHOD; - // for V2 write with BigQuery Storage Write API - RetrySettings bigqueryDataWriteHelperRetrySettings = - RetrySettings.newBuilder().setMaxAttempts(5).build(); - private String fieldList; - - @VisibleForTesting - public static FlinkBigQueryConfig from( - Set> requiredOptions, - Set> optionalOptions, - ReadableConfig readableConfigOptions, - ImmutableMap originalGlobalOptions, - Configuration hadoopConfiguration, - int defaultParallelism, - org.apache.flink.configuration.Configuration sqlConf, - String flinkVersion, - Optional schema) { - FlinkBigQueryConfig config = new FlinkBigQueryConfig(); - ImmutableMap options = - toConfigOptionsKeyMap(requiredOptions, optionalOptions, readableConfigOptions); - ImmutableMap globalOptions = normalizeConf(originalGlobalOptions); - config.flinkBigQueryProxyAndHttpConfig = - FlinkBigQueryProxyAndHttpConfig.from(options, globalOptions, hadoopConfiguration); - - // we need those parameters in case a read from query is issued - config.viewsEnabled = getAnyBooleanOption(globalOptions, options, VIEWS_ENABLED_OPTION, false); - config.flinkVersion = flinkVersion; - config.materializationProject = - getAnyOption( - globalOptions, - options, - ImmutableList.of("materializationProject", "viewMaterializationProject")); - config.materializationDataset = - getAnyOption( - globalOptions, - options, - ImmutableList.of("materializationDataset", "viewMaterializationDataset")); - config.materializationExpirationTimeInMinutes = - getAnyOption(globalOptions, options, "materializationExpirationTimeInMinutes") - .transform(Integer::parseInt) - .or(DEFAULT_MATERIALIZATION_EXPRIRATION_TIME_IN_MINUTES); - if (config.materializationExpirationTimeInMinutes < 1) { - throw new IllegalArgumentException( - "materializationExpirationTimeInMinutes must have a positive value, the configured value" - + " is " - + config.materializationExpirationTimeInMinutes); - } - // get the table details - Optional tableParam = - getOptionFromMultipleParams(options, ImmutableList.of("table", "path"), DEFAULT_FALLBACK) - .toJavaUtil(); - Optional datasetParam = - getOption(options, "dataset").or(config.materializationDataset).toJavaUtil(); - Optional projectParam = - firstPresent( - getOption(options, "project").toJavaUtil(), - com.google.common.base.Optional.fromNullable( - hadoopConfiguration.get(GCS_CONFIG_PROJECT_ID_PROPERTY)) - .toJavaUtil()); - config.partitionType = - getOption(options, "partitionType").transform(TimePartitioning.Type::valueOf); - Optional datePartitionParam = getOption(options, DATE_PARTITION_PARAM).toJavaUtil(); - datePartitionParam.ifPresent( - date -> validateDateFormat(date, config.getPartitionTypeOrDefault(), DATE_PARTITION_PARAM)); - // checking for query - if (tableParam.isPresent()) { - String tableParamStr = tableParam.get().trim().replaceAll("\\s+", " "); - if (isQuery(tableParamStr)) { - // it is a query in practice - config.query = com.google.common.base.Optional.of(tableParamStr); - config.tableId = parseTableId("QUERY", datasetParam, projectParam, datePartitionParam); - } else { - config.selectedFields = options.get("selectedFields"); - config.tableId = - parseTableId(tableParamStr, datasetParam, projectParam, datePartitionParam); - } - } else { - // no table has been provided, it is either a query or an error - config.query = getOption(options, "query").transform(String::trim); - if (config.query.isPresent()) { - config.tableId = parseTableId("QUERY", datasetParam, projectParam, datePartitionParam); - } else { - // No table nor query were set. We cannot go further. - throw new IllegalArgumentException("No table has been specified"); - } - } - - config.parentProjectId = - getAnyOption(globalOptions, options, "parentProject").or(defaultBilledProject()); - config.useParentProjectForMetadataOperations = - getAnyBooleanOption(globalOptions, options, "useParentProjectForMetadataOperations", false); - config.credentialsKey = getAnyOption(globalOptions, options, "credentials"); - config.credentialsFile = - fromJavaUtil( - firstPresent( - getAnyOption(globalOptions, options, "credentialsFile").toJavaUtil(), - com.google.common.base.Optional.fromNullable( - hadoopConfiguration.get(GCS_CONFIG_CREDENTIALS_FILE_PROPERTY)) - .toJavaUtil())); - config.accessToken = getAnyOption(globalOptions, options, "gcpAccessToken"); - config.filter = getOption(options, "filter"); - config.schema = fromJavaUtil(schema); - config.maxParallelism = - getOptionFromMultipleParams( - options, ImmutableList.of("maxParallelism", "parallelism"), DEFAULT_FALLBACK) - .transform(Integer::valueOf) - .orNull(); - config.defaultParallelism = defaultParallelism; - config.temporaryGcsBucket = getAnyOption(globalOptions, options, "temporaryGcsBucket"); - config.persistentGcsBucket = getAnyOption(globalOptions, options, "persistentGcsBucket"); - config.persistentGcsPath = getOption(options, "persistentGcsPath"); - String readDataFormatParam = - getAnyOption(globalOptions, options, "format") - .transform(String::toUpperCase) - .or(DEFAULT_READ_DATA_FORMAT.toString()); - if (!PERMITTED_READ_DATA_FORMATS.contains(readDataFormatParam)) { - throw new IllegalArgumentException( - format( - "Data read format '%s' is not supported. Supported formats are '%s'", - readDataFormatParam, String.join(",", PERMITTED_READ_DATA_FORMATS))); - } - config.useAvroLogicalTypes = - getAnyBooleanOption(globalOptions, options, USE_AVRO_LOGICAL_TYPES_OPTION, false); - config.readDataFormat = DataFormat.valueOf(readDataFormatParam); - config.combinePushedDownFilters = - getAnyBooleanOption(globalOptions, options, "combinePushedDownFilters", true); - - config.partitionField = getOption(options, "partitionField"); - config.partitionExpirationMs = - getOption(options, "partitionExpirationMs").transform(Long::valueOf).orNull(); - config.partitionRequireFilter = - getOption(options, "partitionRequireFilter").transform(Boolean::valueOf); - config.clusteredFields = getOption(options, "clusteredFields").transform(s -> s.split(",")); - - config.createDisposition = - getOption(options, "createDisposition") - .transform(String::toUpperCase) - .transform(JobInfo.CreateDisposition::valueOf); - - config.optimizedEmptyProjection = - getAnyBooleanOption(globalOptions, options, "optimizedEmptyProjection", true); - - boolean allowFieldAddition = - getAnyBooleanOption(globalOptions, options, "allowFieldAddition", false); - boolean allowFieldRelaxation = - getAnyBooleanOption(globalOptions, options, "allowFieldRelaxation", false); - ImmutableList.Builder loadSchemaUpdateOptions = - ImmutableList.builder(); - if (allowFieldAddition) { - loadSchemaUpdateOptions.add(JobInfo.SchemaUpdateOption.ALLOW_FIELD_ADDITION); - } - if (allowFieldRelaxation) { - loadSchemaUpdateOptions.add(JobInfo.SchemaUpdateOption.ALLOW_FIELD_RELAXATION); - } - config.loadSchemaUpdateOptions = loadSchemaUpdateOptions.build(); - config.storageReadEndpoint = getAnyOption(globalOptions, options, "bqStorageReadEndpoint"); - config.encodedCreateReadSessionRequest = - getAnyOption(globalOptions, options, "bqEncodedCreateReadSessionRequest"); - config.numBackgroundThreadsPerStream = - getAnyOption(globalOptions, options, "bqBackgroundThreadsPerStream") - .transform(Integer::parseInt) - .or(0); - config.pushAllFilters = getAnyBooleanOption(globalOptions, options, "pushAllFilters", true); - config.numPrebufferReadRowsResponses = - getAnyOption(globalOptions, options, "bqPrebufferResponsesPerStream") - .transform(Integer::parseInt) - .or(MIN_BUFFERED_RESPONSES_PER_STREAM); - config.numStreamsPerPartition = - getAnyOption(globalOptions, options, "bqNumStreamsPerPartition") - .transform(Integer::parseInt) - .or(MIN_STREAMS_PER_PARTITION); - - String arrowCompressionCodecParam = - getAnyOption(globalOptions, options, ARROW_COMPRESSION_CODEC_OPTION) - .transform(String::toUpperCase) - .or(DEFAULT_ARROW_COMPRESSION_CODEC.toString()); - - config.writeMethod = - getAnyOption(globalOptions, options, "writeMethod") - .transform(WriteMethod::from) - .or(DEFAULT_WRITE_METHOD); - - try { - config.arrowCompressionCodec = CompressionCodec.valueOf(arrowCompressionCodecParam); - } catch (IllegalArgumentException e) { - throw new IllegalArgumentException( - format( - "Compression codec '%s' for Arrow is not supported. Supported formats are %s", - arrowCompressionCodecParam, Arrays.toString(CompressionCodec.values()))); - } - - return config; - } - - private static ImmutableMap toConfigOptionsKeyMap( - Set> requiredOptions, - Set> optionalOptions, - ReadableConfig readableConfig) { - - Map result = new HashMap<>(); - - requiredOptions.stream() - .forEach( - ele -> - result.put( - ele.key(), - readableConfig.get( - ConfigOptions.key(ele.key()).stringType().noDefaultValue()))); - optionalOptions.stream() - .filter(ele -> Objects.nonNull(ele)) - .forEach( - ele -> - result.put( - ele.key(), - readableConfig.get( - ConfigOptions.key(ele.key()).stringType().noDefaultValue()))); - result.values().removeIf(Objects::isNull); - return ImmutableMap.copyOf(result); - } - - @VisibleForTesting - static boolean isQuery(String tableParamStr) { - String potentialQuery = tableParamStr.toLowerCase().replace('\n', ' '); - return LOWERCASE_QUERY_PATTERN.matcher(potentialQuery).matches(); - } - - private static void validateDateFormat( - String date, TimePartitioning.Type partitionType, String optionName) { - try { - Map formatterMap = - ImmutableMap.of( - TimePartitioning.Type.HOUR, DateTimeFormatter.ofPattern("yyyyMMddHH"), // - TimePartitioning.Type.DAY, DateTimeFormatter.BASIC_ISO_DATE, // - TimePartitioning.Type.MONTH, DateTimeFormatter.ofPattern("yyyyMM"), // - TimePartitioning.Type.YEAR, DateTimeFormatter.ofPattern("yyyy")); - DateTimeFormatter dateTimeFormatter = formatterMap.get(partitionType); - dateTimeFormatter.parse(date); - } catch (DateTimeParseException e) { - throw new IllegalArgumentException( - String.format("Invalid argument for option %s, format is YYYYMMDD", optionName)); - } - } - - private static com.google.common.base.Supplier defaultBilledProject() { - return () -> BigQueryOptions.getDefaultInstance().getProjectId(); - } - - private static String getRequiredOption(Map options, String name) { - return getOption(options, name, DEFAULT_FALLBACK) - .toJavaUtil() - .orElseThrow(() -> new IllegalArgumentException(format("Option %s required.", name))); - } - - private static String getRequiredOption( - Map options, String name, com.google.common.base.Supplier fallback) { - return getOption(options, name, DEFAULT_FALLBACK).or(fallback); - } - - private static com.google.common.base.Optional getOption( - Map options, String name) { - return getOption(options, name, DEFAULT_FALLBACK); - } - - private static com.google.common.base.Optional getOption( - Map options, - String name, - Supplier> fallback) { - return fromJavaUtil( - firstPresent(Optional.ofNullable(options.get(name)), fallback.get().toJavaUtil())); - } - - private static com.google.common.base.Optional getOptionFromMultipleParams( - Map options, - Collection names, - Supplier> fallback) { - return names.stream() - .map(name -> getOption(options, name)) - .filter(com.google.common.base.Optional::isPresent) - .findFirst() - .orElseGet(fallback); - } - - private static com.google.common.base.Optional getAnyOption( - ImmutableMap globalOptions, Map options, String name) { - return com.google.common.base.Optional.fromNullable(options.get(name)) - .or(com.google.common.base.Optional.fromNullable(globalOptions.get(name))); - } - - // gives the option to support old configurations as fallback - // Used to provide backward compatibility - private static com.google.common.base.Optional getAnyOption( - ImmutableMap globalOptions, - Map options, - Collection names) { - return names.stream() - .map(name -> getAnyOption(globalOptions, options, name)) - .filter(optional -> optional.isPresent()) - .findFirst() - .orElse(empty()); - } - - private static boolean getAnyBooleanOption( - ImmutableMap globalOptions, - Map options, - String name, - boolean defaultValue) { - return getAnyOption(globalOptions, options, name).transform(Boolean::valueOf).or(defaultValue); - } - - public static ImmutableMap normalizeConf(Map conf) { - Map normalizeConf = - conf.entrySet().stream() - .filter(e -> e.getKey().startsWith(CONF_PREFIX)) - .collect( - Collectors.toMap( - e -> e.getKey().substring(CONF_PREFIX.length()), e -> e.getValue())); - Map result = new HashMap<>(conf); - result.putAll(normalizeConf); - return ImmutableMap.copyOf(result); - } - - private static com.google.common.base.Optional empty() { - return com.google.common.base.Optional.absent(); - } - - private static com.google.common.base.Optional fromJavaUtil(Optional o) { - return com.google.common.base.Optional.fromJavaUtil(o); - } - - public Credentials createCredentials() { - - return new BigQueryCredentialsSupplier( - accessToken.toJavaUtil(), - credentialsKey.toJavaUtil(), - credentialsFile.toJavaUtil(), - flinkBigQueryProxyAndHttpConfig.getProxyUri(), - flinkBigQueryProxyAndHttpConfig.getProxyUsername(), - flinkBigQueryProxyAndHttpConfig.getProxyPassword()) - .getCredentials(); - } - - public TableId getTableId() { - return tableId; - } - - public void setTableId(TableId tableId) { - this.tableId = tableId; - } - - public String getFlinkVersion() { - return this.flinkVersion; - } - - public String getSelectedFields() { - String selectedFieldString = null; - try { - selectedFieldString = - selectedFields != null ? selectedFields : new SqlParser(query).getSelectedFields(); - } catch (JSQLParserException e) { - logger.error("Error while parsing sql query", e); - } - return selectedFieldString; - } - - /** Returns the table id, without the added partition id if it exists. */ - public TableId getTableIdWithoutThePartition() { - String tableAndPartition = tableId.getTable(); - if (!tableAndPartition.contains("$")) { - // there is no partition id - return tableId; - } - String table = tableAndPartition.substring(0, tableAndPartition.indexOf('$')); - return tableId.getProject() != null - ? TableId.of(tableId.getProject(), tableId.getDataset(), table) - : TableId.of(tableId.getDataset(), table); - } - - public Optional getQuery() { - return query.toJavaUtil(); - } - - @Override - public String getParentProjectId() { - return parentProjectId; - } - - @Override - public boolean useParentProjectForMetadataOperations() { - return useParentProjectForMetadataOperations; - } - - @Override - public Optional getCredentialsKey() { - return credentialsKey.toJavaUtil(); - } - - @Override - public Optional getCredentialsFile() { - return credentialsFile.toJavaUtil(); - } - - @Override - public Optional getAccessToken() { - return accessToken.toJavaUtil(); - } - - public Optional getFilter() { - return filter.toJavaUtil(); - } - - public Optional getSchema() { - return schema.toJavaUtil(); - } - - public void setArrowSchemaFields(String fieldList) { - this.fieldList = fieldList; - } - - public String getArrowSchemaFields() { - return this.fieldList; - } - - public OptionalInt getMaxParallelism() { - return maxParallelism == null ? OptionalInt.empty() : OptionalInt.of(maxParallelism); - } - - public int getDefaultParallelism() { - return defaultParallelism; - } - - public Optional getTemporaryGcsBucket() { - return temporaryGcsBucket.toJavaUtil(); - } - - public Optional getPersistentGcsBucket() { - return persistentGcsBucket.toJavaUtil(); - } - - public Optional getPersistentGcsPath() { - return persistentGcsPath.toJavaUtil(); - } - - public DataFormat getReadDataFormat() { - return readDataFormat; - } - - public CompressionCodec getArrowCompressionCodec() { - return arrowCompressionCodec; - } - - public boolean isCombinePushedDownFilters() { - return combinePushedDownFilters; - } - - public boolean isUseAvroLogicalTypes() { - return useAvroLogicalTypes; - } - - public boolean isViewsEnabled() { - return viewsEnabled; - } - - @Override - public Optional getMaterializationProject() { - return materializationProject.toJavaUtil(); - } - - @Override - public Optional getMaterializationDataset() { - return materializationDataset.toJavaUtil(); - } - - public Optional getPartitionField() { - return partitionField.toJavaUtil(); - } - - public OptionalLong getPartitionExpirationMs() { - return partitionExpirationMs == null - ? OptionalLong.empty() - : OptionalLong.of(partitionExpirationMs); - } - - public Optional getPartitionRequireFilter() { - return partitionRequireFilter.toJavaUtil(); - } - - public Optional getPartitionType() { - return partitionType.toJavaUtil(); - } - - public TimePartitioning.Type getPartitionTypeOrDefault() { - return partitionType.or(TimePartitioning.Type.DAY); - } - - public Optional> getClusteredFields() { - return clusteredFields.transform(fields -> ImmutableList.copyOf(fields)).toJavaUtil(); - } - - public Optional getCreateDisposition() { - return createDisposition.toJavaUtil(); - } - - public boolean isOptimizedEmptyProjection() { - return optimizedEmptyProjection; - } - - public ImmutableList getLoadSchemaUpdateOptions() { - return loadSchemaUpdateOptions; - } - - public int getMaterializationExpirationTimeInMinutes() { - return materializationExpirationTimeInMinutes; - } - - public int getMaxReadRowsRetries() { - return maxReadRowsRetries; - } - - public boolean getPushAllFilters() { - return pushAllFilters; - } - - @Override - public int getBigQueryClientConnectTimeout() { - return flinkBigQueryProxyAndHttpConfig - .getHttpConnectTimeout() - .orElse(DEFAULT_BIGQUERY_CLIENT_CONNECT_TIMEOUT); - } - - @Override - public int getBigQueryClientReadTimeout() { - return flinkBigQueryProxyAndHttpConfig - .getHttpReadTimeout() - .orElse(DEFAULT_BIGQUERY_CLIENT_READ_TIMEOUT); - } - - @Override - public BigQueryProxyConfig getBigQueryProxyConfig() { - return flinkBigQueryProxyAndHttpConfig; - } - - @Override - public Optional getEndpoint() { - return storageReadEndpoint.toJavaUtil(); - } - - @Override - public RetrySettings getBigQueryClientRetrySettings() { - return RetrySettings.newBuilder() - .setMaxAttempts( - flinkBigQueryProxyAndHttpConfig - .getHttpMaxRetry() - .orElse(DEFAULT_BIGQUERY_CLIENT_RETRIES)) - .setTotalTimeout(Duration.ofMinutes(10)) - .setInitialRpcTimeout(Duration.ofSeconds(60)) - .setMaxRpcTimeout(Duration.ofMinutes(5)) - .setRpcTimeoutMultiplier(1.6) - .setRetryDelayMultiplier(1.6) - .setInitialRetryDelay(Duration.ofMillis(1250)) - .setMaxRetryDelay(Duration.ofSeconds(5)) - .build(); - } - - public RetrySettings getBigqueryDataWriteHelperRetrySettings() { - return bigqueryDataWriteHelperRetrySettings; - } - - public WriteMethod getWriteMethod() { - return writeMethod; - } - - public ReadSessionCreatorConfig toReadSessionCreatorConfig() { - return new ReadSessionCreatorConfigBuilder() - .setViewsEnabled(viewsEnabled) - .setMaterializationProject(materializationProject.toJavaUtil()) - .setMaterializationDataset(materializationDataset.toJavaUtil()) - .setMaterializationExpirationTimeInMinutes(materializationExpirationTimeInMinutes) - .setReadDataFormat(readDataFormat) - .setMaxReadRowsRetries(maxReadRowsRetries) - .setViewEnabledParamName(VIEWS_ENABLED_OPTION) - .setDefaultParallelism(defaultParallelism) - .setMaxParallelism(getMaxParallelism()) - .setRequestEncodedBase(encodedCreateReadSessionRequest.toJavaUtil()) - .setEndpoint(storageReadEndpoint.toJavaUtil()) - .setBackgroundParsingThreads(numBackgroundThreadsPerStream) - .setPushAllFilters(pushAllFilters) - .setPrebufferReadRowsResponses(numPrebufferReadRowsResponses) - .setStreamsPerPartition(numStreamsPerPartition) - .setArrowCompressionCodec(arrowCompressionCodec) - .build(); - } - - public BigQueryClient.ReadTableOptions toReadTableOptions() { - return new BigQueryClient.ReadTableOptions() { - @Override - public TableId tableId() { - return FlinkBigQueryConfig.this.getTableId(); - } - - @Override - public Optional query() { - return FlinkBigQueryConfig.this.getQuery(); - } - - @Override - public boolean viewsEnabled() { - return FlinkBigQueryConfig.this.isViewsEnabled(); - } - - @Override - public String viewEnabledParamName() { - return FlinkBigQueryConfig.VIEWS_ENABLED_OPTION; - } - - @Override - public int expirationTimeInMinutes() { - return FlinkBigQueryConfig.this.getMaterializationExpirationTimeInMinutes(); - } - }; - } - - static class SqlParser { - List tableList = new ArrayList(); - List selectCols = new ArrayList(); - - SqlParser(com.google.common.base.Optional query) throws JSQLParserException { - String sql = query.get(); - Statement select = (Statement) CCJSqlParserUtil.parse(sql); - TablesNamesFinder tablesNamesFinder = new TablesNamesFinder(); - this.tableList = tablesNamesFinder.getTableList(select); - this.selectCols = ((PlainSelect) ((Select) select).getSelectBody()).getSelectItems(); - } - - String getSelectedFields() { - String selectedFields = ""; - for (SelectItem field : selectCols) { - selectedFields = - selectedFields - + field.toString().substring(field.toString().lastIndexOf(" ") + 1) - + ","; - } - if (selectedFields.endsWith(",")) { - selectedFields = selectedFields.substring(0, selectedFields.length() - 1); - } - return selectedFields; - } - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/FlinkBigQueryException.java b/src/main/java/com/google/cloud/flink/bigquery/FlinkBigQueryException.java deleted file mode 100644 index 1e0fba6a..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/FlinkBigQueryException.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery; - -public class FlinkBigQueryException extends RuntimeException { - - public FlinkBigQueryException(String message, Throwable error) { - super(message, error); - } - - public FlinkBigQueryException(String message) { - super(message); - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/FlinkBigQueryProxyAndHttpConfig.java b/src/main/java/com/google/cloud/flink/bigquery/FlinkBigQueryProxyAndHttpConfig.java deleted file mode 100644 index e1b6a54e..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/FlinkBigQueryProxyAndHttpConfig.java +++ /dev/null @@ -1,241 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery; - -import static com.google.cloud.bigquery.connector.common.BigQueryUtil.firstPresent; -import static com.google.common.base.Optional.fromJavaUtil; -import static com.google.common.base.Optional.fromNullable; -import static com.google.common.base.Preconditions.checkArgument; - -import com.google.cloud.bigquery.connector.common.BigQueryProxyConfig; -import com.google.cloud.bigquery.connector.common.BigQueryProxyTransporterBuilder; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Objects; -import com.google.common.base.Strings; -import com.google.common.collect.ImmutableMap; -import java.io.Serializable; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.Map; -import java.util.Optional; -import org.apache.hadoop.conf.Configuration; - -public class FlinkBigQueryProxyAndHttpConfig implements BigQueryProxyConfig, Serializable { - - private static final long serialVersionUID = 1L; - - public static final String PROXY_ADDRESS_OPTION = "proxyAddress"; - public static final String PROXY_USERNAME_OPTION = "proxyUsername"; - public static final String PROXY_PASSWORD_OPTION = "proxyPassword"; - public static final String HTTP_MAX_RETRY_OPTION = "httpMaxRetry"; - public static final String HTTP_CONNECT_TIMEOUT_OPTION = "httpConnectTimeout"; - public static final String HTTP_READ_TIMEOUT_OPTION = "httpReadTimeout"; - - // HTTP proxy with address in host:port format - public static final String GCS_CONFIG_PROXY_ADDRESS_PROPERTY = "fs.gs.proxy.address"; - public static final String GCS_CONFIG_PROXY_USERNAME_PROPERTY = "fs.gs.proxy.username"; - public static final String GCS_CONFIG_PROXY_PASSWORD_PROPERTY = "fs.gs.proxy.password"; - public static final String GCS_CONFIG_HTTP_MAX_RETRY_PROPERTY = "fs.gs.http.max.retry"; - public static final String GCS_CONFIG_HTTP_CONNECT_TIMEOUT_PROPERTY = - "fs.gs.http.connect-timeout"; - public static final String GCS_CONFIG_HTTP_READ_TIMEOUT_PROPERTY = "fs.gs.http.read-timeout"; - - private com.google.common.base.Optional proxyUri; - private com.google.common.base.Optional proxyUsername; - private com.google.common.base.Optional proxyPassword; - private com.google.common.base.Optional httpMaxRetry; - private com.google.common.base.Optional httpConnectTimeout; - private com.google.common.base.Optional httpReadTimeout; - - @VisibleForTesting - FlinkBigQueryProxyAndHttpConfig() {} - - @VisibleForTesting - public static FlinkBigQueryProxyAndHttpConfig from( - Map options, - ImmutableMap globalOptions, - Configuration hadoopConfiguration) - throws IllegalArgumentException { - FlinkBigQueryProxyAndHttpConfig config = new FlinkBigQueryProxyAndHttpConfig(); - - com.google.common.base.Optional proxyAddress = - getProperty( - options, - globalOptions, - hadoopConfiguration, - PROXY_ADDRESS_OPTION, - GCS_CONFIG_PROXY_ADDRESS_PROPERTY); - config.proxyUri = fromNullable(parseProxyAddress(proxyAddress.or(""))); - config.proxyUsername = - getProperty( - options, - globalOptions, - hadoopConfiguration, - PROXY_USERNAME_OPTION, - GCS_CONFIG_PROXY_USERNAME_PROPERTY); - config.proxyPassword = - getProperty( - options, - globalOptions, - hadoopConfiguration, - PROXY_PASSWORD_OPTION, - GCS_CONFIG_PROXY_PASSWORD_PROPERTY); - checkProxyParamsValidity(config); - - config.httpMaxRetry = - getProperty( - options, - globalOptions, - hadoopConfiguration, - HTTP_MAX_RETRY_OPTION, - GCS_CONFIG_HTTP_MAX_RETRY_PROPERTY) - .transform(Integer::valueOf); - config.httpConnectTimeout = - getProperty( - options, - globalOptions, - hadoopConfiguration, - HTTP_CONNECT_TIMEOUT_OPTION, - GCS_CONFIG_HTTP_CONNECT_TIMEOUT_PROPERTY) - .transform(Integer::valueOf); - config.httpReadTimeout = - getProperty( - options, - globalOptions, - hadoopConfiguration, - HTTP_READ_TIMEOUT_OPTION, - GCS_CONFIG_HTTP_READ_TIMEOUT_PROPERTY) - .transform(Integer::valueOf); - checkHttpParamsValidity(config); - - return config; - } - - private static void checkProxyParamsValidity(FlinkBigQueryProxyAndHttpConfig config) - throws IllegalArgumentException { - if (!config.proxyUri.isPresent() - && (config.proxyUsername.isPresent() || config.proxyPassword.isPresent())) { - throw new IllegalArgumentException( - "Please set proxyAddress in order to use a proxy. " - + "Setting proxyUsername or proxyPassword is not enough"); - } - - BigQueryProxyTransporterBuilder.checkProxyParamsValidity( - config.getProxyUsername(), config.getProxyPassword()); - } - - private static void checkHttpParamsValidity(FlinkBigQueryProxyAndHttpConfig config) - throws IllegalArgumentException { - - if (config.getHttpMaxRetry().isPresent() && config.getHttpMaxRetry().get() < 0) { - throw new IllegalArgumentException("Http Max Retry value cannot be negative"); - } - } - - private static com.google.common.base.Optional getProperty( - Map options, - ImmutableMap globalOptions, - Configuration hadoopConfiguration, - String bqOption, - String gcsProperty) { - return fromJavaUtil( - firstPresent( - getFirstOrSecondOption(options, globalOptions, bqOption).toJavaUtil(), - fromNullable(hadoopConfiguration.get(gcsProperty)).toJavaUtil())); - } - - private static com.google.common.base.Optional getFirstOrSecondOption( - Map options, ImmutableMap globalOptions, String name) { - return com.google.common.base.Optional.fromNullable(options.get(name)) - .or(com.google.common.base.Optional.fromNullable(globalOptions.get(name))); - } - - @VisibleForTesting - public static URI parseProxyAddress(String proxyAddress) { - if (Strings.isNullOrEmpty(proxyAddress)) { - return null; - } - String uriString = (proxyAddress.contains("//") ? "" : "//") + proxyAddress; - try { - URI uri = new URI(uriString); - String scheme = uri.getScheme(); - String host = uri.getHost(); - int port = uri.getPort(); - checkArgument( - Strings.isNullOrEmpty(scheme) || scheme.matches("https?"), - "Proxy address '%s' has invalid scheme '%s'.", - proxyAddress, - scheme); - checkArgument(!Strings.isNullOrEmpty(host), "Proxy address '%s' has no host.", proxyAddress); - checkArgument(port != -1, "Proxy address '%s' has no port.", proxyAddress); - checkArgument( - uri.equals(new URI(scheme, null, host, port, null, null, null)), - "Invalid proxy address '%s'.", - proxyAddress); - return uri; - } catch (URISyntaxException e) { - throw new IllegalArgumentException( - String.format("Invalid proxy address '%s'.", proxyAddress), e); - } - } - - public Optional getProxyUri() { - return proxyUri.toJavaUtil(); - } - - public Optional getProxyUsername() { - return proxyUsername.toJavaUtil(); - } - - public Optional getProxyPassword() { - return proxyPassword.toJavaUtil(); - } - - public Optional getHttpMaxRetry() { - return httpMaxRetry.toJavaUtil(); - } - - public Optional getHttpConnectTimeout() { - return httpConnectTimeout.toJavaUtil(); - } - - public Optional getHttpReadTimeout() { - return httpReadTimeout.toJavaUtil(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof FlinkBigQueryProxyAndHttpConfig)) { - return false; - } - FlinkBigQueryProxyAndHttpConfig that = (FlinkBigQueryProxyAndHttpConfig) o; - return Objects.equal(proxyUri, that.proxyUri) - && Objects.equal(proxyUsername, that.proxyUsername) - && Objects.equal(proxyPassword, that.proxyPassword) - && Objects.equal(httpMaxRetry, that.httpMaxRetry) - && Objects.equal(httpConnectTimeout, that.httpConnectTimeout) - && Objects.equal(httpReadTimeout, that.httpReadTimeout); - } - - @Override - public int hashCode() { - return Objects.hashCode( - proxyUri, proxyUsername, proxyPassword, httpMaxRetry, httpConnectTimeout, httpReadTimeout); - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/arrow/util/ArrowSchemaConverter.java b/src/main/java/com/google/cloud/flink/bigquery/arrow/util/ArrowSchemaConverter.java deleted file mode 100644 index 2ab4f25e..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/arrow/util/ArrowSchemaConverter.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.arrow.util; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.arrow.vector.types.pojo.Field; -import org.apache.arrow.vector.types.pojo.FieldType; -import org.apache.arrow.vector.types.pojo.Schema; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; - -/** - * Converts an Arrow schema into Flink's type information. It uses {@link RowTypeInfo} for - * representing objects and converts Arrow types into types that are compatible with Flink's Table & - * SQL API. - * - *

Note: Changes in this class need to be kept in sync with the corresponding runtime classes - * {@link ArrowRowDataDeserializationSchema}. - */ -public class ArrowSchemaConverter { - - private ArrowSchemaConverter() {} - - public static Schema convertToSchema(RowType rowType) { - Collection fields = - rowType.getFields().stream() - .map(f -> convertToSchema(f.getName(), f.getType())) - .collect(Collectors.toCollection(ArrayList::new)); - return new Schema(fields); - } - - public static Field convertToSchema(String fieldName, LogicalType logicalType) { - FieldType fieldType = - new FieldType( - logicalType.isNullable(), - logicalType.accept(LogicalTypeToArrowTypeConverter.INSTANCE), - null); - List children = null; - if (logicalType instanceof ArrayType) { - children = - Collections.singletonList( - convertToSchema("element", ((ArrayType) logicalType).getElementType())); - } else if (logicalType instanceof RowType) { - RowType rowType = (RowType) logicalType; - children = new ArrayList<>(rowType.getFieldCount()); - for (RowType.RowField field : rowType.getFields()) { - children.add(convertToSchema(field.getName(), field.getType())); - } - } - return new Field(fieldName, fieldType, children); - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/arrow/util/ArrowToRowDataConverters.java b/src/main/java/com/google/cloud/flink/bigquery/arrow/util/ArrowToRowDataConverters.java deleted file mode 100644 index 36c2ba2e..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/arrow/util/ArrowToRowDataConverters.java +++ /dev/null @@ -1,328 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.arrow.util; - -import java.io.Serializable; -import java.lang.reflect.Array; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.ZoneId; -import java.time.temporal.ChronoField; -import java.util.ArrayList; -import java.util.List; -import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.arrow.vector.util.JsonStringHashMap; -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.utils.LogicalTypeUtils; - -/** Tool class used to convert from Arrow {@link GenericRecord} to {@link RowData}. * */ -@Internal -public class ArrowToRowDataConverters { - - /** - * Runtime converter that converts Arrow data structures into objects of Flink Table & SQL - * internal data structures. - */ - @FunctionalInterface - public interface ArrowToRowDataConverter extends Serializable { - Object convert(Object object); - } - - // ------------------------------------------------------------------------------------- - // Runtime Converters - // ------------------------------------------------------------------------------------- - - public static ArrowToRowDataConverter createRowConverter( - RowType rowType, List readSessionFieldNames) { - final ArrowToRowDataConverter[] fieldConverters = - rowType.getFields().stream() - .map(RowType.RowField::getType) - .map(ArrowToRowDataConverters::createNullableConverter) - .toArray(ArrowToRowDataConverter[]::new); - final int arity = rowType.getFieldCount(); - List fieldNameList = rowType.getFieldNames(); - final List arrowFields = new ArrayList(); - - return arrowObject -> { - return getArrowObject(fieldConverters, arity, fieldNameList, arrowFields, arrowObject); - }; - } - - private static Object getArrowObject( - final ArrowToRowDataConverter[] fieldConverters, - final int arity, - final List fieldNameList, - final List arrowFields, - Object arrowObject) { - List rowdatalist = new ArrayList(); - if (arrowObject instanceof VectorSchemaRoot) { - VectorSchemaRoot record = (VectorSchemaRoot) arrowObject; - record.getSchema().getFields().stream() - .forEach( - field -> { - arrowFields.add(field.getName()); - }); - int numOfRows = record.getRowCount(); - for (int row = 0; row < numOfRows; ++row) { - GenericRowData genericRowData = new GenericRowData(arity); - for (int col = 0; col < arity; col++) { - String rowTypeField = fieldNameList.get(col); - int arrowFieldIdx = arrowFields.indexOf(rowTypeField); - genericRowData.setField( - col, - fieldConverters[col].convert( - record.getFieldVectors().get(arrowFieldIdx).getObject(row))); - } - rowdatalist.add(genericRowData); - } - } else if (arrowObject instanceof JsonStringHashMap) { - JsonStringHashMap record = (JsonStringHashMap) arrowObject; - ArrayList columnSet = new ArrayList(record.keySet()); - GenericRowData genericRowData = new GenericRowData(arity); - if (!columnSet.isEmpty()) { - for (int col = 0; col < arity; col++) { - String actualField = fieldNameList.get(col); - if (columnSet.contains(actualField)) { - int actualIndex = columnSet.indexOf(actualField); - genericRowData.setField( - col, fieldConverters[col].convert(record.get(columnSet.get(actualIndex)))); - } else { - genericRowData.setField(col, fieldConverters[col].convert(null)); - } - } - } - return genericRowData; - } - return rowdatalist; - } - - /** Creates a runtime converter which is null safe. */ - private static ArrowToRowDataConverter createNullableConverter(LogicalType type) { - final ArrowToRowDataConverter converter = createConverter(type); - return arrowObject -> { - if (arrowObject == null) { - return null; - } - return converter.convert(arrowObject); - }; - } - - /** Creates a runtime converter which assuming input object is not null. */ - private static ArrowToRowDataConverter createConverter(LogicalType type) { - ArrowToRowDataConverter value; - switch (type.getTypeRoot()) { - case NULL: - return arrowObject -> null; - case TINYINT: - return arrowObject -> arrowObject; - case SMALLINT: - return arrowObject -> ((Integer) arrowObject).shortValue(); - case DATE: - return ArrowToRowDataConverters::convertToDate; - case TIME_WITHOUT_TIME_ZONE: - return ArrowToRowDataConverters::convertToTime; - case TIMESTAMP_WITHOUT_TIME_ZONE: - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return ArrowToRowDataConverters::convertToTimestamp; - case BOOLEAN: // boolean - case INTEGER: // int - case INTERVAL_YEAR_MONTH: // long - case BIGINT: // long - case INTERVAL_DAY_TIME: // long - case FLOAT: // float - case DOUBLE: // double - return arrowObject -> arrowObject; - case CHAR: - case VARCHAR: - return arrowObject -> StringData.fromString(arrowObject.toString()); - case BINARY: - case VARBINARY: - return ArrowToRowDataConverters::convertToBytes; - case DECIMAL: - value = createDecimalConverter((DecimalType) type); - return value; - case ARRAY: - return createArrayConverter((ArrayType) type); - case ROW: - return createRowConverter((RowType) type, ((RowType) type).getFieldNames()); - case MAP: - case RAW: - default: - throw new UnsupportedOperationException("Unsupported type: " + type); - } - } - - private static TimestampData convertToTimestamp(Object object) { - final long millis; - if (object instanceof Long) { - millis = ((Long) object) / 1000; - } else if (object instanceof Instant) { - millis = ((Instant) object).toEpochMilli(); - } else { - JavaUtilConverter javaUtilConverter = JavaUtilConverter.getConverter(); - if (javaUtilConverter != null) { - millis = javaUtilConverter.convertTimestamp(object); - } else { - throw new IllegalArgumentException( - "Unexpected object type for TIMESTAMP logical type. Received: " + object); - } - } - return TimestampData.fromEpochMillis(millis); - } - - private static int convertToDate(Object object) { - if (object instanceof Integer) { - return (Integer) object; - } else if (object instanceof LocalDate) { - return (int) ((LocalDate) object).toEpochDay(); - } else { - JavaUtilConverter javaUtilConverter = JavaUtilConverter.getConverter(); - if (javaUtilConverter != null) { - return (int) javaUtilConverter.convertDate(object); - } else { - throw new IllegalArgumentException( - "Unexpected object type for DATE logical type. Received: " + object); - } - } - } - - private static int convertToTime(Object object) { - ZoneId zoneId = ZoneId.of("UTC"); - final int millis; - if (object instanceof Integer) { - int value = ((Integer) object); - millis = (Integer) (Math.abs(value) / 1000); - } else if (object instanceof Long) { - Long value = ((Long) object); - value = (Math.abs(value) / 1000); - LocalDateTime time = LocalDateTime.ofInstant(Instant.ofEpochMilli(value), zoneId); - millis = (int) time.atZone(zoneId).toInstant().toEpochMilli(); - - } else if (object instanceof LocalTime) { - millis = ((LocalTime) object).get(ChronoField.MILLI_OF_DAY); - } else { - JavaUtilConverter javaUtilConverter = JavaUtilConverter.getConverter(); - if (javaUtilConverter != null) { - millis = javaUtilConverter.convertTime(object); - } else { - throw new IllegalArgumentException( - "Unexpected object type for TIME logical type. Received: " + object); - } - } - return millis; - } - - private static ArrowToRowDataConverter createDecimalConverter(DecimalType decimalType) { - final int precision = decimalType.getPrecision(); - final int scale = decimalType.getScale(); - return arrowObject -> { - final byte[] bytes; - if (arrowObject instanceof ByteBuffer) { - ByteBuffer byteBuffer = (ByteBuffer) arrowObject; - bytes = new byte[byteBuffer.remaining()]; - byteBuffer.get(bytes); - } else if (arrowObject instanceof BigDecimal) { - DecimalData bdValue = - DecimalData.fromBigDecimal((BigDecimal) arrowObject, precision, scale); - return bdValue; - } else { - bytes = (byte[]) arrowObject; - } - return DecimalData.fromUnscaledBytes(bytes, precision, scale); - }; - } - - private static ArrowToRowDataConverter createArrayConverter(ArrayType arrayType) { - final ArrowToRowDataConverter elementConverter = - createNullableConverter(arrayType.getElementType()); - final Class elementClass = - LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType()); - - return arrowObject -> { - final List list = (List) arrowObject; - final int length = list.size(); - final Object[] array = (Object[]) Array.newInstance(elementClass, length); - for (int i = 0; i < length; ++i) { - array[i] = elementConverter.convert(list.get(i)); - } - return new GenericArrayData(array); - }; - } - - private static byte[] convertToBytes(Object object) { - if (object instanceof ByteBuffer) { - ByteBuffer byteBuffer = (ByteBuffer) object; - byte[] bytes = new byte[byteBuffer.remaining()]; - byteBuffer.get(bytes); - return bytes; - } else { - return (byte[]) object; - } - } - - private static class JavaUtilConverter { - - private static JavaUtilConverter instance; - private static boolean instantiated = false; - - public static JavaUtilConverter getConverter() { - if (instantiated) { - return instance; - } - try { - Class.forName( - "java.time.LocalDateTime", false, Thread.currentThread().getContextClassLoader()); - instance = new JavaUtilConverter(); - } catch (Exception e) { - instance = null; - } finally { - instantiated = true; - } - return instance; - } - - public long convertDate(Object object) { - final java.time.LocalDate localDate = (LocalDate) object; - return localDate.toEpochDay(); - } - - public int convertTime(Object object) { - final java.time.LocalDateTime localDateTime = (LocalDateTime) object; - return (int) localDateTime.atZone(ZoneId.of("UTC")).toInstant().toEpochMilli(); - } - - public long convertTimestamp(Object object) { - final java.time.LocalDateTime localDateTime = (LocalDateTime) object; - return localDateTime.atZone(ZoneId.of("UTC")).toInstant().toEpochMilli(); - } - - private JavaUtilConverter() {} - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/arrow/util/LogicalTypeToArrowTypeConverter.java b/src/main/java/com/google/cloud/flink/bigquery/arrow/util/LogicalTypeToArrowTypeConverter.java deleted file mode 100644 index 1e97c501..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/arrow/util/LogicalTypeToArrowTypeConverter.java +++ /dev/null @@ -1,168 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.arrow.util; - -import java.math.BigDecimal; -import org.apache.arrow.vector.types.DateUnit; -import org.apache.arrow.vector.types.FloatingPointPrecision; -import org.apache.arrow.vector.types.TimeUnit; -import org.apache.arrow.vector.types.pojo.ArrowType; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.BigIntType; -import org.apache.flink.table.types.logical.BooleanType; -import org.apache.flink.table.types.logical.DateType; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.DoubleType; -import org.apache.flink.table.types.logical.FloatType; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LegacyTypeInformationType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.SmallIntType; -import org.apache.flink.table.types.logical.TimeType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.TinyIntType; -import org.apache.flink.table.types.logical.VarBinaryType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; - -public class LogicalTypeToArrowTypeConverter extends LogicalTypeDefaultVisitor { - - @Override - protected ArrowType defaultMethod(LogicalType logicalType) { - if (logicalType instanceof LegacyTypeInformationType) { - Class typeClass = - ((LegacyTypeInformationType) logicalType).getTypeInformation().getTypeClass(); - if (typeClass == BigDecimal.class) { - return new ArrowType.Decimal(38, 18); - } - } - throw new UnsupportedOperationException( - String.format( - "Python vectorized UDF doesn't support logical type %s currently.", - logicalType.asSummaryString())); - } - - public static final LogicalTypeToArrowTypeConverter INSTANCE = - new LogicalTypeToArrowTypeConverter(); - - @Override - public ArrowType visit(TinyIntType tinyIntType) { - return new ArrowType.Int(8, true); - } - - @Override - public ArrowType visit(SmallIntType smallIntType) { - return new ArrowType.Int(2 * 8, true); - } - - @Override - public ArrowType visit(IntType intType) { - return new ArrowType.Int(4 * 8, true); - } - - @Override - public ArrowType visit(BigIntType bigIntType) { - return new ArrowType.Int(8 * 8, true); - } - - @Override - public ArrowType visit(BooleanType booleanType) { - return ArrowType.Bool.INSTANCE; - } - - @Override - public ArrowType visit(FloatType floatType) { - return new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE); - } - - @Override - public ArrowType visit(DoubleType doubleType) { - return new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE); - } - - @Override - public ArrowType visit(VarCharType varCharType) { - return ArrowType.Utf8.INSTANCE; - } - - @Override - public ArrowType visit(VarBinaryType varCharType) { - return ArrowType.Binary.INSTANCE; - } - - @Override - public ArrowType visit(DecimalType decimalType) { - return new ArrowType.Decimal(decimalType.getPrecision(), decimalType.getScale()); - } - - @Override - public ArrowType visit(DateType dateType) { - return new ArrowType.Date(DateUnit.DAY); - } - - @Override - public ArrowType visit(TimeType timeType) { - if (timeType.getPrecision() == 0) { - return new ArrowType.Time(TimeUnit.SECOND, 32); - } else if (timeType.getPrecision() >= 1 && timeType.getPrecision() <= 3) { - return new ArrowType.Time(TimeUnit.MILLISECOND, 32); - } else if (timeType.getPrecision() >= 4 && timeType.getPrecision() <= 6) { - return new ArrowType.Time(TimeUnit.MICROSECOND, 64); - } else { - return new ArrowType.Time(TimeUnit.NANOSECOND, 64); - } - } - - @Override - public ArrowType visit(LocalZonedTimestampType localZonedTimestampType) { - if (localZonedTimestampType.getPrecision() == 0) { - return new ArrowType.Timestamp(TimeUnit.SECOND, null); - } else if (localZonedTimestampType.getPrecision() >= 1 - && localZonedTimestampType.getPrecision() <= 3) { - return new ArrowType.Timestamp(TimeUnit.MILLISECOND, null); - } else if (localZonedTimestampType.getPrecision() >= 4 - && localZonedTimestampType.getPrecision() <= 6) { - return new ArrowType.Timestamp(TimeUnit.MICROSECOND, null); - } else { - return new ArrowType.Timestamp(TimeUnit.NANOSECOND, null); - } - } - - @Override - public ArrowType visit(TimestampType timestampType) { - if (timestampType.getPrecision() == 0) { - return new ArrowType.Timestamp(TimeUnit.SECOND, null); - } else if (timestampType.getPrecision() >= 1 && timestampType.getPrecision() <= 3) { - return new ArrowType.Timestamp(TimeUnit.MILLISECOND, null); - } else if (timestampType.getPrecision() >= 4 && timestampType.getPrecision() <= 6) { - return new ArrowType.Timestamp(TimeUnit.MICROSECOND, null); - } else { - return new ArrowType.Timestamp(TimeUnit.NANOSECOND, null); - } - } - - @Override - public ArrowType visit(ArrayType arrayType) { - return ArrowType.List.INSTANCE; - } - - @Override - public ArrowType visit(RowType rowType) { - return ArrowType.Struct.INSTANCE; - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/common/BigQueryDirectWriterCommitMessageContext.java b/src/main/java/com/google/cloud/flink/bigquery/common/BigQueryDirectWriterCommitMessageContext.java deleted file mode 100644 index b15be2d1..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/common/BigQueryDirectWriterCommitMessageContext.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Copyright 2018 Google Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.common; - -public class BigQueryDirectWriterCommitMessageContext implements WriterCommitMessageContext { - - private static final long serialVersionUID = -1562914502592461805L; - private final String writeStreamName; - - private final String tablePath; - private final long rowCount; - - public BigQueryDirectWriterCommitMessageContext( - String writeStreamName, String tablePath, long rowCount) { - this.writeStreamName = writeStreamName; - this.tablePath = tablePath; - this.rowCount = rowCount; - } - - public String getWriteStreamName() { - return writeStreamName; - } - - public String getTablePath() { - return tablePath; - } - - public long getRowCount() { - return rowCount; - } - - @Override - public String toString() { - return "BigQueryWriterCommitMessage{tableId='" + tablePath + '\'' + '}'; - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/common/FlinkBigQueryConnectorUserAgentProvider.java b/src/main/java/com/google/cloud/flink/bigquery/common/FlinkBigQueryConnectorUserAgentProvider.java deleted file mode 100644 index c29c2ac3..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/common/FlinkBigQueryConnectorUserAgentProvider.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * Copyright 2018 Google Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.common; - -import static java.lang.String.format; -import static java.nio.charset.StandardCharsets.UTF_8; - -import com.google.cloud.bigquery.connector.common.UserAgentProvider; -import com.google.cloud.flink.bigquery.FlinkBigQueryException; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.io.CharStreams; -import com.google.common.io.Closeables; -import java.io.IOException; -import java.io.InputStreamReader; -import java.util.Optional; -import org.apache.http.client.config.RequestConfig; -import org.apache.http.client.methods.CloseableHttpResponse; -import org.apache.http.client.methods.HttpGet; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.impl.client.HttpClients; -import scala.util.Properties; - -/** Provides the versions of the client environment in an anonymous way. */ -public class FlinkBigQueryConnectorUserAgentProvider implements UserAgentProvider { - - @VisibleForTesting - static String GCP_REGION_PART = getGcpRegion().map(region -> " region/" + region).orElse(""); - - @VisibleForTesting - static String DATAPROC_IMAGE_PART = - Optional.ofNullable(System.getenv("DATAPROC_IMAGE_VERSION")) - .map(image -> " dataproc-image/" + image) - .orElse(""); - - private static String FLINK_VERSION; - private static String JAVA_VERSION = System.getProperty("java.runtime.version"); - private static String SCALA_VERSION = Properties.versionNumberString(); - static final String USER_AGENT = - format( - "flink-bigquery-connector/%s flink/%s java/%s scala/%s%s%s", - FlinkBigQueryUtil.CONNECTOR_VERSION, - FLINK_VERSION, - JAVA_VERSION, - SCALA_VERSION, - GCP_REGION_PART, - DATAPROC_IMAGE_PART); - - public FlinkBigQueryConnectorUserAgentProvider(String flinkVersion) { - FLINK_VERSION = flinkVersion; - } - - // Queries the GCE metadata server - @VisibleForTesting - static Optional getGcpRegion() { - RequestConfig config = - RequestConfig.custom() - .setConnectTimeout(100) - .setConnectionRequestTimeout(100) - .setSocketTimeout(100) - .build(); - CloseableHttpClient httpClient = HttpClients.custom().setDefaultRequestConfig(config).build(); - HttpGet httpGet = - new HttpGet("http://metadata.google.internal/computeMetadata/v1/instance/zone"); - httpGet.addHeader("Metadata-Flavor", "Google"); - try (CloseableHttpResponse response = httpClient.execute(httpGet)) { - if (response.getStatusLine().getStatusCode() == 200) { - String body = - CharStreams.toString(new InputStreamReader(response.getEntity().getContent(), UTF_8)); - return Optional.of(body.substring(body.lastIndexOf('/') + 1)); - } else { - return Optional.empty(); - } - } catch (Exception e) { - return Optional.empty(); - } finally { - try { - Closeables.close(httpClient, true); - } catch (IOException e) { - throw new FlinkBigQueryException("Error while closing the http client connection"); - } - } - } - - @Override - public String getUserAgent() { - return USER_AGENT; - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/common/FlinkBigQueryUtil.java b/src/main/java/com/google/cloud/flink/bigquery/common/FlinkBigQueryUtil.java deleted file mode 100644 index 00e85139..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/common/FlinkBigQueryUtil.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Copyright 2020 Google Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.common; - -import com.google.cloud.flink.bigquery.FlinkBigQueryException; -import java.io.IOException; -import java.util.Properties; - -/** Flink related utilities */ -public class FlinkBigQueryUtil { - - static final Properties BUILD_PROPERTIES = loadBuildProperties(); - - static final String CONNECTOR_VERSION = BUILD_PROPERTIES.getProperty("connector.version"); - - private static Properties loadBuildProperties() { - try { - Properties buildProperties = new Properties(); - buildProperties.load( - FlinkBigQueryUtil.class.getResourceAsStream("/flink-bigquery-connector.properties")); - return buildProperties; - } catch (IOException e) { - throw new FlinkBigQueryException("Error while loading properties file"); - } - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/common/UserAgentHeaderProvider.java b/src/main/java/com/google/cloud/flink/bigquery/common/UserAgentHeaderProvider.java deleted file mode 100644 index fac18348..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/common/UserAgentHeaderProvider.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.common; - -import com.google.api.gax.rpc.HeaderProvider; -import com.google.common.base.Objects; -import com.google.common.collect.ImmutableMap; -import java.io.Serializable; -import java.util.Map; - -public class UserAgentHeaderProvider implements HeaderProvider, Serializable { - - private static final long serialVersionUID = 1L; - private final String userAgent; - - public UserAgentHeaderProvider(String userAgent) { - this.userAgent = userAgent; - } - - @Override - public Map getHeaders() { - return ImmutableMap.of("user-agent", userAgent); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof UserAgentHeaderProvider)) { - return false; - } - UserAgentHeaderProvider that = (UserAgentHeaderProvider) o; - return Objects.equal(userAgent, that.userAgent); - } - - @Override - public int hashCode() { - return Objects.hashCode(userAgent); - } -} diff --git a/src/main/java/com/google/cloud/flink/bigquery/common/WriterCommitMessageContext.java b/src/main/java/com/google/cloud/flink/bigquery/common/WriterCommitMessageContext.java deleted file mode 100644 index fcd1e94a..00000000 --- a/src/main/java/com/google/cloud/flink/bigquery/common/WriterCommitMessageContext.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * Copyright 2021 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * https://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.common; - -import java.io.Serializable; - -public interface WriterCommitMessageContext extends Serializable {} diff --git a/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory deleted file mode 100644 index 7b271983..00000000 --- a/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ /dev/null @@ -1,2 +0,0 @@ -com.google.cloud.flink.bigquery.BigQueryDynamicTableFactory -com.google.cloud.flink.bigquery.ArrowFormatFactory diff --git a/src/main/resources/flink-bigquery-connector.properties b/src/main/resources/flink-bigquery-connector.properties deleted file mode 100644 index 58158550..00000000 --- a/src/main/resources/flink-bigquery-connector.properties +++ /dev/null @@ -1 +0,0 @@ -connector.version=${project.version} diff --git a/src/test/java/com/google/cloud/flink/bigquery/BigQueryDirectWriterCommitMessageContextTest.java b/src/test/java/com/google/cloud/flink/bigquery/BigQueryDirectWriterCommitMessageContextTest.java deleted file mode 100644 index f0d2d8b8..00000000 --- a/src/test/java/com/google/cloud/flink/bigquery/BigQueryDirectWriterCommitMessageContextTest.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery; - -import static com.google.common.truth.Truth.assertThat; - -import com.google.cloud.flink.bigquery.common.BigQueryDirectWriterCommitMessageContext; -import org.junit.Before; -import org.junit.Test; - -public class BigQueryDirectWriterCommitMessageContextTest { - BigQueryDirectWriterCommitMessageContext messageContext; - - @Before - public void setup() { - messageContext = - new BigQueryDirectWriterCommitMessageContext("Stream0", "project/dataset/table", 100); - } - - @Test - public void getWriteStreamNameTest() { - String streamName = messageContext.getWriteStreamName(); - assertThat(streamName).isNotNull(); - assertThat(streamName).isEqualTo("Stream0"); - } - - @Test - public void getTablePathTest() { - String tablePath = messageContext.getTablePath(); - assertThat(tablePath).isNotNull(); - assertThat(tablePath).isEqualTo("project/dataset/table"); - } - - @Test - public void getRowCountTest() { - Long rowCount = messageContext.getRowCount(); - assertThat(rowCount).isNotNull(); - assertThat(rowCount).isEqualTo(100); - } - - @Test - public void toStringTest() { - String messagecontextString = messageContext.toString(); - assertThat(messagecontextString).isNotNull(); - assertThat(messagecontextString) - .isEqualTo("BigQueryWriterCommitMessage{tableId='project/dataset/table'}"); - } -} diff --git a/src/test/java/com/google/cloud/flink/bigquery/FlinkBigQueryProxyAndHttpConfigTest.java b/src/test/java/com/google/cloud/flink/bigquery/FlinkBigQueryProxyAndHttpConfigTest.java deleted file mode 100644 index 5a52135d..00000000 --- a/src/test/java/com/google/cloud/flink/bigquery/FlinkBigQueryProxyAndHttpConfigTest.java +++ /dev/null @@ -1,341 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery; - -import static com.google.common.truth.Truth.assertThat; -import static org.junit.Assert.assertThrows; - -import com.google.api.client.http.apache.v2.ApacheHttpTransport; -import com.google.api.core.ApiFunction; -import com.google.auth.http.HttpTransportFactory; -import com.google.cloud.bigquery.connector.common.BigQueryProxyTransporterBuilder; -import com.google.common.collect.ImmutableMap; -import io.grpc.ManagedChannelBuilder; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectOutputStream; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.hadoop.conf.Configuration; -import org.junit.Test; - -public class FlinkBigQueryProxyAndHttpConfigTest { - - private final ImmutableMap defaultOptions = - ImmutableMap.builder() - .put("proxyAddress", "http://bq-connector-host:1234") - .put("proxyUsername", "bq-connector-user") - .put("proxyPassword", "bq-connector-password") - .put("httpMaxRetry", "10") - .put("httpConnectTimeout", "10000") - .put("httpReadTimeout", "20000") - .build(); - - private final ImmutableMap defaultGlobalOptions = - ImmutableMap.builder() - .put("flink.datasource.bigquery.proxyAddress", "http://bq-connector-host-global:1234") - .put("flink.datasource.bigquery.proxyUsername", "bq-connector-user-global") - .put("flink.datasource.bigquery.proxyPassword", "bq-connector-password-global") - .put("flink.datasource.bigquery.httpMaxRetry", "20") - .put("flink.datasource.bigquery.httpConnectTimeout", "20000") - .put("flink.datasource.bigquery.httpReadTimeout", "30000") - .build(); - - private final Configuration defaultHadoopConfiguration = getHadoopConfiguration(); - - private Configuration getHadoopConfiguration() { - Configuration hadoopConfiguration = new Configuration(); - hadoopConfiguration.set("fs.gs.proxy.address", "http://bq-connector-host-hadoop:1234"); - hadoopConfiguration.set("fs.gs.proxy.username", "bq-connector-user-hadoop"); - hadoopConfiguration.set("fs.gs.proxy.password", "bq-connector-password-hadoop"); - hadoopConfiguration.set("fs.gs.http.max.retry", "30"); - hadoopConfiguration.set("fs.gs.http.connect-timeout", "30000"); - hadoopConfiguration.set("fs.gs.http.read-timeout", "40000"); - return hadoopConfiguration; - } - - private static final Optional optionalProxyURI = - Optional.of(URI.create("http://bq-connector-transporter-builder-host:1234")); - private static final Optional optionalProxyUserName = - Optional.of("transporter-builder-user"); - private static final Optional optionalProxyPassword = - Optional.of("transporter-builder-password"); - - @Test - public void testSerializability() - throws IOException { // need to confirm the parameter of from method - org.apache.flink.configuration.Configuration options = - new org.apache.flink.configuration.Configuration(); - ConfigOption table = ConfigOptions.key("table").stringType().noDefaultValue(); - ConfigOption selectedFields = - ConfigOptions.key("selectedFields").stringType().noDefaultValue(); - options.set(table, "bigquery-public-data.samples.shakespeare"); - options.set(selectedFields, "word,word_count"); - new ObjectOutputStream(new ByteArrayOutputStream()) - .writeObject( - FlinkBigQueryProxyAndHttpConfig.from( - defaultOptions, defaultGlobalOptions, defaultHadoopConfiguration)); - } - - @Test - public void testConfigFromOptions() throws URISyntaxException { - Configuration emptyHadoopConfiguration = new Configuration(); - - FlinkBigQueryProxyAndHttpConfig config = - FlinkBigQueryProxyAndHttpConfig.from( - defaultOptions, - ImmutableMap.of(), // empty - // globalOptions - emptyHadoopConfiguration); - - assertThat(config.getProxyUri()) - .isEqualTo(Optional.of(getURI("http", "bq-connector-host", 1234))); - assertThat(config.getProxyUsername()).isEqualTo(Optional.of("bq-connector-user")); - assertThat(config.getProxyPassword()).isEqualTo(Optional.of("bq-connector-password")); - assertThat(config.getHttpMaxRetry()).isEqualTo(Optional.of(10)); - assertThat(config.getHttpConnectTimeout()).isEqualTo(Optional.of(10000)); - assertThat(config.getHttpReadTimeout()).isEqualTo(Optional.of(20000)); - } - - @Test - public void fromTest() throws URISyntaxException { - FlinkBigQueryProxyAndHttpConfig config = - FlinkBigQueryProxyAndHttpConfig.from( - ImmutableMap.of(), // empty - // options - ImmutableMap.of(), // empty global options - defaultHadoopConfiguration); - assertThat(config.getProxyUri()) - .isEqualTo(Optional.of(getURI("http", "bq-connector-host-hadoop", 1234))); - assertThat(config.getProxyUsername()).isEqualTo(Optional.of("bq-connector-user-hadoop")); - assertThat(config.getProxyPassword()).isEqualTo(Optional.of("bq-connector-password-hadoop")); - assertThat(config.getHttpMaxRetry()).isEqualTo(Optional.of(30)); - assertThat(config.getHttpConnectTimeout()).isEqualTo(Optional.of(30000)); - assertThat(config.getHttpReadTimeout()).isEqualTo(Optional.of(40000)); - } - - @Test - public void testConfigFromGlobalOptions() throws URISyntaxException { - Configuration emptyHadoopConfiguration = new Configuration(); - ImmutableMap globalOptions = - FlinkBigQueryConfig.normalizeConf(defaultGlobalOptions); - FlinkBigQueryProxyAndHttpConfig config = - FlinkBigQueryProxyAndHttpConfig.from( - ImmutableMap.of(), // empty option - globalOptions, - emptyHadoopConfiguration); - - assertThat(config.getProxyUri()) - .isNotEqualTo(Optional.of(getURI("http", "bq-connector-host-hadoop", 1234))); - assertThat(config.getProxyUsername()).isNotEqualTo(Optional.of("bq-connector-user-hadoop")); - assertThat(config.getProxyPassword()).isNotEqualTo(Optional.of("bq-connector-password-hadoop")); - assertThat(config.getHttpMaxRetry()).isNotEqualTo(Optional.of(30)); - assertThat(config.getHttpConnectTimeout()).isNotEqualTo(Optional.of(30000)); - assertThat(config.getHttpReadTimeout()).isNotEqualTo(Optional.of(40000)); - } - - @Test - public void testConfigFromHadoopConfigurationOptions() throws URISyntaxException { - FlinkBigQueryProxyAndHttpConfig config = - FlinkBigQueryProxyAndHttpConfig.from( - ImmutableMap.of(), // empty - // options - ImmutableMap.of(), // empty global options - defaultHadoopConfiguration); - - assertThat(config.getProxyUri()) - .isEqualTo(Optional.of(getURI("http", "bq-connector-host-hadoop", 1234))); - assertThat(config.getProxyUsername()).isEqualTo(Optional.of("bq-connector-user-hadoop")); - assertThat(config.getProxyPassword()).isEqualTo(Optional.of("bq-connector-password-hadoop")); - assertThat(config.getHttpMaxRetry()).isEqualTo(Optional.of(30)); - assertThat(config.getHttpConnectTimeout()).isEqualTo(Optional.of(30000)); - assertThat(config.getHttpReadTimeout()).isEqualTo(Optional.of(40000)); - } - - @Test - public void testConfigWithGlobalParametersAndHadoopConfig() throws URISyntaxException { - ImmutableMap globalOptions = - FlinkBigQueryConfig.normalizeConf(defaultGlobalOptions); - FlinkBigQueryProxyAndHttpConfig config = - FlinkBigQueryProxyAndHttpConfig.from( - ImmutableMap.of(), // empty - // options - globalOptions, - defaultHadoopConfiguration); - - assertThat(config.getProxyUri()) - .isEqualTo(Optional.of(getURI("http", "bq-connector-host-global", 1234))); - assertThat(config.getProxyUsername()).isEqualTo(Optional.of("bq-connector-user-global")); - assertThat(config.getProxyPassword()).isEqualTo(Optional.of("bq-connector-password-global")); - assertThat(config.getHttpMaxRetry()).isEqualTo(Optional.of(20)); - assertThat(config.getHttpConnectTimeout()).isEqualTo(Optional.of(20000)); - assertThat(config.getHttpReadTimeout()).isEqualTo(Optional.of(30000)); - } - - @Test - public void testParseProxyAddress() throws Exception { - // map of input string v/s expected return - HashMap inputOutputMap = new HashMap<>(); - inputOutputMap.put("bq-connector-host:1234", getURI(null, "bq-connector-host", 1234)); - inputOutputMap.put("http://bq-connector-host:1234", getURI("http", "bq-connector-host", 1234)); - inputOutputMap.put( - "https://bq-connector-host:1234", getURI("https", "bq-connector-host", 1234)); - - for (Map.Entry entry : inputOutputMap.entrySet()) { - String address = entry.getKey(); - URI expectedUri = entry.getValue(); - URI uri = FlinkBigQueryProxyAndHttpConfig.parseProxyAddress(address); - assertThat(uri).isEqualTo(expectedUri); - } - } - - @Test - public void testParseProxyAddressIllegalPath() { - ArrayList addresses = new ArrayList<>(); - addresses.add("bq-connector-host-with-illegal-char^:1234"); - addresses.add("bq-connector-host:1234/some/path"); - - for (String address : addresses) { - IllegalArgumentException exception = - assertThrows( - IllegalArgumentException.class, - () -> FlinkBigQueryProxyAndHttpConfig.parseProxyAddress(address)); - assertThat(exception) - .hasMessageThat() - .isEqualTo(String.format("Invalid proxy address '%s'.", address)); - } - } - - @Test - public void testParseProxyAddressNoPort() { - ArrayList addresses = new ArrayList<>(); - addresses.add("bq-connector-host"); - addresses.add("http://bq-connector-host"); - addresses.add("https://bq-connector-host"); - - for (String address : addresses) { - IllegalArgumentException exception = - assertThrows( - IllegalArgumentException.class, - () -> FlinkBigQueryProxyAndHttpConfig.parseProxyAddress(address)); - assertThat(exception) - .hasMessageThat() - .isEqualTo(String.format("Proxy address '%s' has no port.", address)); - } - } - - @Test - public void testParseProxyAddressWrongScheme() { - ArrayList addresses = new ArrayList<>(); - addresses.add("socks5://bq-connector-host:1234"); - addresses.add("htt://bq-connector-host:1234"); // a missing p in http - - for (String address : addresses) { - IllegalArgumentException exception = - assertThrows( - IllegalArgumentException.class, - () -> FlinkBigQueryProxyAndHttpConfig.parseProxyAddress(address)); - assertThat(exception) - .hasMessageThat() - .contains(String.format("Proxy address '%s' has invalid scheme", address)); - } - } - - @Test - public void testParseProxyAddressNoHost() { - String address = ":1234"; - - IllegalArgumentException exception = - assertThrows( - IllegalArgumentException.class, - () -> FlinkBigQueryProxyAndHttpConfig.parseProxyAddress(address)); - - assertThat(exception) - .hasMessageThat() - .isEqualTo(String.format("Proxy address '%s' has no host.", address)); - } - - private URI getURI(String scheme, String host, int port) throws URISyntaxException { - return new URI(scheme, null, host, port, null, null, null); - } - - @Test - public void testBigQueryProxyTransporterBuilder() { - ApiFunction apiFunction = - BigQueryProxyTransporterBuilder.createGrpcChannelConfigurator( - optionalProxyURI, optionalProxyUserName, optionalProxyPassword); - - assertThat(apiFunction.apply(ManagedChannelBuilder.forTarget("test-target"))) - .isInstanceOf(ManagedChannelBuilder.class); - - HttpTransportFactory httpTransportFactory = - BigQueryProxyTransporterBuilder.createHttpTransportFactory( - optionalProxyURI, optionalProxyUserName, optionalProxyPassword); - - assertThat(httpTransportFactory.create()).isInstanceOf(ApacheHttpTransport.class); - } - - @Test - public void testBigQueryProxyTransporterBuilderWithErrors() { - IllegalArgumentException exceptionWithPasswordHttp = - assertThrows( - IllegalArgumentException.class, - () -> - BigQueryProxyTransporterBuilder.createHttpTransportFactory( - optionalProxyURI, Optional.empty(), optionalProxyPassword)); - - IllegalArgumentException exceptionWithUserNameHttp = - assertThrows( - IllegalArgumentException.class, - () -> - BigQueryProxyTransporterBuilder.createHttpTransportFactory( - optionalProxyURI, optionalProxyUserName, Optional.empty())); - - IllegalArgumentException exceptionWithPasswordGrpc = - assertThrows( - IllegalArgumentException.class, - () -> - BigQueryProxyTransporterBuilder.createGrpcChannelConfigurator( - optionalProxyURI, Optional.empty(), optionalProxyPassword)); - - IllegalArgumentException exceptionWithUserNameGrpc = - assertThrows( - IllegalArgumentException.class, - () -> - BigQueryProxyTransporterBuilder.createGrpcChannelConfigurator( - optionalProxyURI, optionalProxyUserName, Optional.empty())); - - Arrays.asList( - exceptionWithPasswordHttp, - exceptionWithUserNameHttp, - exceptionWithPasswordGrpc, - exceptionWithUserNameGrpc) - .stream() - .forEach( - exception -> - assertThat(exception) - .hasMessageThat() - .contains( - "Both proxyUsername and proxyPassword should be defined or not defined together")); - } -} diff --git a/src/test/java/com/google/cloud/flink/bigquery/UserAgentHeaderProviderTest.java b/src/test/java/com/google/cloud/flink/bigquery/UserAgentHeaderProviderTest.java deleted file mode 100644 index 98cee27d..00000000 --- a/src/test/java/com/google/cloud/flink/bigquery/UserAgentHeaderProviderTest.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Copyright 2020 Google Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery; - -import static com.google.common.truth.Truth.assertThat; - -import com.google.cloud.flink.bigquery.common.UserAgentHeaderProvider; -import java.util.Map; -import org.junit.Test; - -public class UserAgentHeaderProviderTest { - @Test - public void getHeadersTest() { - UserAgentHeaderProvider userAgentHeaderProvider = - new UserAgentHeaderProvider("test agent header"); - Map resultMap = userAgentHeaderProvider.getHeaders(); - String userAgent = resultMap.get("user-agent"); - assertThat(resultMap).isNotNull(); - assertThat(userAgent).isNotNull(); - assertThat(userAgent).isEqualTo("test agent header"); - } -} diff --git a/src/test/java/com/google/cloud/flink/bigquery/integration/Constants.java b/src/test/java/com/google/cloud/flink/bigquery/integration/Constants.java deleted file mode 100644 index 0ed91d4d..00000000 --- a/src/test/java/com/google/cloud/flink/bigquery/integration/Constants.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.integration; - -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; - -public class Constants { - - static final String ALL_TYPES_TABLE_NAME = "all_types"; - static final String LARGE_TABLE_FIELD = "is_male"; - static final String LARGE_TABLE_PROJECT_ID = "bigquery-public-data"; - static final String LARGE_TABLE_DATASET = "samples"; - static final String LARGE_TABLE = "natality"; - static final TableSchema WORDCOUNT_TABLE_SCHEMA = - new TableSchema.Builder() - .field("word", DataTypes.STRING()) - .field("word_count", DataTypes.BIGINT()) - .build(); - - static final TableSchema FLINK_TEST_TABLE_SCHEMA = - new TableSchema.Builder() - .field("numeric_datatype", DataTypes.DECIMAL(38, 9)) - .field("string_datatype", DataTypes.STRING()) - .field("bytes_datatype", DataTypes.BYTES()) - .field("integer_datatype", DataTypes.BIGINT()) - .field("float_datatype", DataTypes.DOUBLE()) - .field("boolean_datatype", DataTypes.BOOLEAN()) - .field("timestamp_datatype", DataTypes.TIMESTAMP()) - .field("date_datatype", DataTypes.DATE()) - .field("datetime_datatype", DataTypes.TIMESTAMP()) - .field("geography_datatype", DataTypes.STRING()) - .build(); - - static final TableSchema LARGE_TABLE_SCHEMA = - new TableSchema.Builder().field("is_male", DataTypes.BOOLEAN()).build(); -} diff --git a/src/test/java/com/google/cloud/flink/bigquery/integration/FlinkBigQueryIntegrationTestBase.java b/src/test/java/com/google/cloud/flink/bigquery/integration/FlinkBigQueryIntegrationTestBase.java deleted file mode 100644 index 77ac206b..00000000 --- a/src/test/java/com/google/cloud/flink/bigquery/integration/FlinkBigQueryIntegrationTestBase.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.integration; - -import com.google.cloud.bigquery.BigQueryOptions; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.rules.ExternalResource; - -public class FlinkBigQueryIntegrationTestBase { - - @ClassRule public static TestDataset testDataset = new TestDataset(); - - protected String testTable; - public static StreamTableEnvironment flinkTableEnv; - - @Before - public void createTestTable() { - testTable = "test_" + System.nanoTime(); - } - - public FlinkBigQueryIntegrationTestBase() { - - BigQueryOptions.getDefaultInstance().getService(); - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - FlinkBigQueryIntegrationTestBase.flinkTableEnv = StreamTableEnvironment.create(env); - } - - protected static class TestDataset extends ExternalResource { - - String testDataset = - String.format("flink_bigquery_%d_%d", System.currentTimeMillis(), System.nanoTime()); - - @Override - protected void before() throws Throwable { - IntegrationTestUtils.createDataset(testDataset); - IntegrationTestUtils.createTable(testDataset, Constants.ALL_TYPES_TABLE_NAME, "read"); - } - - @Override - protected void after() { - IntegrationTestUtils.deleteDatasetAndTables(testDataset); - } - - @Override - public String toString() { - return testDataset; - } - } -} diff --git a/src/test/java/com/google/cloud/flink/bigquery/integration/FlinkReadByFormatIntegrationTest.java b/src/test/java/com/google/cloud/flink/bigquery/integration/FlinkReadByFormatIntegrationTest.java deleted file mode 100644 index ca79368d..00000000 --- a/src/test/java/com/google/cloud/flink/bigquery/integration/FlinkReadByFormatIntegrationTest.java +++ /dev/null @@ -1,151 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.integration; - -import static com.google.common.truth.Truth.assertThat; -import static org.apache.flink.table.api.Expressions.$; - -import java.util.Optional; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.junit.Test; - -public class FlinkReadByFormatIntegrationTest extends FlinkReadIntegrationTest { - - protected String dataFormat; - - public FlinkReadByFormatIntegrationTest() { - super(); - } - - @Test - public void testOutOfOrderColumns() throws Exception { - - String bigqueryReadTable = "bigquery-public-data.samples.shakespeare"; - String flinkSrcTable = "FlinkSrcTable"; - String srcQueryString = "CREATE TABLE " + flinkSrcTable + " (word STRING , word_count BIGINT)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'table' = '" - + bigqueryReadTable - + "',\n" - + " 'filter' = 'word_count > 500',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "' ,\n" - + " 'selectedFields' = 'word,word_count' \n" - + ")"); - Table table1 = flinkTableEnv.from(flinkSrcTable); - Table result = table1.select($("word_count"), $("word")); - int count = 0; - TableResult tableResult = result.execute(); - try (CloseableIterator it = tableResult.collect()) { - while (it.hasNext()) { - it.next(); - count += 1; - } - } - assertThat(count).isEqualTo(96); - assertThat(result.getSchema().getFieldDataType(0)).isEqualTo(Optional.of(DataTypes.BIGINT())); - assertThat(result.getSchema().getFieldDataType(1)).isEqualTo(Optional.of(DataTypes.STRING())); - } - - @Test - public void testDefaultNumberOfPartitions() { - String bigqueryReadTable = "bigquery-public-data.samples.shakespeare"; - String flinkSrcTable = "FlinkSrcTable"; - String srcQueryString = "CREATE TABLE " + flinkSrcTable + " (word STRING , word_count BIGINT)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'table' = '" - + bigqueryReadTable - + "',\n" - + " 'filter' = 'word_count > 500',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "' ,\n" - + " 'selectedFields' = 'word,word_count' \n" - + ")"); - Table result = flinkTableEnv.from(flinkSrcTable); - DataStream ds = flinkTableEnv.toAppendStream(result, Row.class); - assertThat(ds.getExecutionConfig().getParallelism()).isEqualTo(1); - } - - @Test - public void testSelectAllColumnsFromATable() { - - String bigqueryReadTable = "bigquery-public-data.samples.shakespeare"; - String flinkSrcTable = "FlinkSrcTable"; - String srcQueryString = "CREATE TABLE " + flinkSrcTable + " (word STRING , word_count BIGINT)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'table' = '" - + bigqueryReadTable - + "',\n" - + " 'filter' = 'word_count > 500',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "' ,\n" - + " 'selectedFields' = 'word,word_count' \n" - + ")"); - Table result = flinkTableEnv.from(flinkSrcTable); - - assertThat(result.getSchema().getFieldDataType(0)).isEqualTo(Optional.of(DataTypes.STRING())); - assertThat(result.getSchema().getFieldDataType(1)).isEqualTo(Optional.of(DataTypes.BIGINT())); - } - - @Test - public void testViewWithDifferentColumnsForSelectAndFilter() { - String bigqueryReadTable = "bigquery-public-data.samples.shakespeare"; - String srcQueryString = "CREATE TABLE " + flinkSrcTable + " (word STRING , word_count BIGINT)"; - String filter = "word_count > 500"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'table' = '" - + bigqueryReadTable - + "',\n" - + " 'filter' = '" - + filter - + "',\n" - + " 'selectedFields' = 'word',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "' \n" - + ")"); - Table result = flinkTableEnv.from(flinkSrcTable); - assertThat(result.getSchema().getFieldDataType(0)).isEqualTo(Optional.of(DataTypes.STRING())); - } -} diff --git a/src/test/java/com/google/cloud/flink/bigquery/integration/FlinkReadFromQueryIntegrationTest.java b/src/test/java/com/google/cloud/flink/bigquery/integration/FlinkReadFromQueryIntegrationTest.java deleted file mode 100644 index 6393d8a7..00000000 --- a/src/test/java/com/google/cloud/flink/bigquery/integration/FlinkReadFromQueryIntegrationTest.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.integration; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertThrows; - -import com.google.cloud.bigquery.BigQueryOptions; -import com.google.cloud.flink.bigquery.FlinkBigQueryException; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.junit.Test; - -public class FlinkReadFromQueryIntegrationTest extends FlinkBigQueryIntegrationTestBase { - - StreamTableEnvironment flinkTableEnv; - - public FlinkReadFromQueryIntegrationTest() { - - BigQueryOptions.getDefaultInstance().getService(); - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); // source only supports parallelism of 1 - flinkTableEnv = StreamTableEnvironment.create(env); - } - - @Test - public void testReadFromQuery() { - String sql = - "SELECT tag, COUNT(*) countVal FROM ( SELECT SPLIT(tags, \"|\") tags FROM `q-gcp-6750-pso-gs-flink-22-01.testDataset.posts_questions` a WHERE EXTRACT(YEAR FROM creation_date)>=2014 ), UNNEST(tags) tag GROUP BY 1 ORDER BY 2 DESC LIMIT 10 "; - String flinkSrcTable = "FlinkSrcTable"; - String srcQueryString = "CREATE TABLE " + flinkSrcTable + " (tag STRING,tag_count BIGINT)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'query' = '" - + sql - + "',\n" - + " 'maxParallelism' = '10',\n" - + " 'materializationProject' = 'q-gcp-6750-pso-gs-flink-22-01',\n" - + " 'materializationDataset' = 'testDataset',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "'\n" - + ")"); - final Table sourceTable = flinkTableEnv.from(flinkSrcTable); - TableResult datatable = sourceTable.execute(); - assertNotNull(datatable); - assertEquals(2, datatable.getTableSchema().getFieldCount()); - } - - @Test - public void testBadSql() { - String flinkSrcTable = "FlinkSrcTable"; - String sql = - "SELECT tagging, COUNT(*) countVal FROM ( SELECT SPLIT(tags, \"|\") tagging FROM `q-gcp-6750-pso-gs-flink-22-01.testDataset.posts_questions` a WHERE EXTRACT(YEAR FROM creation_date)>=2014 ), UNNEST(tags) tag GROUP BY 1 ORDER BY 2 DESC LIMIT 10 "; - assertThrows( - RuntimeException.class, - () -> { - String srcQueryString = - "CREATE TABLE " + flinkSrcTable + " (tag STRING,tag_count BIGINT)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'query' = '" - + sql - + "',\n" - + " 'maxParallelism' = '10',\n" - + " 'materializationProject' = 'q-gcp-6750-pso-gs-flink-22-01',\n" - + " 'materializationDataset' = 'testDataset',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "'\n" - + ")"); - Table result = flinkTableEnv.from(flinkSrcTable); - result.execute(); - throw new FlinkBigQueryException("Column not found"); - }); - } - - @Test - public void testReadFromQueryWithNewLine() { - String sql = - "SELECT tag, COUNT(*) countVal \n" - + "FROM ( SELECT SPLIT(tags, \"|\") tags FROM `q-gcp-6750-pso-gs-flink-22-01.testDataset.posts_questions` a \n" - + "WHERE EXTRACT(YEAR FROM creation_date)>=2014 ), UNNEST(tags) tag GROUP BY 1 ORDER BY 2 DESC LIMIT 10 "; - - String flinkSrcTable = "FlinkSrcTable"; - String srcQueryString = "CREATE TABLE " + flinkSrcTable + " (tag STRING,tag_count BIGINT)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'query' = '" - + sql - + "',\n" - + " 'maxParallelism' = '10',\n" - + " 'materializationProject' = 'q-gcp-6750-pso-gs-flink-22-01',\n" - + " 'materializationDataset' = 'testDataset',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "'\n" - + ")"); - final Table sourceTable = flinkTableEnv.from(flinkSrcTable); - TableResult datatable = sourceTable.execute(); - assertNotNull(datatable); - assertEquals(2, datatable.getTableSchema().getFieldCount()); - } -} diff --git a/src/test/java/com/google/cloud/flink/bigquery/integration/FlinkReadIntegrationTest.java b/src/test/java/com/google/cloud/flink/bigquery/integration/FlinkReadIntegrationTest.java deleted file mode 100644 index 1e2ce615..00000000 --- a/src/test/java/com/google/cloud/flink/bigquery/integration/FlinkReadIntegrationTest.java +++ /dev/null @@ -1,310 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.integration; - -import static com.google.common.truth.Truth.assertThat; -import static org.apache.flink.table.api.Expressions.$; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThrows; - -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.junit.Ignore; -import org.junit.Test; - -public class FlinkReadIntegrationTest extends FlinkBigQueryIntegrationTestBase { - String flinkSrcTable; - private static final String ALL_TYPES_TABLE_NAME = "all_types"; - - public FlinkReadIntegrationTest() { - super(); - flinkSrcTable = "FlinkSrcTable"; - } - - private void testWordCount(TableResult tableRes) { - assertThat(tableRes.getTableSchema()).isEqualTo(Constants.WORDCOUNT_TABLE_SCHEMA); - } - - @Test - public void testReadWithOption() { - String bigqueryReadTable = "bigquery-public-data.samples.shakespeare"; - String srcQueryString = "CREATE TABLE " + flinkSrcTable + " (word STRING , word_count BIGINT)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'table' = '" - + bigqueryReadTable - + "',\n" - + " 'selectedFields' = 'word,word_count',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "' \n" - + ")"); - Table result = flinkTableEnv.from(flinkSrcTable); - Table datatable = - result.where($("word_count").isGreaterOrEqual(100)).select($("word"), $("word_count")); - TableResult tableapi = datatable.execute(); - testWordCount(tableapi); - } - - // We are passing filter in table API (Filter will work at flink level) - @Test - public void testReadWithFilterInTableAPI() throws Exception { - String bigqueryReadTable = "bigquery-public-data.samples.shakespeare"; - String srcQueryString = "CREATE TABLE " + flinkSrcTable + " (word STRING , word_count BIGINT)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'table' = '" - + bigqueryReadTable - + "',\n" - + " 'selectedFields' = 'word,word_count',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "' \n" - + ")"); - Table result = flinkTableEnv.from(flinkSrcTable); - Table datatable = - result.where($("word_count").isGreaterOrEqual(500)).select($("word"), $("word_count")); - int count = 0; - TableResult tableResult = datatable.execute(); - try (CloseableIterator it = tableResult.collect()) { - while (it.hasNext()) { - it.next(); - count += 1; - } - } - assertEquals(count, 96); - } - - // We are passing filter as an option (Filter will work at Storage API level) - @Test - public void testReadWithFilter() throws Exception { - String bigqueryReadTable = "bigquery-public-data.samples.shakespeare"; - String filter = "word_count > 500 and word=\"I\""; - String srcQueryString = "CREATE TABLE " + flinkSrcTable + " (word STRING , word_count BIGINT)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'table' = '" - + bigqueryReadTable - + "',\n" - + " 'selectedFields' = 'word,word_count',\n" - + " 'filter' = '" - + filter - + "',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "' \n" - + ")"); - Table result = flinkTableEnv.from(flinkSrcTable); - Table datatable = result.select($("word"), $("word_count")); - int count = 0; - TableResult tableResult = datatable.execute(); - try (CloseableIterator it = tableResult.collect()) { - while (it.hasNext()) { - it.next(); - count += 1; - } - } - assertThat(count).isEqualTo(24); - } - - // TODO: Few data types are not supported by Flink , custom data types research - // is under progress. - @Test - public void testReadForDifferentDataTypes() { - String bigqueryReadTable = testDataset.toString() + "." + ALL_TYPES_TABLE_NAME; - String selectedFields = - "numeric_datatype,string_datatype,bytes_datatype,integer_datatype," - + "float_datatype,boolean_datatype,timestamp_datatype," - + "date_datatype,datetime_datatype,geography_datatype" - + ""; - String srcQueryString = - "CREATE TABLE " - + flinkSrcTable - + " (numeric_datatype DECIMAL(38,9)," - + "string_datatype STRING , bytes_datatype BYTES, integer_datatype BIGINT," - + " float_datatype DOUBLE,boolean_datatype BOOLEAN, timestamp_datatype TIMESTAMP," - + " date_datatype DATE,datetime_datatype TIMESTAMP, geography_datatype STRING" - + ")"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'table' = '" - + bigqueryReadTable - + "',\n" - + " 'selectedFields' = '" - + selectedFields - + "'\n" - + ")"); - Table result = flinkTableEnv.from(flinkSrcTable); - TableResult tableapi = result.execute(); - assertThat(tableapi.getTableSchema()).isEqualTo(Constants.FLINK_TEST_TABLE_SCHEMA); - } - - @Test - public void testReadCompressed() { - String bigqueryReadTable = "bigquery-public-data.samples.shakespeare"; - String srcQueryString = "CREATE TABLE " + flinkSrcTable + " (word STRING , word_count BIGINT)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'table' = '" - + bigqueryReadTable - + "',\n" - + " 'bqEncodedCreateReadSessionRequest' = 'EgZCBBoCEAI',\n" - + " 'selectedFields' = 'word,word_count',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "' \n" - + ")"); - Table result = flinkTableEnv.from(flinkSrcTable); - TableResult tableapi = result.execute(); - testWordCount(tableapi); - } - - @Test - public void testReadCompressedWith1BackgroundThreads() { - String bigqueryReadTable = "bigquery-public-data.samples.shakespeare"; - String table = "flink_test"; - String srcQueryString = "CREATE TABLE " + table + " (word STRING , word_count BIGINT)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'table' = '" - + bigqueryReadTable - + "',\n" - + " 'selectedFields' = 'word,word_count',\n" - + " 'bqEncodedCreateReadSessionRequest' = 'EgZCBBoCEAI',\n" - + " 'bqBackgroundThreadsPerStream' = '1',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "' \n" - + ")"); - Table result = flinkTableEnv.from(table); - TableResult tableapi = result.execute(); - testWordCount(tableapi); - } - - @Test - public void testReadCompressedWith4BackgroundThreads() { - String bigqueryReadTable = "bigquery-public-data.samples.shakespeare"; - String srcQueryString = "CREATE TABLE " + flinkSrcTable + " (word STRING , word_count BIGINT)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'table' = '" - + bigqueryReadTable - + "',\n" - + " 'selectedFields' = 'word,word_count',\n" - + " 'bqEncodedCreateReadSessionRequest' = 'EgZCBBoCEAI',\n" - + " 'bqBackgroundThreadsPerStream' = '4',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "' \n" - + ")"); - Table result = flinkTableEnv.from(flinkSrcTable); - TableResult tableapi = result.execute(); - testWordCount(tableapi); - } - - // TODO : Code is executed in 15 sec , thought it is not working as expected - // with timeout. - @Ignore - @Test(timeout = 50000) // throwing null pointer exception when use timeout - public void testHeadDoesNotTimeoutAndOOM() { - String bigqueryReadTable = - Constants.LARGE_TABLE_PROJECT_ID - + "." - + Constants.LARGE_TABLE_DATASET - + "." - + Constants.LARGE_TABLE; - - String srcQueryString = "CREATE TABLE " + flinkSrcTable + " (is_male BOOLEAN)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'table' = '" - + bigqueryReadTable - + "',\n" - + " 'selectedFields' = '" - + Constants.LARGE_TABLE_FIELD - + "',\n" - + " 'bqBackgroundThreadsPerStream' = '4',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "' \n" - + ")"); - Table result = flinkTableEnv.from(flinkSrcTable); - TableResult tableapi = result.execute(); - assertThat(tableapi.getTableSchema()).isEqualTo(Constants.LARGE_TABLE_SCHEMA); - } - - @Test - public void testNonExistentSchema() { - String bigqueryReadTable = "bigquery-public-data.samples.shakespeare"; - assertThrows( - "Trying to read a non existing table should throw an exception", - ValidationException.class, - () -> { - String srcQueryString = "CREATE TABLE " + flinkSrcTable + " (test STRING)"; - flinkTableEnv.executeSql( - srcQueryString - + "\n" - + "WITH (\n" - + " 'connector' = 'bigquery',\n" - + " 'format' = 'arrow',\n" - + " 'table' = '" - + bigqueryReadTable - + "',\n" - + " 'selectedFields' = 'test',\n" - + " 'credentialsFile' = '" - + System.getenv("GOOGLE_APPLICATION_CREDENTIALS") - + "' \n" - + ")"); - Table result = flinkTableEnv.from(flinkSrcTable); - result.execute(); - }); - } -} diff --git a/src/test/java/com/google/cloud/flink/bigquery/integration/IntegrationTestUtils.java b/src/test/java/com/google/cloud/flink/bigquery/integration/IntegrationTestUtils.java deleted file mode 100644 index 3233e068..00000000 --- a/src/test/java/com/google/cloud/flink/bigquery/integration/IntegrationTestUtils.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Copyright 2018 Google Inc. All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -/* - * Copyright 2022 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.flink.bigquery.integration; - -import com.google.cloud.bigquery.BigQuery; -import com.google.cloud.bigquery.BigQueryOptions; -import com.google.cloud.bigquery.DatasetId; -import com.google.cloud.bigquery.DatasetInfo; -import com.google.cloud.bigquery.Field; -import com.google.cloud.bigquery.Field.Mode; -import com.google.cloud.bigquery.FieldList; -import com.google.cloud.bigquery.Schema; -import com.google.cloud.bigquery.StandardSQLTypeName; -import com.google.cloud.bigquery.StandardTableDefinition; -import com.google.cloud.bigquery.TableId; -import com.google.cloud.bigquery.TableInfo; -import com.google.cloud.bigquery.ViewDefinition; -import com.google.cloud.bigquery.connector.common.BigQueryClient; -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.Base64; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class IntegrationTestUtils { - - static Logger logger = LoggerFactory.getLogger(IntegrationTestUtils.class); - - public static BigQuery getBigquery() { - return BigQueryOptions.getDefaultInstance().getService(); - } - - public static void createDataset(String dataset) { - BigQuery bq = getBigquery(); - DatasetId datasetId = DatasetId.of(dataset); - logger.warn("Creating test dataset: {}", datasetId); - bq.create(DatasetInfo.of(datasetId)); - } - - public static void runQuery(String query) { - BigQueryClient bigQueryClient = - new BigQueryClient(getBigquery(), Optional.empty(), Optional.empty()); - bigQueryClient.query(query); - } - - public static void deleteDatasetAndTables(String dataset) { - BigQuery bq = getBigquery(); - logger.warn("Deleting test dataset '{}' and its contents", dataset); - bq.delete(DatasetId.of(dataset), BigQuery.DatasetDeleteOption.deleteContents()); - } - - static void createView(String dataset, String table, String view) { - BigQuery bq = getBigquery(); - String query = String.format("SELECT * FROM %s.%s", dataset, table); - TableId tableId = TableId.of(dataset, view); - ViewDefinition viewDefinition = ViewDefinition.newBuilder(query).setUseLegacySql(false).build(); - bq.create(TableInfo.of(tableId, viewDefinition)); - } - - public static void createTable(String dataset, String table, String function) - throws UnsupportedEncodingException { - BigQuery bq = getBigquery(); - ArrayList listOfFileds = new ArrayList(); - listOfFileds.add( - Field.newBuilder("numeric_datatype", StandardSQLTypeName.NUMERIC) - .setMode(Mode.NULLABLE) - .build()); - listOfFileds.add( - Field.newBuilder("string_datatype", StandardSQLTypeName.STRING) - .setMode(Mode.NULLABLE) - .build()); - listOfFileds.add( - Field.newBuilder("bytes_datatype", StandardSQLTypeName.BYTES) - .setMode(Mode.NULLABLE) - .build()); - listOfFileds.add( - Field.newBuilder("integer_datatype", StandardSQLTypeName.INT64) - .setMode(Mode.NULLABLE) - .build()); - listOfFileds.add( - Field.newBuilder("float_datatype", StandardSQLTypeName.FLOAT64) - .setMode(Mode.NULLABLE) - .build()); - listOfFileds.add( - Field.newBuilder("boolean_datatype", StandardSQLTypeName.BOOL) - .setMode(Mode.NULLABLE) - .build()); - listOfFileds.add( - Field.newBuilder("timestamp_datatype", StandardSQLTypeName.TIMESTAMP) - .setMode(Mode.NULLABLE) - .build()); - listOfFileds.add( - Field.newBuilder("date_datatype", StandardSQLTypeName.DATE).setMode(Mode.NULLABLE).build()); - listOfFileds.add( - Field.newBuilder("datetime_datatype", StandardSQLTypeName.DATETIME) - .setMode(Mode.NULLABLE) - .build()); - listOfFileds.add( - Field.newBuilder("geography_datatype", StandardSQLTypeName.STRING) - .setMode(Mode.NULLABLE) - .build()); - FieldList fieldlist = FieldList.of(listOfFileds); - Schema schema = Schema.of(fieldlist); - - TableId tableId = TableId.of(dataset, table); - StandardTableDefinition tableDefinition; - tableDefinition = StandardTableDefinition.of(schema); - TableInfo tableInfo = TableInfo.newBuilder(tableId, tableDefinition).build(); - bq.create(tableInfo); - - String base64encodedString = Base64.getEncoder().encodeToString("byte-test".getBytes("utf-8")); - - if (function.equals("read")) { - Map rowContent = new HashMap<>(); - rowContent.put("numeric_datatype", 123.345); - rowContent.put("string_datatype", "flink"); - rowContent.put("bytes_datatype", base64encodedString); - rowContent.put("integer_datatype", 12345); - rowContent.put("float_datatype", 50.05f); - rowContent.put("boolean_datatype", true); - rowContent.put("timestamp_datatype", "2022-03-17 17:11:53 UTC"); - rowContent.put("date_datatype", "2022-01-01"); - rowContent.put("datetime_datatype", "2022-03-17T13:20:23.439071"); - rowContent.put("geography_datatype", "POINT(51.500989020415 -0.124710813123368)"); - } - } -} From 069567347b57962b43a5cf9692041dad4320be84 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 27 Jun 2023 14:09:21 -0700 Subject: [PATCH 02/34] updates in ignore and readme files --- .gitignore | 9 +++++++ README.md | 70 ++++++++++++++++++++++++++++++++++++++++++++++++++++-- 2 files changed, 77 insertions(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index 95a384f5..f7a101f7 100644 --- a/.gitignore +++ b/.gitignore @@ -10,3 +10,12 @@ buildNumber.properties # https://github.com/takari/maven-wrapper#usage-without-binary-jar .mvn/wrapper/maven-wrapper.jar .idea/ +.DS_Store +tools/flink +tools/flink-* +tools/releasing/release +tools/japicmp-output +java.header +.java-version +nb-* +**nbproject \ No newline at end of file diff --git a/README.md b/README.md index 58a13b73..ee0c5c62 100644 --- a/README.md +++ b/README.md @@ -1,5 +1,71 @@ -# Flink BigQuery Connector (Under Development) -BigQuery integration to Apache Flink's Table API +# Apache Flink Google BigQuery Connector (Under Development) [![Code Quality: Java](https://img.shields.io/lgtm/grade/java/g/GoogleCloudDataproc/flink-bigquery-connector.svg?logo=lgtm&logoWidth=18)](https://lgtm.com/projects/g/GoogleCloudDataproc/flink-bigquery-connector/context:java) [![codecov](https://codecov.io/gh/GoogleCloudDataproc/flink-bigquery-connector/branch/master/graph/badge.svg)](https://codecov.io/gh/GoogleCloudDataproc/flink-bigquery-connector) + +This repository contains the Apache Flink BigQuery connector. + +## Apache Flink + +Apache Flink is an open source stream processing framework with powerful stream- and batch-processing capabilities. + +Learn more about Flink at [https://flink.apache.org/](https://flink.apache.org/) + +## Building the Apache Flink BigQuery Connector from Source + +Prerequisites: + +* Unix-like environment (we use Linux, Mac OS X) +* Git +* Maven (we recommend version 3.8.6) +* Java 11 + +``` +git clone https://github.com/GoogleCloudDataproc/flink-bigquery-connector +cd flink-connector-bigquery +mvn clean package -DskipTests +``` + +The resulting jars can be found in the `target` directory of the respective module. + +## Developing Flink + +The Flink committers use IntelliJ IDEA to develop the Flink codebase. +We recommend IntelliJ IDEA for developing projects that involve Scala code. + +Minimal requirements for an IDE are: +* Support for Java and Scala (also mixed projects) +* Support for Maven with Java and Scala + +### IntelliJ IDEA + +The IntelliJ IDE supports Maven out of the box and offers a plugin for Scala development. + +* IntelliJ download: [https://www.jetbrains.com/idea/](https://www.jetbrains.com/idea/) +* IntelliJ Scala Plugin: [https://plugins.jetbrains.com/plugin/?id=1347](https://plugins.jetbrains.com/plugin/?id=1347) + +Check out our [Setting up IntelliJ](https://nightlies.apache.org/flink/flink-docs-master/flinkDev/ide_setup.html#intellij-idea) guide for details. + +## Support + +Don’t hesitate to ask! + +Contact the developers and community on the [mailing lists](https://flink.apache.org/community.html#mailing-lists) if you need any help. + +[Open an issue](https://issues.apache.org/jira/browse/FLINK) if you found a bug in Flink. + +## Documentation + +The documentation of Apache Flink is located on the website: [https://flink.apache.org](https://flink.apache.org) +or in the `docs/` directory of the source code. + +## Fork and Contribute + +This is an active open-source project. We are always open to people who want to use the system or contribute to it. +Contact us if you are looking for implementation tasks that fit your skills. +This article describes [how to contribute to Apache Flink](https://flink.apache.org/contributing/how-to-contribute.html). + +## About + +Apache Flink is an open source project of The Apache Software Foundation (ASF). +The Apache Flink project originated from the [Stratosphere](http://stratosphere.eu) research project. From e36cc446fcd9f8256ba9898713b66e5858aff417 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 27 Jun 2023 14:10:34 -0700 Subject: [PATCH 03/34] prepping the pom addition, added parent's compliance tools --- .asf.yaml | 12 + .editorconfig | 276 +++++++++++++++++ NOTICE | 16 + tools/ci/log4j.properties | 43 +++ tools/maven/checkstyle.xml | 561 +++++++++++++++++++++++++++++++++++ tools/maven/suppressions.xml | 26 ++ 6 files changed, 934 insertions(+) create mode 100644 .asf.yaml create mode 100644 .editorconfig create mode 100644 NOTICE create mode 100644 tools/ci/log4j.properties create mode 100644 tools/maven/checkstyle.xml create mode 100644 tools/maven/suppressions.xml diff --git a/.asf.yaml b/.asf.yaml new file mode 100644 index 00000000..34a25a5e --- /dev/null +++ b/.asf.yaml @@ -0,0 +1,12 @@ +github: + enabled_merge_buttons: + squash: true + merge: false + rebase: true + labels: + - flink + - bigquery + - connector + autolink_jira: FLINK + collaborators: + - flinkbot diff --git a/.editorconfig b/.editorconfig new file mode 100644 index 00000000..b66771d6 --- /dev/null +++ b/.editorconfig @@ -0,0 +1,276 @@ +root = true + +[*] +charset = utf-8 +end_of_line = lf +insert_final_newline = true +max_line_length = 100 +# ij_formatter_off_tag = @formatter:off +# ij_formatter_on_tag = @formatter:on +# ij_formatter_tags_enabled = false +# ij_smart_tabs = false +# ij_wrap_on_typing = false + +[*.java] +indent_size = 4 +indent_style = space +tab_width = 4 +ij_continuation_indent_size = 8 +# ij_java_align_consecutive_assignments = false +# ij_java_align_consecutive_variable_declarations = false +# ij_java_align_group_field_declarations = false +# ij_java_align_multiline_annotation_parameters = false +# ij_java_align_multiline_array_initializer_expression = false +# ij_java_align_multiline_assignment = false +# ij_java_align_multiline_binary_operation = false +# ij_java_align_multiline_chained_methods = false +# ij_java_align_multiline_extends_list = false +# ij_java_align_multiline_for = true +# ij_java_align_multiline_method_parentheses = false +# ij_java_align_multiline_parameters = true +# ij_java_align_multiline_parameters_in_calls = false +# ij_java_align_multiline_parenthesized_expression = false +# ij_java_align_multiline_records = true +# ij_java_align_multiline_resources = true +# ij_java_align_multiline_ternary_operation = false +# ij_java_align_multiline_text_blocks = false +# ij_java_align_multiline_throws_list = false +# ij_java_align_subsequent_simple_methods = false +# ij_java_align_throws_keyword = false +# ij_java_annotation_parameter_wrap = off +# ij_java_array_initializer_new_line_after_left_brace = false +# ij_java_array_initializer_right_brace_on_new_line = false +# ij_java_array_initializer_wrap = off +# ij_java_assert_statement_colon_on_next_line = false +# ij_java_assert_statement_wrap = off +# ij_java_assignment_wrap = off +ij_java_binary_operation_sign_on_next_line = true +ij_java_binary_operation_wrap = normal +# ij_java_blank_lines_after_anonymous_class_header = 0 +# ij_java_blank_lines_after_class_header = 0 +# ij_java_blank_lines_after_imports = 1 +# ij_java_blank_lines_after_package = 1 +# ij_java_blank_lines_around_class = 1 +# ij_java_blank_lines_around_field = 0 +# ij_java_blank_lines_around_field_in_interface = 0 +# ij_java_blank_lines_around_initializer = 1 +# ij_java_blank_lines_around_method = 1 +# ij_java_blank_lines_around_method_in_interface = 1 +# ij_java_blank_lines_before_class_end = 0 +# ij_java_blank_lines_before_imports = 1 +# ij_java_blank_lines_before_method_body = 0 +# ij_java_blank_lines_before_package = 0 +# ij_java_block_brace_style = end_of_line +# ij_java_block_comment_at_first_column = true +ij_java_call_parameters_new_line_after_left_paren = true +# ij_java_call_parameters_right_paren_on_new_line = false +ij_java_call_parameters_wrap = on_every_item +# ij_java_case_statement_on_separate_line = true +# ij_java_catch_on_new_line = false +# ij_java_class_annotation_wrap = split_into_lines +# ij_java_class_brace_style = end_of_line +ij_java_class_count_to_use_import_on_demand = 9999 +# ij_java_class_names_in_javadoc = 1 +# ij_java_do_not_indent_top_level_class_members = false +# ij_java_do_not_wrap_after_single_annotation = false +# ij_java_do_while_brace_force = never +# ij_java_doc_add_blank_line_after_description = true +ij_java_doc_add_blank_line_after_param_comments = true +ij_java_doc_add_blank_line_after_return = true +# ij_java_doc_add_p_tag_on_empty_lines = true +ij_java_doc_align_exception_comments = false +ij_java_doc_align_param_comments = false +ij_java_doc_do_not_wrap_if_one_line = true +ij_java_doc_enable_formatting = true +# ij_java_doc_enable_leading_asterisks = true +ij_java_doc_indent_on_continuation = true +ij_java_doc_keep_empty_lines = true +# ij_java_doc_keep_empty_parameter_tag = true +# ij_java_doc_keep_empty_return_tag = true +# ij_java_doc_keep_empty_throws_tag = true +# ij_java_doc_keep_invalid_tags = true +# ij_java_doc_param_description_on_new_line = false +ij_java_doc_preserve_line_breaks = false +# ij_java_doc_use_throws_not_exception_tag = true +# ij_java_else_on_new_line = false +# ij_java_entity_dd_suffix = EJB +# ij_java_entity_eb_suffix = Bean +# ij_java_entity_hi_suffix = Home +# ij_java_entity_lhi_prefix = Local +# ij_java_entity_lhi_suffix = Home +# ij_java_entity_li_prefix = Local +# ij_java_entity_pk_class = java.lang.String +# ij_java_entity_vo_suffix = VO +# ij_java_enum_constants_wrap = off +# ij_java_extends_keyword_wrap = off +# ij_java_extends_list_wrap = off +# ij_java_field_annotation_wrap = split_into_lines +# ij_java_finally_on_new_line = false +# ij_java_for_brace_force = never +# ij_java_for_statement_new_line_after_left_paren = false +# ij_java_for_statement_right_paren_on_new_line = false +# ij_java_for_statement_wrap = off +# ij_java_generate_final_locals = false +# ij_java_generate_final_parameters = false +# ij_java_if_brace_force = never +ij_java_imports_layout = org.apache.flink.**,|,org.apache.flink.shaded.**,|,*,|,javax.**,|,java.**,|,scala.**,|,$* +# ij_java_indent_case_from_switch = true +# ij_java_insert_inner_class_imports = false +# ij_java_insert_override_annotation = true +# ij_java_keep_blank_lines_before_right_brace = 2 +# ij_java_keep_blank_lines_between_package_declaration_and_header = 2 +# ij_java_keep_blank_lines_in_code = 2 +# ij_java_keep_blank_lines_in_declarations = 2 +# ij_java_keep_control_statement_in_one_line = true +# ij_java_keep_first_column_comment = true +# ij_java_keep_indents_on_empty_lines = false +# ij_java_keep_line_breaks = true +# ij_java_keep_multiple_expressions_in_one_line = false +# ij_java_keep_simple_blocks_in_one_line = false +# ij_java_keep_simple_classes_in_one_line = false +# ij_java_keep_simple_lambdas_in_one_line = false +# ij_java_keep_simple_methods_in_one_line = false +# ij_java_label_indent_absolute = false +# ij_java_label_indent_size = 0 +# ij_java_lambda_brace_style = end_of_line +ij_java_layout_static_imports_separately = true +# ij_java_line_comment_add_space = false +# ij_java_line_comment_at_first_column = true +# ij_java_message_dd_suffix = EJB +# ij_java_message_eb_suffix = Bean +# ij_java_method_annotation_wrap = split_into_lines +# ij_java_method_brace_style = end_of_line +ij_java_method_call_chain_wrap = on_every_item +ij_java_method_parameters_new_line_after_left_paren = true +# ij_java_method_parameters_right_paren_on_new_line = false +ij_java_method_parameters_wrap = on_every_item +# ij_java_modifier_list_wrap = false +ij_java_names_count_to_use_import_on_demand = 9999 +# ij_java_new_line_after_lparen_in_record_header = false +# ij_java_packages_to_use_import_on_demand = java.awt.*,javax.swing.* +# ij_java_parameter_annotation_wrap = off +# ij_java_parentheses_expression_new_line_after_left_paren = false +# ij_java_parentheses_expression_right_paren_on_new_line = false +# ij_java_place_assignment_sign_on_next_line = false +# ij_java_prefer_longer_names = true +# ij_java_prefer_parameters_wrap = false +# ij_java_record_components_wrap = normal +# ij_java_repeat_synchronized = true +# ij_java_replace_instanceof_and_cast = false +# ij_java_replace_null_check = true +# ij_java_replace_sum_lambda_with_method_ref = true +# ij_java_resource_list_new_line_after_left_paren = false +# ij_java_resource_list_right_paren_on_new_line = false +# ij_java_resource_list_wrap = off +# ij_java_rparen_on_new_line_in_record_header = false +# ij_java_session_dd_suffix = EJB +# ij_java_session_eb_suffix = Bean +# ij_java_session_hi_suffix = Home +# ij_java_session_lhi_prefix = Local +# ij_java_session_lhi_suffix = Home +# ij_java_session_li_prefix = Local +# ij_java_session_si_suffix = Service +# ij_java_space_after_closing_angle_bracket_in_type_argument = false +# ij_java_space_after_colon = true +# ij_java_space_after_comma = true +# ij_java_space_after_comma_in_type_arguments = true +# ij_java_space_after_for_semicolon = true +# ij_java_space_after_quest = true +# ij_java_space_after_type_cast = true +# ij_java_space_before_annotation_array_initializer_left_brace = false +# ij_java_space_before_annotation_parameter_list = false +# ij_java_space_before_array_initializer_left_brace = false +# ij_java_space_before_catch_keyword = true +# ij_java_space_before_catch_left_brace = true +# ij_java_space_before_catch_parentheses = true +# ij_java_space_before_class_left_brace = true +# ij_java_space_before_colon = true +# ij_java_space_before_colon_in_foreach = true +# ij_java_space_before_comma = false +# ij_java_space_before_do_left_brace = true +# ij_java_space_before_else_keyword = true +# ij_java_space_before_else_left_brace = true +# ij_java_space_before_finally_keyword = true +# ij_java_space_before_finally_left_brace = true +# ij_java_space_before_for_left_brace = true +# ij_java_space_before_for_parentheses = true +# ij_java_space_before_for_semicolon = false +# ij_java_space_before_if_left_brace = true +# ij_java_space_before_if_parentheses = true +# ij_java_space_before_method_call_parentheses = false +# ij_java_space_before_method_left_brace = true +# ij_java_space_before_method_parentheses = false +# ij_java_space_before_opening_angle_bracket_in_type_parameter = false +# ij_java_space_before_quest = true +# ij_java_space_before_switch_left_brace = true +# ij_java_space_before_switch_parentheses = true +# ij_java_space_before_synchronized_left_brace = true +# ij_java_space_before_synchronized_parentheses = true +# ij_java_space_before_try_left_brace = true +# ij_java_space_before_try_parentheses = true +# ij_java_space_before_type_parameter_list = false +# ij_java_space_before_while_keyword = true +# ij_java_space_before_while_left_brace = true +# ij_java_space_before_while_parentheses = true +# ij_java_space_inside_one_line_enum_braces = false +# ij_java_space_within_empty_array_initializer_braces = false +# ij_java_space_within_empty_method_call_parentheses = false +# ij_java_space_within_empty_method_parentheses = false +# ij_java_spaces_around_additive_operators = true +# ij_java_spaces_around_assignment_operators = true +# ij_java_spaces_around_bitwise_operators = true +# ij_java_spaces_around_equality_operators = true +# ij_java_spaces_around_lambda_arrow = true +# ij_java_spaces_around_logical_operators = true +# ij_java_spaces_around_method_ref_dbl_colon = false +# ij_java_spaces_around_multiplicative_operators = true +# ij_java_spaces_around_relational_operators = true +# ij_java_spaces_around_shift_operators = true +# ij_java_spaces_around_type_bounds_in_type_parameters = true +# ij_java_spaces_around_unary_operator = false +# ij_java_spaces_within_angle_brackets = false +# ij_java_spaces_within_annotation_parentheses = false +# ij_java_spaces_within_array_initializer_braces = false +# ij_java_spaces_within_braces = false +# ij_java_spaces_within_brackets = false +# ij_java_spaces_within_cast_parentheses = false +# ij_java_spaces_within_catch_parentheses = false +# ij_java_spaces_within_for_parentheses = false +# ij_java_spaces_within_if_parentheses = false +# ij_java_spaces_within_method_call_parentheses = false +# ij_java_spaces_within_method_parentheses = false +# ij_java_spaces_within_parentheses = false +# ij_java_spaces_within_switch_parentheses = false +# ij_java_spaces_within_synchronized_parentheses = false +# ij_java_spaces_within_try_parentheses = false +# ij_java_spaces_within_while_parentheses = false +# ij_java_special_else_if_treatment = true +# ij_java_subclass_name_suffix = Impl +# ij_java_ternary_operation_signs_on_next_line = false +# ij_java_ternary_operation_wrap = off +# ij_java_test_name_suffix = Test +# ij_java_throws_keyword_wrap = off +# ij_java_throws_list_wrap = off +# ij_java_use_external_annotations = false +# ij_java_use_fq_class_names = false +# ij_java_use_relative_indents = false +# ij_java_use_single_class_imports = true +ij_java_variable_annotation_wrap = normal +# ij_java_visibility = public +# ij_java_while_brace_force = never +# ij_java_while_on_new_line = false +# ij_java_wrap_comments = false +ij_java_wrap_first_method_in_call_chain = true +# ij_java_wrap_long_lines = false + +[*.xml] +indent_style = tab +indent_size = 4 + +[*.scala] +indent_style = space +indent_size = 2 + +[*.py] +indent_style = space +indent_size = 4 \ No newline at end of file diff --git a/NOTICE b/NOTICE new file mode 100644 index 00000000..225d05e4 --- /dev/null +++ b/NOTICE @@ -0,0 +1,16 @@ +Apache Flink BigQuery Connector +Copyright 2023 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +Permission to use, copy, modify, and/or distribute this software for any purpose with or without fee is hereby +granted, provided that this permission notice appear in all copies. + +THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH REGARD TO THIS SOFTWARE INCLUDING +ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, +DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, +WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE +USE OR PERFORMANCE OF THIS SOFTWARE. + + diff --git a/tools/ci/log4j.properties b/tools/ci/log4j.properties new file mode 100644 index 00000000..7daf1c33 --- /dev/null +++ b/tools/ci/log4j.properties @@ -0,0 +1,43 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +rootLogger.level = INFO +rootLogger.appenderRef.out.ref = ConsoleAppender + +# ----------------------------------------------------------------------------- +# Console (use 'console') +# ----------------------------------------------------------------------------- + +appender.console.name = ConsoleAppender +appender.console.type = CONSOLE +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{HH:mm:ss,SSS} [%20t] %-5p %-60c %x - %m%n + +# ----------------------------------------------------------------------------- +# File (use 'file') +# ----------------------------------------------------------------------------- +appender.file.name = FileAppender +appender.file.type = FILE +appender.file.fileName = ${sys:log.dir}/mvn-${sys:mvn.forkNumber:-output}.log +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{HH:mm:ss,SSS} [%20t] %-5p %-60c %x - %m%n +appender.file.createOnDemand = true + +# suppress the irrelevant (wrong) warnings from the netty channel handler +logger.netty.name = org.jboss.netty.channel.DefaultChannelPipeline +logger.netty.level = ERROR diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml new file mode 100644 index 00000000..3fc47117 --- /dev/null +++ b/tools/maven/checkstyle.xml @@ -0,0 +1,561 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml new file mode 100644 index 00000000..5d96995b --- /dev/null +++ b/tools/maven/suppressions.xml @@ -0,0 +1,26 @@ + + + + + + + From e1678038e7cb9078f923ff6aea6c88ce0cca04e3 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 27 Jun 2023 20:06:45 -0700 Subject: [PATCH 04/34] adding parent pom and the connector impl project pom --- flink-connector-bigquery/pom.xml | 182 ++++++++++++++ pom.xml | 418 +++++++++++++++++++++++++++++++ 2 files changed, 600 insertions(+) create mode 100644 flink-connector-bigquery/pom.xml create mode 100644 pom.xml diff --git a/flink-connector-bigquery/pom.xml b/flink-connector-bigquery/pom.xml new file mode 100644 index 00000000..40c913dc --- /dev/null +++ b/flink-connector-bigquery/pom.xml @@ -0,0 +1,182 @@ + + + + + 4.0.0 + + + flink-connector-bigquery-parent + org.apache.flink + 1.1-SNAPSHOT + + + flink-connector-bigquery + Flink : Connectors : Google BigQuery + + jar + + + + + org.apache.flink + flink-connector-base + ${flink.version} + provided + + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + + + org.apache.flink + flink-avro + + + org.apache.flink + flink-metrics-dropwizard + + + + + com.google.cloud + google-cloud-bigquerystorage + + + + + com.google.cloud + google-cloud-bigquery + + + + + dev.failsafe + failsafe + + + + + + + org.apache.flink + flink-table-api-java-bridge + ${flink.version} + provided + true + + + + + + org.testcontainers + mongodb + test + + + + org.apache.flink + flink-test-utils + test + + + + org.apache.flink + flink-connector-test-utils + ${flink.version} + test + + + + org.apache.flink + flink-runtime + ${flink.version} + test-jar + test + + + + org.apache.flink + flink-streaming-java + ${flink.version} + test-jar + test + + + + + org.apache.flink + flink-table-planner-loader + ${flink.version} + test + + + + org.apache.flink + flink-table-runtime + ${flink.version} + test + + + + org.apache.flink + flink-table-common + ${flink.version} + test-jar + test + + + + org.apache.flink + flink-connector-base + ${flink.version} + test-jar + test + + + + + + org.apache.flink + flink-architecture-tests-test + test + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + + diff --git a/pom.xml b/pom.xml new file mode 100644 index 00000000..d3f3d0e4 --- /dev/null +++ b/pom.xml @@ -0,0 +1,418 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connector-parent + 1.0.0 + + + org.apache.flink + flink-connector-bigquery-parent + 1.1-SNAPSHOT + + Flink : Connectors : Google BigQuery Parent + pom + https://flink.apache.org + 2023 + + + + The Apache Software License, Version 2.0 + https://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + 26.15.0 + + 1.17.0 + 16.1 + + 5.8.1 + 3.21.0 + 1.17.2 + 3.4.6 + + false + 3.0.0-1.16 + + 1.7.36 + 2.17.2 + + flink-connector-bigquery-parent + + + + flink-connector-bigquery + + + + + com.google.cloud + google-cloud-bigquerystorage + + + org.apache.flink + flink-avro + + + org.apache.flink + flink-metrics-dropwizard + + + org.apache.flink + flink-shaded-force-shading + + + + + dev.failsafe + failsafe + + + + + org.slf4j + slf4j-api + provided + + + + + com.google.code.findbugs + jsr305 + provided + + + + + org.junit.jupiter + junit-jupiter + test + + + + org.assertj + assertj-core + test + + + + org.mockito + mockito-inline + jar + test + + + + org.mockito + mockito-core + jar + test + + + + org.testcontainers + junit-jupiter + test + + + + + + org.apache.logging.log4j + log4j-slf4j-impl + test + + + + org.apache.logging.log4j + log4j-api + test + + + + org.apache.logging.log4j + log4j-core + test + + + + org.apache.flink + flink-test-utils-junit + test + + + + + org.apache.flink + flink-architecture-tests-test + test + + + org.apache.flink + flink-architecture-tests-production + test + + + + + + + + + + com.google.cloud + libraries-bom + ${google-lib-bom.version} + pom + import + + + + + dev.failsafe + failsafe + 3.3.2 + + + + + org.apache.flink + flink-avro + ${flink.version} + + + + org.apache.flink + flink-metrics-dropwizard + ${flink.version} + + + + org.apache.flink + flink-test-utils + ${flink.version} + test + + + + org.apache.flink + flink-test-utils-junit + ${flink.version} + test + + + + org.apache.flink + flink-shaded-force-shading + ${flink.shaded.version} + + + + + + org.apache.flink + flink-architecture-tests-base + ${flink.version} + test + + + + org.apache.flink + flink-architecture-tests-test + ${flink.version} + test + + + + org.apache.flink + flink-architecture-tests-production + ${flink.version} + test + + + + + com.google.code.findbugs + jsr305 + 1.3.9 + + + + org.slf4j + slf4j-api + ${slf4j.version} + + + + org.apache.logging.log4j + log4j-slf4j-impl + ${log4j.version} + + + + org.apache.logging.log4j + log4j-api + ${log4j.version} + + + + org.apache.logging.log4j + log4j-core + ${log4j.version} + + + + + com.fasterxml.jackson + jackson-bom + pom + import + 2.13.4.20221013 + + + + + org.junit + junit-bom + ${junit5.version} + pom + import + + + + org.mockito + mockito-inline + ${mockito.version} + jar + test + + + + org.mockito + mockito-core + ${mockito.version} + jar + test + + + + org.assertj + assertj-core + ${assertj.version} + test + + + + + com.esotericsoftware.kryo + kryo + 2.24.0 + + + + + org.objenesis + objenesis + 2.1 + + + + org.testcontainers + testcontainers-bom + ${testcontainers.version} + pom + import + + + + + + + + org.codehaus.mojo + exec-maven-plugin + false + + + org.apache.flink + flink-ci-tools + ${flink.version} + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + + io.github.zentol.japicmp + japicmp-maven-plugin + + + + org.apache.rat + apache-rat-plugin + false + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + com.diffplug.spotless + spotless-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + + + org.apache.maven.plugins + maven-surefire-plugin + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + + org.apache.maven.plugins + maven-shade-plugin + + + + org.commonjava.maven.plugins + directory-maven-plugin + + + + From 8043378cb5cd10ce9a807d8910d5149bdad0fdac Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 27 Jun 2023 21:38:56 -0700 Subject: [PATCH 05/34] adding common functionalities --- flink-connector-bigquery/pom.xml | 51 +++- .../common/utils/BigQueryStateSerde.java | 129 ++++++++++ .../common/utils/SchemaTransform.java | 232 ++++++++++++++++++ .../common/utils/BigQueryStateSerdeTest.java | 83 +++++++ .../common/utils/SchemaTransformTest.java | 231 +++++++++++++++++ pom.xml | 36 ++- 6 files changed, 734 insertions(+), 28 deletions(-) create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/BigQueryStateSerde.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/BigQueryStateSerdeTest.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransformTest.java diff --git a/flink-connector-bigquery/pom.xml b/flink-connector-bigquery/pom.xml index 40c913dc..b7226623 100644 --- a/flink-connector-bigquery/pom.xml +++ b/flink-connector-bigquery/pom.xml @@ -88,12 +88,6 @@ under the License. - - org.testcontainers - mongodb - test - - org.apache.flink flink-test-utils @@ -176,6 +170,51 @@ under the License. + + org.apache.maven.plugins + maven-surefire-plugin + + + default-test + test + + test + + + ${argLine} -XX:+UseG1GC -Xms256m -Xmx1024m + + + + integration-tests + integration-test + + test + + + ${argLine} -XX:+UseG1GC -Xms256m -Xmx2048m + + + + + + org.jacoco + jacoco-maven-plugin + + + prepare-agent + + prepare-agent + + + + report + prepare-package + + report + + + + diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/BigQueryStateSerde.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/BigQueryStateSerde.java new file mode 100644 index 00000000..7be61ca1 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/BigQueryStateSerde.java @@ -0,0 +1,129 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.common.utils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.function.BiConsumerWithException; +import org.apache.flink.util.function.FunctionWithException; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** A utility class with some helper method for serde in the BigQuery source and its state. */ +@Internal +public class BigQueryStateSerde { + + /** Private constructor for utility class. */ + private BigQueryStateSerde() {} + + /** + * Serializes a list of data and writes it into a data output stream. + * + * @param The type of the list's elements. + * @param out The data output stream. + * @param list The data to be serialized. + * @param serializer The serialization function of the list's elements. + * @throws IOException In case of serialization or stream write problems. + */ + public static void serializeList( + DataOutputStream out, + List list, + BiConsumerWithException serializer) + throws IOException { + out.writeInt(list.size()); + for (T t : list) { + serializer.accept(out, t); + } + } + + /** + * De-serializes a list from the data input stream. + * + * @param The type of the list's elements. + * @param in the data input stream. + * @param deserializer the de-serialization function for the list's elements. + * @return A fully initialized list with elements de-serialized from the data input stream. + * @throws IOException In case of de-serialization or read problems. + */ + public static List deserializeList( + DataInputStream in, FunctionWithException deserializer) + throws IOException { + int size = in.readInt(); + List list = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + T t = deserializer.apply(in); + list.add(t); + } + + return list; + } + + /** + * Serializes a map of data and writes it into a data output stream. + * + * @param The type of the map's keys. + * @param The type of the map's values. + * @param out The data output stream. + * @param map The data output stream. + * @param keySerializer Serialization function for the map's keys. + * @param valueSerializer Serialization function for the map's values. + * @throws IOException In case of serialization or stream write problems. + */ + public static void serializeMap( + DataOutputStream out, + Map map, + BiConsumerWithException keySerializer, + BiConsumerWithException valueSerializer) + throws IOException { + out.writeInt(map.size()); + for (Map.Entry entry : map.entrySet()) { + keySerializer.accept(out, entry.getKey()); + valueSerializer.accept(out, entry.getValue()); + } + } + + /** + * Serializes a list from the data input stream. + * + * @param The type of the map's keys. + * @param The type of the map's values. + * @param in the data input stream. + * @param keyDeserializer De-serialization function for the map's keys. + * @param valueDeserializer De-serialization function for the map's values. + * @return A fully initialized map instance, with elements read from the data input stream. + * @throws IOException In case of de-serialization or read problems. + */ + public static Map deserializeMap( + DataInputStream in, + FunctionWithException keyDeserializer, + FunctionWithException valueDeserializer) + throws IOException { + int size = in.readInt(); + Map result = new HashMap<>(size); + for (int i = 0; i < size; i++) { + K key = keyDeserializer.apply(in); + V value = valueDeserializer.apply(in); + result.put(key, value); + } + return result; + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java new file mode 100644 index 00000000..4400fa51 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java @@ -0,0 +1,232 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.common.utils; + +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableCollection; +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMultimap; +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.FieldList; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * A utility class that helps on the transformation of BigQuery {@link TableSchema} into Avro {@link + * Schema}. Some methods are heavily influenced on the Apache Beam implementation (not externally + * accessible methods). + */ +public class SchemaTransform { + + static final String NAMESPACE = "org.apache.flink.connector.bigquery"; + /** + * Defines the valid mapping between BigQuery types and native Avro types. + * + *

Some BigQuery types are duplicated here since slightly different Avro records are produced + * when exporting data in Avro format and when reading data directly using the read API. + */ + static final ImmutableMultimap BIG_QUERY_TO_AVRO_TYPES = + ImmutableMultimap.builder() + .put("STRING", Schema.Type.STRING) + .put("GEOGRAPHY", Schema.Type.STRING) + .put("BYTES", Schema.Type.BYTES) + .put("INTEGER", Schema.Type.LONG) + .put("INT64", Schema.Type.LONG) + .put("FLOAT", Schema.Type.DOUBLE) + .put("FLOAT64", Schema.Type.DOUBLE) + .put("NUMERIC", Schema.Type.BYTES) + .put("BIGNUMERIC", Schema.Type.BYTES) + .put("BOOLEAN", Schema.Type.BOOLEAN) + .put("BOOL", Schema.Type.BOOLEAN) + .put("TIMESTAMP", Schema.Type.LONG) + .put("RECORD", Schema.Type.RECORD) + .put("STRUCT", Schema.Type.RECORD) + .put("DATE", Schema.Type.STRING) + .put("DATE", Schema.Type.INT) + .put("DATETIME", Schema.Type.STRING) + .put("TIME", Schema.Type.STRING) + .put("TIME", Schema.Type.LONG) + .put("JSON", Schema.Type.STRING) + .build(); + + public static Schema toGenericAvroSchema( + String schemaName, List fieldSchemas, String namespace) { + + String nextNamespace = + namespace == null ? null : String.format("%s.%s", namespace, schemaName); + + List avroFields = new ArrayList<>(); + for (TableFieldSchema bigQueryField : fieldSchemas) { + avroFields.add(convertField(bigQueryField, nextNamespace)); + } + return Schema.createRecord( + schemaName, + "Translated Avro Schema for " + schemaName, + namespace == null ? NAMESPACE : namespace, + false, + avroFields); + } + + public static Schema toGenericAvroSchema( + String schemaName, List fieldSchemas) { + return toGenericAvroSchema( + schemaName, fieldSchemas, hasNamespaceCollision(fieldSchemas) ? NAMESPACE : null); + } + + // To maintain backwards compatibility we only disambiguate collisions in the field namespaces + // as these never worked with this piece of code. + private static boolean hasNamespaceCollision(List fieldSchemas) { + Set recordTypeFieldNames = new HashSet<>(); + + List fieldsToCheck = new ArrayList<>(); + for (fieldsToCheck.addAll(fieldSchemas); !fieldsToCheck.isEmpty(); ) { + TableFieldSchema field = fieldsToCheck.remove(0); + if ("STRUCT".equals(field.getType()) || "RECORD".equals(field.getType())) { + if (recordTypeFieldNames.contains(field.getName())) { + return true; + } + recordTypeFieldNames.add(field.getName()); + fieldsToCheck.addAll(field.getFields()); + } + } + + // No collisions present + return false; + } + + @SuppressWarnings({ + "nullness" // Avro library not annotated + }) + private static Schema.Field convertField(TableFieldSchema bigQueryField, String namespace) { + ImmutableCollection avroTypes = + BIG_QUERY_TO_AVRO_TYPES.get(bigQueryField.getType()); + if (avroTypes.isEmpty()) { + throw new IllegalArgumentException( + "Unable to map BigQuery field type " + + bigQueryField.getType() + + " to avro type."); + } + + Schema.Type avroType = avroTypes.iterator().next(); + Schema elementSchema; + if (avroType == Schema.Type.RECORD) { + elementSchema = + toGenericAvroSchema( + bigQueryField.getName(), bigQueryField.getFields(), namespace); + } else { + elementSchema = handleAvroLogicalTypes(bigQueryField, avroType); + } + Schema fieldSchema; + if (bigQueryField.getMode() == null || "NULLABLE".equals(bigQueryField.getMode())) { + fieldSchema = Schema.createUnion(Schema.create(Schema.Type.NULL), elementSchema); + } else if (Objects.equals(bigQueryField.getMode(), "REQUIRED")) { + fieldSchema = elementSchema; + } else if ("REPEATED".equals(bigQueryField.getMode())) { + fieldSchema = Schema.createArray(elementSchema); + } else { + throw new IllegalArgumentException( + String.format("Unknown BigQuery Field Mode: %s", bigQueryField.getMode())); + } + return new Schema.Field( + bigQueryField.getName(), + fieldSchema, + bigQueryField.getDescription(), + (Object) null /* Cast to avoid deprecated JsonNode constructor. */); + } + + private static Schema handleAvroLogicalTypes( + TableFieldSchema bigQueryField, Schema.Type avroType) { + String bqType = bigQueryField.getType(); + switch (bqType) { + case "NUMERIC": + // Default value based on + // https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types#decimal_types + int precision = + Optional.ofNullable(bigQueryField.getPrecision()).orElse(38L).intValue(); + int scale = Optional.ofNullable(bigQueryField.getScale()).orElse(9L).intValue(); + return LogicalTypes.decimal(precision, scale) + .addToSchema(Schema.create(Schema.Type.BYTES)); + case "BIGNUMERIC": + // Default value based on + // https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types#decimal_types + int precisionBigNumeric = + Optional.ofNullable(bigQueryField.getPrecision()).orElse(77L).intValue(); + int scaleBigNumeric = + Optional.ofNullable(bigQueryField.getScale()).orElse(38L).intValue(); + return LogicalTypes.decimal(precisionBigNumeric, scaleBigNumeric) + .addToSchema(Schema.create(Schema.Type.BYTES)); + case "TIMESTAMP": + return LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + case "GEOGRAPHY": + Schema geoSchema = Schema.create(Schema.Type.STRING); + geoSchema.addProp(LogicalType.LOGICAL_TYPE_PROP, "geography_wkt"); + return geoSchema; + default: + return Schema.create(avroType); + } + } + + static List fieldListToListOfTableFieldSchema(FieldList fieldList) { + return Optional.ofNullable(fieldList) + .map( + fList -> + fList.stream() + .map( + field -> + new TableFieldSchema() + .setName(field.getName()) + .setDescription( + field.getDescription()) + .setDefaultValueExpression( + field + .getDefaultValueExpression()) + .setCollation(field.getCollation()) + .setMode( + Optional.ofNullable( + field + .getMode()) + .map(m -> m.name()) + .orElse(null)) + .setType(field.getType().name()) + .setFields( + fieldListToListOfTableFieldSchema( + field + .getSubFields()))) + .collect(Collectors.toList())) + .orElse(Lists.newArrayList()); + } + + /** + * Transforms a BigQuery {@link com.google.cloud.bigquery.Schema} into a {@link TableSchema}. + * + * @param schema the schema from the API. + * @return a TableSchema instance. + */ + public static TableSchema bigQuerySchemaToTableSchema(com.google.cloud.bigquery.Schema schema) { + return new TableSchema().setFields(fieldListToListOfTableFieldSchema(schema.getFields())); + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/BigQueryStateSerdeTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/BigQueryStateSerdeTest.java new file mode 100644 index 00000000..9382b7cf --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/BigQueryStateSerdeTest.java @@ -0,0 +1,83 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.common.utils; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; +import org.apache.flink.shaded.guava30.com.google.common.collect.Maps; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** */ +public class BigQueryStateSerdeTest { + + @Test + public void testListSerDe() throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + + List original = Lists.newArrayList("first", "second", "third", "fourth"); + BigQueryStateSerde.serializeList(out, original, DataOutputStream::writeUTF); + out.flush(); + byte[] serialized = baos.toByteArray(); + + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + + List deserialized = + BigQueryStateSerde.deserializeList(in, DataInput::readUTF); + + Assertions.assertThat(original).isEqualTo(deserialized); + } + } + } + + @Test + public void testMapSerDe() throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + + Map original = Maps.newHashMap(); + original.put("key1", "value1"); + original.put("key2", "value2"); + original.put("key3", "value3"); + BigQueryStateSerde.serializeMap( + out, original, DataOutputStream::writeUTF, DataOutputStream::writeUTF); + out.flush(); + byte[] serialized = baos.toByteArray(); + + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + + Map deserialized = + BigQueryStateSerde.deserializeMap( + in, DataInput::readUTF, DataInput::readUTF); + + Assertions.assertThat(original).isEqualTo(deserialized); + } + } + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransformTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransformTest.java new file mode 100644 index 00000000..c0976952 --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransformTest.java @@ -0,0 +1,231 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.common.utils; + +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.StandardSQLTypeName; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import java.util.List; + +/** */ +public class SchemaTransformTest { + private final List subFields = + Lists.newArrayList( + new TableFieldSchema() + .setName("species") + .setType("STRING") + .setMode("NULLABLE")); + /* + * Note that the quality and quantity fields do not have their mode set, so they should default + * to NULLABLE. This is an important test of BigQuery semantics. + * + * All the other fields we set in this function are required on the Schema response. + * + * See https://cloud.google.com/bigquery/docs/reference/v2/tables#schema + */ + private final List fields = + Lists.newArrayList( + new TableFieldSchema().setName("number").setType("INTEGER").setMode("REQUIRED"), + new TableFieldSchema().setName("species").setType("STRING").setMode("NULLABLE"), + new TableFieldSchema() + .setName("quality") + .setType("FLOAT") /* default to NULLABLE */, + new TableFieldSchema() + .setName("quantity") + .setType("INTEGER") /* default to NULLABLE */, + new TableFieldSchema() + .setName("birthday") + .setType("TIMESTAMP") + .setMode("NULLABLE"), + new TableFieldSchema() + .setName("birthdayMoney") + .setType("NUMERIC") + .setMode("NULLABLE"), + new TableFieldSchema() + .setName("lotteryWinnings") + .setType("BIGNUMERIC") + .setMode("NULLABLE"), + new TableFieldSchema() + .setName("flighted") + .setType("BOOLEAN") + .setMode("NULLABLE"), + new TableFieldSchema().setName("sound").setType("BYTES").setMode("NULLABLE"), + new TableFieldSchema() + .setName("anniversaryDate") + .setType("DATE") + .setMode("NULLABLE"), + new TableFieldSchema() + .setName("anniversaryDatetime") + .setType("DATETIME") + .setMode("NULLABLE"), + new TableFieldSchema() + .setName("anniversaryTime") + .setType("TIME") + .setMode("NULLABLE"), + new TableFieldSchema() + .setName("scion") + .setType("RECORD") + .setMode("NULLABLE") + .setFields(subFields), + new TableFieldSchema() + .setName("associates") + .setType("RECORD") + .setMode("REPEATED") + .setFields(subFields), + new TableFieldSchema() + .setName("geoPositions") + .setType("GEOGRAPHY") + .setMode("NULLABLE")); + + @Test + public void testConvertBigQuerySchemaToAvroSchema() { + TableSchema tableSchema = new TableSchema(); + tableSchema.setFields(fields); + Schema avroSchema = + SchemaTransform.toGenericAvroSchema("testSchema", tableSchema.getFields()); + + Assertions.assertThat(avroSchema.getField("number").schema()) + .isEqualTo(Schema.create(Schema.Type.LONG)); + + Assertions.assertThat(avroSchema.getField("species").schema()) + .isEqualTo( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.STRING))); + Assertions.assertThat(avroSchema.getField("quality").schema()) + .isEqualTo( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.DOUBLE))); + Assertions.assertThat(avroSchema.getField("quantity").schema()) + .isEqualTo( + Schema.createUnion( + Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.LONG))); + Assertions.assertThat(avroSchema.getField("birthday").schema()) + .isEqualTo( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.timestampMicros() + .addToSchema(Schema.create(Schema.Type.LONG)))); + Assertions.assertThat(avroSchema.getField("birthdayMoney").schema()) + .isEqualTo( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.decimal(38, 9) + .addToSchema(Schema.create(Schema.Type.BYTES)))); + Assertions.assertThat(avroSchema.getField("lotteryWinnings").schema()) + .isEqualTo( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.decimal(77, 38) + .addToSchema(Schema.create(Schema.Type.BYTES)))); + Assertions.assertThat(avroSchema.getField("flighted").schema()) + .isEqualTo( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.BOOLEAN))); + Assertions.assertThat(avroSchema.getField("sound").schema()) + .isEqualTo( + Schema.createUnion( + Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.BYTES))); + Assertions.assertThat(avroSchema.getField("anniversaryDate").schema()) + .isEqualTo( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.STRING))); + Assertions.assertThat(avroSchema.getField("anniversaryDatetime").schema()) + .isEqualTo( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.STRING))); + Assertions.assertThat(avroSchema.getField("anniversaryTime").schema()) + .isEqualTo( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.STRING))); + Schema geoSchema = Schema.create(Schema.Type.STRING); + geoSchema.addProp(LogicalType.LOGICAL_TYPE_PROP, "geography_wkt"); + Assertions.assertThat(avroSchema.getField("geoPositions").schema()) + .isEqualTo(Schema.createUnion(Schema.create(Schema.Type.NULL), geoSchema)); + Assertions.assertThat(avroSchema.getField("scion").schema()) + .isEqualTo( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.createRecord( + "scion", + "Translated Avro Schema for scion", + SchemaTransform.NAMESPACE, + false, + ImmutableList.of( + new Schema.Field( + "species", + Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.STRING)), + null, + (Object) null))))); + Assertions.assertThat(avroSchema.getField("associates").schema()) + .isEqualTo( + Schema.createArray( + Schema.createRecord( + "associates", + "Translated Avro Schema for associates", + SchemaTransform.NAMESPACE, + false, + ImmutableList.of( + new Schema.Field( + "species", + Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.STRING)), + null, + (Object) null))))); + } + + @Test + public void testBQSchemaToTableSchema() { + + String fieldName = "field1"; + + TableSchema expected = + new TableSchema() + .setFields( + Lists.newArrayList( + new TableFieldSchema() + .setName(fieldName) + .setType("STRING") + .setFields(Lists.newArrayList()))); + + com.google.cloud.bigquery.Schema schema = + com.google.cloud.bigquery.Schema.of( + Lists.newArrayList(Field.of(fieldName, StandardSQLTypeName.STRING))); + + TableSchema transformed = SchemaTransform.bigQuerySchemaToTableSchema(schema); + + Assertions.assertThat(transformed).isEqualTo(expected); + } +} diff --git a/pom.xml b/pom.xml index d3f3d0e4..9a343107 100644 --- a/pom.xml +++ b/pom.xml @@ -45,21 +45,21 @@ under the License. - 26.15.0 + 26.16.0 - 1.17.0 - 16.1 + 1.17.1 + 17.0 - 5.8.1 - 3.21.0 - 1.17.2 - 3.4.6 + 5.9.3 + 3.24.2 + 5.2.0 false 3.0.0-1.16 + 0.8.10 - 1.7.36 - 2.17.2 + 2.0.7 + 2.20.0 flink-connector-bigquery-parent @@ -133,12 +133,6 @@ under the License. test - - org.testcontainers - junit-jupiter - test - - @@ -340,13 +334,6 @@ under the License. 2.1 - - org.testcontainers - testcontainers-bom - ${testcontainers.version} - pom - import - @@ -413,6 +400,11 @@ under the License. org.commonjava.maven.plugins directory-maven-plugin + + org.jacoco + jacoco-maven-plugin + ${jacoco.version} + From b455b4e6480a4b23a55e9e663ef77500e455da78 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Wed, 28 Jun 2023 00:33:41 -0700 Subject: [PATCH 06/34] added the bigquery services wrapper and factories --- flink-connector-bigquery/pom.xml | 7 + .../common/config/BigQueryConnectOptions.java | 167 +++++++++ .../common/config/CredentialsOptions.java | 131 +++++++ .../utils/GoogleCredentialsSupplier.java | 91 +++++ .../services/BigQueryServiceImpl.java | 348 ++++++++++++++++++ .../bigquery/services/BigQueryServices.java | 160 ++++++++ .../services/BigQueryServicesFactory.java | 109 ++++++ .../bigquery/services/BigQueryUtils.java | 167 +++++++++ .../bigquery/services/QueryResultInfo.java | 155 ++++++++ .../services/BigQueryServicesTest.java | 63 ++++ .../bigquery/services/BigQueryUtilsTest.java | 90 +++++ .../services/QueryResultInfoTest.java | 53 +++ 12 files changed, 1541 insertions(+) create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/config/BigQueryConnectOptions.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/config/CredentialsOptions.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/GoogleCredentialsSupplier.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServiceImpl.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServices.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesFactory.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryUtils.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/QueryResultInfo.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/BigQueryServicesTest.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/BigQueryUtilsTest.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/QueryResultInfoTest.java diff --git a/flink-connector-bigquery/pom.xml b/flink-connector-bigquery/pom.xml index b7226623..befb6af2 100644 --- a/flink-connector-bigquery/pom.xml +++ b/flink-connector-bigquery/pom.xml @@ -199,6 +199,13 @@ under the License. org.jacoco jacoco-maven-plugin + + + **/com/google/cloud/flink/bigquery/common/config/* + **/com/google/cloud/flink/bigquery/common/utils/GoogleCredentialsSupplier.class + **/com/google/cloud/flink/bigquery/services/**Impl.class + + prepare-agent diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/config/BigQueryConnectOptions.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/config/BigQueryConnectOptions.java new file mode 100644 index 00000000..53b2bf15 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/config/BigQueryConnectOptions.java @@ -0,0 +1,167 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.common.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.function.SerializableSupplier; + +import com.google.auto.value.AutoValue; +import com.google.cloud.flink.bigquery.services.BigQueryServices; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Objects; + +/** BigQuery client connection configuration. */ +@AutoValue +@PublicEvolving +public abstract class BigQueryConnectOptions implements Serializable { + + public abstract String getProjectId(); + + public abstract String getDataset(); + + public abstract String getTable(); + + @Nullable + public abstract CredentialsOptions getCredentialsOptions(); + + @Nullable + public abstract SerializableSupplier getTestingBigQueryServices(); + + @Override + public String toString() { + return String.format( + "[project : %s, dataset : %s, table : %s]", + getProjectId(), getDataset(), getTable()); + } + + @Override + public int hashCode() { + int hash = 5; + hash = 61 * hash + Objects.hashCode(getProjectId()); + hash = 61 * hash + Objects.hashCode(getDataset()); + hash = 61 * hash + Objects.hashCode(getTable()); + hash = 61 * hash + Objects.hashCode(getCredentialsOptions()); + return hash; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final BigQueryConnectOptions other = (BigQueryConnectOptions) obj; + if (!Objects.equals(this.getProjectId(), other.getProjectId())) { + return false; + } + if (!Objects.equals(this.getDataset(), other.getDataset())) { + return false; + } + if (!Objects.equals(this.getTable(), other.getTable())) { + return false; + } + return Objects.equals(this.getCredentialsOptions(), other.getCredentialsOptions()); + } + + /** + * Creates a builder for the instance. + * + * @return A Builder instance. + * @throws java.io.IOException + */ + public static Builder builder() throws IOException { + return new AutoValue_BigQueryConnectOptions.Builder() + .setCredentialsOptions(CredentialsOptions.builder().build()); + } + + public static Builder builderForQuerySource() throws IOException { + return new AutoValue_BigQueryConnectOptions.Builder() + .setCredentialsOptions(CredentialsOptions.builder().build()) + .setProjectId("") + .setDataset("") + .setTable(""); + } + + /** + * Transforms the instance into a builder instance for property modification. + * + * @return A {@link Builder} instance for the type. + */ + public abstract Builder toBuilder(); + + /** Builder class for BigQueryConnectOptions. */ + @AutoValue.Builder + public abstract static class Builder { + /** + * Sets the project for the BigQuery resource. + * + * @param projectId A GCP project name + * @return A BigQueryConnectOptions builder instance + */ + public abstract Builder setProjectId(String projectId); + + /** + * Sets the dataset for the BigQuery resource. + * + * @param dataset A BigQuery dataset name + * @return A BigQueryConnectOptions builder instance + */ + public abstract Builder setDataset(String dataset); + + /** + * Sets the BigQuery table resource name. + * + * @param table A BigQuery table name + * @return A BigQueryConnectOptions builder instance + */ + public abstract Builder setTable(String table); + + /** + * Sets the GCP credentials options. + * + * @param options A credentials option instance. + * @return A BigQueryConnectOptions builder instance + */ + public abstract Builder setCredentialsOptions(CredentialsOptions options); + + /** + * Sets a testing implementation for the BigQuery services, needs to be supplied in runtime + * to avoid serialization problems. + * + * @param bqServices An test instance of the {@link BigQueryServices} class. + * @return A BigQueryConnectOptions builder instance + */ + public abstract Builder setTestingBigQueryServices( + SerializableSupplier bqServices); + + /** + * Creates the BigQueryConnectOptions object. + * + * @return the options instance. + */ + public abstract BigQueryConnectOptions build(); + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/config/CredentialsOptions.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/config/CredentialsOptions.java new file mode 100644 index 00000000..55210362 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/config/CredentialsOptions.java @@ -0,0 +1,131 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.common.config; + +import org.apache.flink.annotation.PublicEvolving; + +import com.google.auth.Credentials; +import com.google.auto.value.AutoValue; +import com.google.cloud.flink.bigquery.common.utils.GoogleCredentialsSupplier; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Objects; +import java.util.Optional; + +/** An options object that covers the possible {@link Credentials} configurations. */ +@AutoValue +@PublicEvolving +public abstract class CredentialsOptions implements Serializable { + + @Nullable + public abstract String getCredentialsFile(); + + @Nullable + public abstract String getCredentialsKey(); + + @Nullable + public abstract String getAccessToken(); + + /** + * Returns the Google Credentials created given the provided configuration. + * + * @return The Google Credentials instance. + */ + public Credentials getCredentials() { + return GoogleCredentialsSupplier.supplyCredentialsFromSources( + Optional.ofNullable(getAccessToken()), + Optional.ofNullable(getCredentialsKey()), + Optional.ofNullable(getCredentialsFile())); + } + + @Override + public int hashCode() { + int hash = 5; + hash = 61 * hash + Objects.hashCode(getCredentialsFile()); + hash = 61 * hash + Objects.hashCode(getCredentialsKey()); + hash = 61 * hash + Objects.hashCode(getAccessToken()); + return hash; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final CredentialsOptions other = (CredentialsOptions) obj; + if (!Objects.equals(this.getCredentialsFile(), other.getCredentialsFile())) { + return false; + } + if (!Objects.equals(this.getCredentialsKey(), other.getCredentialsKey())) { + return false; + } + return Objects.equals(this.getAccessToken(), other.getAccessToken()); + } + + /** + * Creates a builder class for the {@link CredentialsOptions} class. + * + * @return A builder class. + */ + public static CredentialsOptions.Builder builder() { + return new AutoValue_CredentialsOptions.Builder(); + } + + /** A builder class for the {@link CredentialsOptions} class. */ + @AutoValue.Builder + public abstract static class Builder { + + /** + * Sets the credentials using a file system location. + * + * @param credentialsFile the path of the credentials file. + * @return this builder's instance + */ + public abstract Builder setCredentialsFile(String credentialsFile); + + /** + * Sets the credentials using a credentials key, encoded in Base64. + * + * @param credentialsKey The credentials key. + * @return this builder's instance + */ + public abstract Builder setCredentialsKey(String credentialsKey); + + /** + * Sets the credentials using a GCP access token. + * + * @param credentialsToken The GCP access token. + * @return this builder's instance + */ + public abstract Builder setAccessToken(String credentialsToken); + + /** + * Builds a fully initialized {@link CredentialsOptions} instance. + * + * @return The {@link CredentialsOptions} instance. + */ + public abstract CredentialsOptions build(); + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/GoogleCredentialsSupplier.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/GoogleCredentialsSupplier.java new file mode 100644 index 00000000..61737317 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/GoogleCredentialsSupplier.java @@ -0,0 +1,91 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.common.utils; + +import org.apache.flink.annotation.Internal; + +import org.apache.flink.shaded.curator5.com.google.common.io.BaseEncoding; + +import com.google.auth.Credentials; +import com.google.auth.oauth2.AccessToken; +import com.google.auth.oauth2.GoogleCredentials; + +import java.io.ByteArrayInputStream; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Optional; + +/** A utility class to supply credentials given the multiple possible configuration sources. */ +@Internal +public class GoogleCredentialsSupplier { + private GoogleCredentialsSupplier() {} + + /** + * Supplies a Google {@link Credentials} object, given the possible configurations. + * + * @param accessToken The actual access token as a string. + * @param credentialsKey The actual key encoded in a Base64 based string. + * @param credentialsFile The location of the credentials file. + * @return A fully initialized {@link Credentials} object. + */ + public static Credentials supplyCredentialsFromSources( + Optional accessToken, + Optional credentialsKey, + Optional credentialsFile) { + Credentials credentials; + if (accessToken.isPresent()) { + credentials = createCredentialsFromAccessToken(accessToken.get()); + } else if (credentialsKey.isPresent()) { + credentials = createCredentialsFromKey(credentialsKey.get()); + } else if (credentialsFile.isPresent()) { + credentials = createCredentialsFromFile(credentialsFile.get()); + } else { + credentials = createDefaultCredentials(); + } + return credentials; + } + + private static Credentials createCredentialsFromAccessToken(String accessToken) { + return GoogleCredentials.create(new AccessToken(accessToken, null)); + } + + private static Credentials createCredentialsFromKey(String key) { + try { + return GoogleCredentials.fromStream( + new ByteArrayInputStream(BaseEncoding.base64().decode(key))); + } catch (IOException e) { + throw new UncheckedIOException("Failed to create Credentials from key", e); + } + } + + private static Credentials createCredentialsFromFile(String file) { + try { + return GoogleCredentials.fromStream(new FileInputStream(file)); + } catch (IOException e) { + throw new UncheckedIOException("Failed to create Credentials from file", e); + } + } + + private static Credentials createDefaultCredentials() { + try { + return GoogleCredentials.getApplicationDefault(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to create default Credentials", e); + } + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServiceImpl.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServiceImpl.java new file mode 100644 index 00000000..9e6fd32e --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServiceImpl.java @@ -0,0 +1,348 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.services; + +import org.apache.flink.FlinkVersion; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple2; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import com.google.api.gax.core.FixedCredentialsProvider; +import com.google.api.gax.rpc.FixedHeaderProvider; +import com.google.api.gax.rpc.HeaderProvider; +import com.google.api.gax.rpc.ServerStream; +import com.google.api.gax.rpc.UnaryCallSettings; +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.Dataset; +import com.google.api.services.bigquery.model.ErrorProto; +import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.JobConfigurationQuery; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.BigQueryOptions; +import com.google.cloud.bigquery.QueryJobConfiguration; +import com.google.cloud.bigquery.StandardSQLTypeName; +import com.google.cloud.bigquery.TableId; +import com.google.cloud.bigquery.TableResult; +import com.google.cloud.bigquery.storage.v1.BigQueryReadClient; +import com.google.cloud.bigquery.storage.v1.BigQueryReadSettings; +import com.google.cloud.bigquery.storage.v1.CreateReadSessionRequest; +import com.google.cloud.bigquery.storage.v1.ReadRowsRequest; +import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; +import com.google.cloud.bigquery.storage.v1.ReadSession; +import com.google.cloud.bigquery.storage.v1.SplitReadStreamRequest; +import com.google.cloud.bigquery.storage.v1.SplitReadStreamResponse; +import com.google.cloud.flink.bigquery.common.config.CredentialsOptions; +import com.google.cloud.flink.bigquery.common.utils.SchemaTransform; +import org.threeten.bp.Duration; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +/** Implementation of the {@link BigQueryServices} interface that wraps the actual clients. */ +@Internal +public class BigQueryServiceImpl implements BigQueryServices { + + @Override + public StorageReadClient getStorageClient(CredentialsOptions credentialsOptions) + throws IOException { + return new StorageReadClientImpl(credentialsOptions); + } + + @Override + public QueryDataClient getQueryDataClient(CredentialsOptions creadentialsOptions) { + return new QueryDataClientImpl(creadentialsOptions); + } + + /** + * A simple implementation that wraps a BigQuery ServerStream. + * + * @param The type of the underlying streamed data. + */ + public static class BigQueryServerStreamImpl implements BigQueryServerStream { + + private final ServerStream serverStream; + + public BigQueryServerStreamImpl(ServerStream serverStream) { + this.serverStream = serverStream; + } + + @Override + public Iterator iterator() { + return serverStream.iterator(); + } + + @Override + public void cancel() { + serverStream.cancel(); + } + } + + /** A simple implementation of a mocked BigQuery read client wrapper. */ + public static class StorageReadClientImpl implements StorageReadClient { + private static final HeaderProvider USER_AGENT_HEADER_PROVIDER = + FixedHeaderProvider.create( + "user-agent", "Apache_Flink_Java/" + FlinkVersion.current().toString()); + + private final BigQueryReadClient client; + + private StorageReadClientImpl(CredentialsOptions options) throws IOException { + BigQueryReadSettings.Builder settingsBuilder = + BigQueryReadSettings.newBuilder() + .setCredentialsProvider( + FixedCredentialsProvider.create(options.getCredentials())) + .setTransportChannelProvider( + BigQueryReadSettings.defaultGrpcTransportProviderBuilder() + .setHeaderProvider(USER_AGENT_HEADER_PROVIDER) + .build()); + + UnaryCallSettings.Builder + createReadSessionSettings = + settingsBuilder.getStubSettingsBuilder().createReadSessionSettings(); + + createReadSessionSettings.setRetrySettings( + createReadSessionSettings + .getRetrySettings() + .toBuilder() + .setInitialRpcTimeout(Duration.ofHours(2)) + .setMaxRpcTimeout(Duration.ofHours(2)) + .setTotalTimeout(Duration.ofHours(2)) + .build()); + + UnaryCallSettings.Builder + splitReadStreamSettings = + settingsBuilder.getStubSettingsBuilder().splitReadStreamSettings(); + + splitReadStreamSettings.setRetrySettings( + splitReadStreamSettings + .getRetrySettings() + .toBuilder() + .setInitialRpcTimeout(Duration.ofSeconds(30)) + .setMaxRpcTimeout(Duration.ofSeconds(30)) + .setTotalTimeout(Duration.ofSeconds(30)) + .build()); + + this.client = BigQueryReadClient.create(settingsBuilder.build()); + } + + @Override + public ReadSession createReadSession(CreateReadSessionRequest request) { + return client.createReadSession(request); + } + + @Override + public BigQueryServerStream readRows(ReadRowsRequest request) { + return new BigQueryServerStreamImpl<>(client.readRowsCallable().call(request)); + } + + @Override + public void close() { + client.close(); + } + } + + /** A wrapper implementation for the BigQuery service client library methods. */ + public static class QueryDataClientImpl implements QueryDataClient { + private final BigQuery bigQuery; + private final Bigquery bigquery; + + public QueryDataClientImpl(CredentialsOptions options) { + bigQuery = + BigQueryOptions.newBuilder() + .setCredentials(options.getCredentials()) + .build() + .getService(); + bigquery = BigQueryUtils.newBigqueryBuilder(options).build(); + } + + @Override + public List retrieveTablePartitions(String project, String dataset, String table) { + try { + String query = + Lists.newArrayList( + "SELECT", + " partition_id", + "FROM", + String.format( + " `%s.%s.INFORMATION_SCHEMA.PARTITIONS`", + project, dataset), + "WHERE", + " partition_id <> '__STREAMING_UNPARTITIONED__'", + String.format(" table_catalog = '%s'", project), + String.format(" AND table_schema = '%s'", dataset), + String.format(" AND table_name = '%s'", table), + "ORDER BY 1 DESC;") + .stream() + .collect(Collectors.joining("\n")); + + QueryJobConfiguration queryConfig = QueryJobConfiguration.newBuilder(query).build(); + + TableResult results = bigQuery.query(queryConfig); + + return StreamSupport.stream(results.iterateAll().spliterator(), false) + .flatMap(row -> row.stream()) + .map(fValue -> fValue.getStringValue()) + .collect(Collectors.toList()); + } catch (Exception ex) { + throw new RuntimeException( + String.format( + "Problems while trying to retrieve table partitions (table: %s.%s.%s).", + project, dataset, table), + ex); + } + } + + @Override + public Optional> retrievePartitionColumnName( + String project, String dataset, String table) { + try { + String query = + Lists.newArrayList( + "SELECT", + " column_name, data_type", + "FROM", + String.format( + " `%s.%s.INFORMATION_SCHEMA.COLUMNS`", + project, dataset), + "WHERE", + String.format(" table_catalog = '%s'", project), + String.format(" AND table_schema = '%s'", dataset), + String.format(" AND table_name = '%s'", table), + " AND is_partitioning_column = 'YES';") + .stream() + .collect(Collectors.joining("\n")); + + QueryJobConfiguration queryConfig = QueryJobConfiguration.newBuilder(query).build(); + + TableResult results = bigQuery.query(queryConfig); + + return StreamSupport.stream(results.iterateAll().spliterator(), false) + .map( + row -> + row.stream() + .map(fValue -> fValue.getStringValue()) + .collect(Collectors.toList())) + .map(list -> new Tuple2(list.get(0), list.get(1))) + .map( + t -> + new Tuple2( + t.f0, StandardSQLTypeName.valueOf(t.f1))) + .findFirst(); + } catch (Exception ex) { + throw new RuntimeException( + String.format( + "Problems while trying to retrieve table partition's column name (table: %s.%s.%s).", + project, dataset, table), + ex); + } + } + + @Override + public TableSchema getTableSchema(String project, String dataset, String table) { + return SchemaTransform.bigQuerySchemaToTableSchema( + bigQuery.getTable(TableId.of(project, dataset, table)) + .getDefinition() + .getSchema()); + } + + @Override + public Job dryRunQuery(String projectId, String query) { + try { + JobConfigurationQuery queryConfiguration = + new JobConfigurationQuery() + .setQuery(query) + .setUseQueryCache(true) + .setUseLegacySql(false); + /** first we need to execute a dry-run to understand the expected query location. */ + return BigQueryUtils.dryRunQuery(bigquery, projectId, queryConfiguration, null); + + } catch (Exception ex) { + throw new RuntimeException( + "Problems occurred while trying to dry-run a BigQuery query job.", ex); + } + } + + @Override + public Optional runQuery(String projectId, String query) { + try { + JobConfigurationQuery queryConfiguration = + new JobConfigurationQuery() + .setQuery(query) + .setUseQueryCache(true) + .setUseLegacySql(false); + /** first we need to execute a dry-run to understand the expected query location. */ + Job dryRun = + BigQueryUtils.dryRunQuery(bigquery, projectId, queryConfiguration, null); + + if (dryRun.getStatus().getErrors() != null) { + return Optional.of(dryRun.getStatus().getErrors()) + .map(errors -> processErrorMessages(errors)) + .map(errors -> QueryResultInfo.failed(errors)); + } + List referencedTables = + dryRun.getStatistics().getQuery().getReferencedTables(); + TableReference firstTable = referencedTables.get(0); + Dataset dataset = + BigQueryUtils.datasetInfo( + bigquery, firstTable.getProjectId(), firstTable.getDatasetId()); + + /** + * Then we run the query and check the results to provide errors or a set of + * project, dataset and table to be read. + */ + Job job = + BigQueryUtils.runQuery( + bigquery, projectId, queryConfiguration, dataset.getLocation()); + + TableReference queryDestTable = + job.getConfiguration().getQuery().getDestinationTable(); + + return Optional.of( + Optional.ofNullable(job.getStatus()) + .flatMap(s -> Optional.ofNullable(s.getErrors())) + .map(errors -> processErrorMessages(errors)) + .map(errors -> QueryResultInfo.failed(errors)) + .orElse( + QueryResultInfo.succeed( + queryDestTable.getProjectId(), + queryDestTable.getDatasetId(), + queryDestTable.getTableId()))); + } catch (Exception ex) { + throw new RuntimeException( + "Problems occurred while trying to run a BigQuery query job.", ex); + } + } + + static List processErrorMessages(List errors) { + return errors.stream() + .map( + error -> + String.format( + "Message: '%s'," + " reason: '%s'," + " location: '%s'", + error.getMessage(), + error.getReason(), + error.getLocation())) + .collect(Collectors.toList()); + } + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServices.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServices.java new file mode 100644 index 00000000..1a099de1 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServices.java @@ -0,0 +1,160 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.services; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple2; + +import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.StandardSQLTypeName; +import com.google.cloud.bigquery.storage.v1.CreateReadSessionRequest; +import com.google.cloud.bigquery.storage.v1.ReadRowsRequest; +import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; +import com.google.cloud.bigquery.storage.v1.ReadSession; +import com.google.cloud.flink.bigquery.common.config.CredentialsOptions; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import java.util.Optional; + +/** + * Interface defining the behavior to access and operate the needed BigQuery services. This + * definitions should simplify the faking or mocking of the actual implementations when testing. + */ +@Internal +public interface BigQueryServices extends Serializable { + + /** + * Retrieves a real, mock or fake {@link QueryDataClient}. + * + * @param credentialsOptions The options for the read operation. + * @return a Query data client for BigQuery. + */ + QueryDataClient getQueryDataClient(CredentialsOptions credentialsOptions); + + /** + * Returns a real, mock, or fake {@link StorageReadClient}. + * + * @param credentialsOptions The options for the read operation. + * @return a storage read client object. + * @throws IOException + */ + StorageReadClient getStorageClient(CredentialsOptions credentialsOptions) throws IOException; + + /** + * Container for reading data from streaming endpoints. + * + *

An implementation does not need to be thread-safe. + * + * @param The type of the response. + */ + interface BigQueryServerStream extends Iterable, Serializable { + /** + * Cancels the stream, releasing any client- and server-side resources. This method may be + * called multiple times and from any thread. + */ + void cancel(); + } + + /** An interface representing a client object for making calls to the BigQuery Storage API. */ + interface StorageReadClient extends AutoCloseable { + /** + * Create a new read session against an existing table.This method variant collects request + * count metric, table id in the request. + * + * @param request + * @return + */ + ReadSession createReadSession(CreateReadSessionRequest request); + + /** + * Read rows in the context of a specific read stream. + * + * @param request The request for the storage API + * @return a server stream response with the read rows. + */ + BigQueryServerStream readRows(ReadRowsRequest request); + + /** + * Close the client object. + * + *

The override is required since {@link AutoCloseable} allows the close method to raise + * an exception. + */ + @Override + void close(); + } + + /** + * An interface representing the client interactions needed to retrieve data from BigQuery using + * SQL queries. + */ + interface QueryDataClient extends Serializable { + /** + * Returns a list with the table's existing partitions. + * + * @param project The GCP project. + * @param dataset The BigQuery dataset. + * @param table The BigQuery table. + * @return A list of the partition identifiers. + */ + List retrieveTablePartitions(String project, String dataset, String table); + + /** + * Returns, in case of having one, the partition column name of the table and its type. + * + * @param project The GCP project. + * @param dataset The BigQuery dataset. + * @param table The BigQuery table. + * @return The partition column name and its type, if it has one. + */ + Optional> retrievePartitionColumnName( + String project, String dataset, String table); + + /** + * Returns the {@link TableSchema} of the specified BigQuery table. + * + * @param project The GCP project. + * @param dataset The BigQuery dataset. + * @param table The BigQuery table. + * @return The BigQuery table {@link TableSchema}. + */ + TableSchema getTableSchema(String project, String dataset, String table); + + /** + * Executes a BigQuery query and returns the information about the execution results + * (including if succeeded of failed related information). No data is being returned by this + * method, just a description of what happened with the execution. + * + * @param projectId The project where the query will be run. + * @param query The query to run. + * @return Possibly information of the query execution or empty when not run. + */ + Optional runQuery(String projectId, String query); + + /** + * Executes a BigQuery dry run for the provided query and return the job information. + * + * @param projectId The project where the query will be run. + * @param query The query to run. + * @return The dry run job's information. + */ + Job dryRunQuery(String projectId, String query); + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesFactory.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesFactory.java new file mode 100644 index 00000000..8773ee62 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesFactory.java @@ -0,0 +1,109 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.services; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.util.Preconditions; + +import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; + +import java.io.IOException; + +/** + * A factory class to dispatch the right implementation of the BigQuery services functionalities. + * This class can be configured to use a mock implementation of the BigQuery services, simplifying + * testing of the library. + */ +@Internal +public class BigQueryServicesFactory { + + private static final BigQueryServicesFactory INSTANCE = new BigQueryServicesFactory(); + private static final BigQueryServices SERVICES = new BigQueryServiceImpl(); + + private Boolean isTestingEnabled = false; + private BigQueryServices testingServices; + private BigQueryConnectOptions bqConnectOptions; + + private BigQueryServicesFactory() {} + + /** + * Returns the factory instance, given the current factory's internal state. + * + * @param options The BigQuery connect options. + * @return A factory instance. + */ + public static BigQueryServicesFactory instance(BigQueryConnectOptions options) { + INSTANCE.bqConnectOptions = options; + if (options.getTestingBigQueryServices() == null) { + return INSTANCE.defaultImplementation(); + } else { + return INSTANCE.withTestingServices(options.getTestingBigQueryServices().get()); + } + } + + /** + * Returns a BigQuery storage read client, given the factory's current internal state. + * + * @return A BigQuery storage read client. + */ + public BigQueryServices.StorageReadClient storageRead() throws IOException { + if (isTestingEnabled) { + return testingServices.getStorageClient(bqConnectOptions.getCredentialsOptions()); + } + return SERVICES.getStorageClient(bqConnectOptions.getCredentialsOptions()); + } + + /** + * Returns a BigQuery query data client, given the factory's current internal state. + * + * @return A BigQuery query data client. + */ + public BigQueryServices.QueryDataClient queryClient() { + if (isTestingEnabled) { + return testingServices.getQueryDataClient(bqConnectOptions.getCredentialsOptions()); + } + return SERVICES.getQueryDataClient(bqConnectOptions.getCredentialsOptions()); + } + + @VisibleForTesting + BigQueryServicesFactory withTestingServices(BigQueryServices testingServices) { + Preconditions.checkNotNull(testingServices); + isTestingEnabled = true; + this.testingServices = testingServices; + return this; + } + + /** + * Returns the factory instance, with its default implementation (using GCP BigQuery). + * + * @return A factory instance in its default state. + */ + public BigQueryServicesFactory defaultImplementation() { + isTestingEnabled = false; + this.testingServices = null; + return this; + } + + public Boolean getIsTestingEnabled() { + return isTestingEnabled; + } + + public BigQueryServices getTestingServices() { + return testingServices; + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryUtils.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryUtils.java new file mode 100644 index 00000000..47afa700 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryUtils.java @@ -0,0 +1,167 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.services; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.util.EnvironmentInformation; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport; +import com.google.api.client.http.HttpTransport; +import com.google.api.client.json.JsonFactory; +import com.google.api.client.json.gson.GsonFactory; +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.Dataset; +import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.JobConfiguration; +import com.google.api.services.bigquery.model.JobConfigurationQuery; +import com.google.api.services.bigquery.model.JobReference; +import com.google.auth.http.HttpCredentialsAdapter; +import com.google.cloud.flink.bigquery.common.config.CredentialsOptions; +import dev.failsafe.Failsafe; +import dev.failsafe.FailsafeExecutor; +import dev.failsafe.RetryPolicy; +import dev.failsafe.function.CheckedSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.time.Duration; +import java.util.UUID; + +/** Collection of functionalities that simplify BigQuery services interactions. */ +@Internal +public class BigQueryUtils { + private static final Logger LOG = LoggerFactory.getLogger(BigQueryUtils.class); + + static final Long BACKOFF_DELAY_IN_SECONDS = 5L; + static final Long BACKOFF_MAX_DELAY_IN_MINUTES = 5L; + static final Double RETRY_JITTER_PROB = 0.2; + static Integer maxRetryCount = 3; + + /** Global instance of the JSON factory. */ + private static final JsonFactory JSON_FACTORY; + + /** Global instance of the HTTP transport. */ + private static final HttpTransport HTTP_TRANSPORT; + + static { + try { + JSON_FACTORY = GsonFactory.getDefaultInstance(); + HTTP_TRANSPORT = GoogleNetHttpTransport.newTrustedTransport(); + } catch (GeneralSecurityException | IOException e) { + throw new RuntimeException(e); + } + } + + private BigQueryUtils() {} + + public static Bigquery.Builder newBigqueryBuilder(CredentialsOptions options) { + + return new Bigquery.Builder( + HTTP_TRANSPORT, + JSON_FACTORY, + new HttpCredentialsAdapter(options.getCredentials())) + .setApplicationName( + "BigQuery Connector for Apache Flink version " + + EnvironmentInformation.getVersion()); + } + + public static String bqSanitizedRandomUUID() { + return UUID.randomUUID().toString().replaceAll("-", ""); + } + + static FailsafeExecutor buildRetriableExecutorForOperation(String operationName) { + return Failsafe.with( + RetryPolicy.builder() + .handle(Lists.newArrayList(IOException.class)) + .withMaxAttempts(maxRetryCount) + .withBackoff( + Duration.ofSeconds(BACKOFF_DELAY_IN_SECONDS), + Duration.ofMinutes(BACKOFF_MAX_DELAY_IN_MINUTES)) + .withJitter(RETRY_JITTER_PROB) + .onFailedAttempt( + e -> + LOG.error( + "Execution failed for operation: " + operationName, + e.getLastException())) + .onRetry( + r -> + LOG.info( + "Retrying operation {}, for {} time.", + operationName, + r.getExecutionCount())) + .onRetriesExceeded( + e -> + LOG.error( + "Failed to execute operation {}, retries exhausted.")) + .build()); + } + + static T executeOperation( + FailsafeExecutor failsafeExecutor, CheckedSupplier operation) { + return failsafeExecutor.get(() -> operation.get()); + } + + static Job runInsertJob(Bigquery client, String projectId, Job job) throws IOException { + return client.jobs().insert(projectId, job).setPrettyPrint(false).execute(); + } + + public static Job dryRunQuery( + Bigquery client, String projectId, JobConfigurationQuery queryConfig, String location) + throws InterruptedException, IOException { + String jobId = "apacheflink_dryRun_" + bqSanitizedRandomUUID(); + JobReference jobRef = + new JobReference().setLocation(location).setProjectId(projectId).setJobId(jobId); + Job job = + new Job() + .setJobReference(jobRef) + .setConfiguration( + new JobConfiguration().setQuery(queryConfig).setDryRun(true)); + + return executeOperation( + buildRetriableExecutorForOperation(jobId), + () -> runInsertJob(client, projectId, job)); + } + + public static Job runQuery( + Bigquery client, String projectId, JobConfigurationQuery queryConfig, String location) + throws InterruptedException, IOException { + String jobId = "apacheflink_queryjob_" + bqSanitizedRandomUUID(); + JobReference jobRef = + new JobReference().setLocation(location).setProjectId(projectId).setJobId(jobId); + Job job = + new Job() + .setJobReference(jobRef) + .setConfiguration( + new JobConfiguration().setQuery(queryConfig).setDryRun(false)); + + return executeOperation( + buildRetriableExecutorForOperation(jobId), + () -> runInsertJob(client, projectId, job)); + } + + public static Dataset datasetInfo(Bigquery client, String projectId, String datasetId) + throws IOException, InterruptedException { + return executeOperation( + buildRetriableExecutorForOperation( + String.format("GetDataset - %s.%s", projectId, datasetId)), + () -> client.datasets().get(projectId, datasetId).setPrettyPrint(false).execute()); + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/QueryResultInfo.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/QueryResultInfo.java new file mode 100644 index 00000000..da6993a5 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/QueryResultInfo.java @@ -0,0 +1,155 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.services; + +import org.apache.flink.annotation.Internal; + +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; + +import java.io.Serializable; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** Represents the result information of a BigQuery query execution. */ +@Internal +public class QueryResultInfo implements Serializable { + + /** The status of the BigQuery query execution. */ + public enum Status { + SUCCEED, + FAILED + } + + private final Status status; + private final List errorMessages; + private final String destinationProject; + private final String destinationDataset; + private final String destinationTable; + + private QueryResultInfo(Status status, List errors) { + this.status = status; + this.errorMessages = errors; + this.destinationProject = null; + this.destinationDataset = null; + this.destinationTable = null; + } + + private QueryResultInfo(Status status, String project, String dataset, String table) { + this.status = status; + this.errorMessages = ImmutableList.of(); + this.destinationProject = project; + this.destinationDataset = dataset; + this.destinationTable = table; + } + + /** + * Creates a query result info for a failed query job. + * + * @param errors A list of error messages from the execution. + * @return A query result info. + */ + public static QueryResultInfo failed(List errors) { + return new QueryResultInfo(Status.FAILED, errors); + } + + /** + * Creates a query result info for a successful job. + * + * @param project the project for the destination table result. + * @param dataset the dataset for the destination table result. + * @param table the table for the destination table result. + * @return A query result info. + */ + public static QueryResultInfo succeed(String project, String dataset, String table) { + return new QueryResultInfo(Status.SUCCEED, project, dataset, table); + } + + public Status getStatus() { + return status; + } + + public Optional> getErrorMessages() { + return Optional.ofNullable(errorMessages); + } + + public Optional getDestinationProject() { + return Optional.ofNullable(destinationProject); + } + + public Optional getDestinationDataset() { + return Optional.ofNullable(destinationDataset); + } + + public Optional getDestinationTable() { + return Optional.ofNullable(destinationTable); + } + + @Override + public int hashCode() { + int hash = 7; + hash = 97 * hash + Objects.hashCode(this.status); + hash = 97 * hash + Objects.hashCode(this.errorMessages); + hash = 97 * hash + Objects.hashCode(this.destinationProject); + hash = 97 * hash + Objects.hashCode(this.destinationDataset); + hash = 97 * hash + Objects.hashCode(this.destinationTable); + return hash; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final QueryResultInfo other = (QueryResultInfo) obj; + if (!Objects.equals(this.destinationProject, other.destinationProject)) { + return false; + } + if (!Objects.equals(this.destinationDataset, other.destinationDataset)) { + return false; + } + if (!Objects.equals(this.destinationTable, other.destinationTable)) { + return false; + } + if (this.status != other.status) { + return false; + } + return Objects.equals(this.errorMessages, other.errorMessages); + } + + @Override + public String toString() { + return "QueryResultInfo{" + + "status=" + + status + + ", errorMessages=" + + errorMessages + + ", destinationProject=" + + destinationProject + + ", destinationDataset=" + + destinationDataset + + ", destinationTable=" + + destinationTable + + '}'; + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/BigQueryServicesTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/BigQueryServicesTest.java new file mode 100644 index 00000000..6a2ceb6f --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/BigQueryServicesTest.java @@ -0,0 +1,63 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.services; + +import org.apache.flink.util.function.SerializableSupplier; + +import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; +import com.google.cloud.flink.bigquery.common.config.CredentialsOptions; +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import java.io.IOException; + +/** */ +public class BigQueryServicesTest { + @Test + public void testFactoryWithTestServices() throws IOException { + SerializableSupplier dummyServices = + () -> + new BigQueryServices() { + @Override + public BigQueryServices.QueryDataClient getQueryDataClient( + CredentialsOptions credentialsOptions) { + return null; + } + + @Override + public BigQueryServices.StorageReadClient getStorageClient( + CredentialsOptions credentialsOptions) throws IOException { + return null; + } + }; + BigQueryServicesFactory original = + BigQueryServicesFactory.instance( + BigQueryConnectOptions.builderForQuerySource() + .setTestingBigQueryServices(dummyServices) + .build()); + + Assertions.assertThat(original.getIsTestingEnabled()).isTrue(); + Assertions.assertThat(original.getTestingServices()).isNotNull(); + Assertions.assertThat(original.queryClient()).isNull(); + Assertions.assertThat(original.storageRead()).isNull(); + + original.defaultImplementation(); + + Assertions.assertThat(original.getIsTestingEnabled()).isFalse(); + Assertions.assertThat(original.getTestingServices()).isNull(); + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/BigQueryUtilsTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/BigQueryUtilsTest.java new file mode 100644 index 00000000..4b4c91df --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/BigQueryUtilsTest.java @@ -0,0 +1,90 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.services; + +import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.Dataset; +import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.JobConfigurationQuery; +import org.junit.Test; +import org.mockito.ArgumentMatchers; +import org.mockito.Mockito; + +import java.io.IOException; + +/** */ +public class BigQueryUtilsTest { + + @Test + public void testRetries() throws IOException, InterruptedException { + + Bigquery client = Mockito.mock(Bigquery.class); + Bigquery.Jobs jobs = Mockito.mock(Bigquery.Jobs.class); + Bigquery.Jobs.Insert insert = Mockito.mock(Bigquery.Jobs.Insert.class); + Mockito.when(client.jobs()).thenReturn(jobs); + Mockito.when(jobs.insert(ArgumentMatchers.any(), ArgumentMatchers.any())) + .thenReturn(insert); + Mockito.when(insert.setPrettyPrint(false)).thenReturn(insert); + Mockito.when(insert.execute()).thenThrow(new IOException("Expected")); + + try { + BigQueryUtils.maxRetryCount = 2; + BigQueryUtils.dryRunQuery(client, "", null, ""); + } catch (Exception ex) { + // swallow the expected error + } + // check there was a retry because we always fail + Mockito.verify(insert, Mockito.times(2)).execute(); + } + + @Test + public void testNoRetriesJob() throws IOException, InterruptedException { + + Bigquery client = Mockito.mock(Bigquery.class); + Bigquery.Jobs jobs = Mockito.mock(Bigquery.Jobs.class); + Bigquery.Jobs.Insert insert = Mockito.mock(Bigquery.Jobs.Insert.class); + + Mockito.when(client.jobs()).thenReturn(jobs); + Mockito.when(jobs.insert(ArgumentMatchers.any(), ArgumentMatchers.any())) + .thenReturn(insert); + Mockito.when(insert.setPrettyPrint(false)).thenReturn(insert); + Mockito.when(insert.execute()).thenReturn(new Job()); + + BigQueryUtils.maxRetryCount = 5; + BigQueryUtils.runQuery(client, "", new JobConfigurationQuery(), ""); + + // check there was only one request, since no errors occurred + Mockito.verify(insert, Mockito.times(1)).execute(); + } + + @Test + public void testNoRetriesDataset() throws IOException, InterruptedException { + Bigquery client = Mockito.mock(Bigquery.class); + + Bigquery.Datasets datasets = Mockito.mock(Bigquery.Datasets.class); + Bigquery.Datasets.Get got = Mockito.mock(Bigquery.Datasets.Get.class); + Mockito.when(client.datasets()).thenReturn(datasets); + Mockito.when(datasets.get(ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(got); + Mockito.when(got.setPrettyPrint(false)).thenReturn(got); + Mockito.when(got.execute()).thenReturn(new Dataset()); + + BigQueryUtils.maxRetryCount = 100; + BigQueryUtils.datasetInfo(client, "", ""); + // check no retries either + Mockito.verify(got, Mockito.times(1)).execute(); + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/QueryResultInfoTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/QueryResultInfoTest.java new file mode 100644 index 00000000..e3f7a87d --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/QueryResultInfoTest.java @@ -0,0 +1,53 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.services; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +/** */ +public class QueryResultInfoTest { + + @Test + public void testQueryResultInfoFailed() { + QueryResultInfo qri = QueryResultInfo.failed(Lists.newArrayList()); + Assertions.assertThat(qri.getStatus()).isEqualTo(QueryResultInfo.Status.FAILED); + Assertions.assertThat(qri.getDestinationProject()).isEmpty(); + Assertions.assertThat(qri.getDestinationDataset()).isEmpty(); + Assertions.assertThat(qri.getDestinationTable()).isEmpty(); + Assertions.assertThat(qri.getErrorMessages()).isNotEmpty(); + } + + @Test + public void testQueryResultInfoSucceeded() { + QueryResultInfo qri = QueryResultInfo.succeed("", "", ""); + Assertions.assertThat(qri.getStatus()).isEqualTo(QueryResultInfo.Status.SUCCEED); + Assertions.assertThat(qri.getDestinationProject()).isNotEmpty(); + Assertions.assertThat(qri.getDestinationDataset()).isNotEmpty(); + Assertions.assertThat(qri.getDestinationTable()).isNotEmpty(); + Assertions.assertThat(qri.getErrorMessages().get()).isEmpty(); + } + + @Test + public void testNotEquals() { + QueryResultInfo succeed = QueryResultInfo.succeed("", "", ""); + QueryResultInfo failed = QueryResultInfo.failed(Lists.newArrayList()); + Assertions.assertThat(succeed).isNotEqualTo(failed); + } +} From 27cd837acc98f30d7f62068844996eb25845c57e Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Wed, 28 Jun 2023 11:02:25 -0700 Subject: [PATCH 07/34] creates the split, its state and the enumerator state --- .../enumerator/BigQuerySourceEnumState.java | 130 ++++++++++++++++++ .../BigQuerySourceEnumStateSerializer.java | 118 ++++++++++++++++ .../source/split/BigQuerySourceSplit.java | 85 ++++++++++++ .../split/BigQuerySourceSplitSerializer.java | 90 ++++++++++++ .../split/BigQuerySourceSplitState.java | 72 ++++++++++ ...BigQuerySourceEnumStateSerializerTest.java | 96 +++++++++++++ .../BigQuerySourceSplitSerializerTest.java | 51 +++++++ .../split/BigQuerySourceSplitStateTest.java | 54 ++++++++ 8 files changed, 696 insertions(+) create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumState.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializer.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplit.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializer.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitState.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializerTest.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializerTest.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumState.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumState.java new file mode 100644 index 00000000..81614eac --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumState.java @@ -0,0 +1,130 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.enumerator; + +import org.apache.flink.annotation.PublicEvolving; + +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplit; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** The state representation for the BigQuery source enumerator. */ +@PublicEvolving +public class BigQuerySourceEnumState { + + private final List remaniningTableStreams; + private final List completedTableStreams; + private final List remainingSourceSplits; + private final Map assignedSourceSplits; + private final Boolean initialized; + + public BigQuerySourceEnumState( + List remaniningTableStreams, + List completedTableStreams, + List remainingSourceSplits, + Map assignedSourceSplits, + Boolean initialized) { + this.remaniningTableStreams = remaniningTableStreams; + this.completedTableStreams = completedTableStreams; + this.remainingSourceSplits = remainingSourceSplits; + this.assignedSourceSplits = assignedSourceSplits; + this.initialized = initialized; + } + + public List getRemaniningTableStreams() { + return remaniningTableStreams; + } + + public List getCompletedTableStreams() { + return completedTableStreams; + } + + public List getRemainingSourceSplits() { + return remainingSourceSplits; + } + + public Map getAssignedSourceSplits() { + return assignedSourceSplits; + } + + public Boolean isInitialized() { + return initialized; + } + + public static BigQuerySourceEnumState initialState() { + return new BigQuerySourceEnumState( + new ArrayList<>(), new ArrayList<>(), new ArrayList<>(), new HashMap<>(), false); + } + + @Override + public int hashCode() { + int hash = 3; + hash = 29 * hash + Objects.hashCode(this.remaniningTableStreams); + hash = 29 * hash + Objects.hashCode(this.completedTableStreams); + hash = 29 * hash + Objects.hashCode(this.remainingSourceSplits); + hash = 29 * hash + Objects.hashCode(this.assignedSourceSplits); + hash = 29 * hash + Objects.hashCode(this.initialized); + return hash; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final BigQuerySourceEnumState other = (BigQuerySourceEnumState) obj; + if (!Objects.equals(this.remaniningTableStreams, other.remaniningTableStreams)) { + return false; + } + if (!Objects.equals(this.completedTableStreams, other.completedTableStreams)) { + return false; + } + if (!Objects.equals(this.remainingSourceSplits, other.remainingSourceSplits)) { + return false; + } + if (!Objects.equals(this.assignedSourceSplits, other.assignedSourceSplits)) { + return false; + } + return Objects.equals(this.initialized, other.initialized); + } + + @Override + public String toString() { + return "BigQuerySourceEnumState{" + + "remaniningTableStreams=" + + remaniningTableStreams + + ", completedTableStreams=" + + completedTableStreams + + ", remainingSourceSplits=" + + remainingSourceSplits + + ", assignedSourceSplits=" + + assignedSourceSplits + + ", initialized=" + + initialized + + '}'; + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializer.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializer.java new file mode 100644 index 00000000..3551aaa9 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializer.java @@ -0,0 +1,118 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import com.google.cloud.flink.bigquery.common.utils.BigQueryStateSerde; +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplit; +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplitSerializer; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** The {@link SimpleVersionedSerializer} for the enumerator state of BigQuery source. */ +@Internal +public class BigQuerySourceEnumStateSerializer + implements SimpleVersionedSerializer { + public static final BigQuerySourceEnumStateSerializer INSTANCE = + new BigQuerySourceEnumStateSerializer(); + + private BigQuerySourceEnumStateSerializer() { + // singleton instance + } + + @Override + public int getVersion() { + return BigQuerySourceSplitSerializer.CURRENT_VERSION; + } + + @Override + public byte[] serialize(BigQuerySourceEnumState state) throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + BigQueryStateSerde.serializeList( + out, state.getRemaniningTableStreams(), DataOutputStream::writeUTF); + + BigQueryStateSerde.serializeList( + out, state.getCompletedTableStreams(), DataOutputStream::writeUTF); + + BigQueryStateSerde.serializeList( + out, + state.getRemainingSourceSplits(), + BigQuerySourceSplitSerializer.INSTANCE::serializeBigQuerySourceSplit); + + BigQueryStateSerde.serializeMap( + out, + state.getAssignedSourceSplits(), + DataOutputStream::writeUTF, + BigQuerySourceSplitSerializer.INSTANCE::serializeBigQuerySourceSplit); + + out.writeBoolean(state.isInitialized()); + + out.flush(); + return baos.toByteArray(); + } + } + + @Override + public BigQuerySourceEnumState deserialize(int version, byte[] serialized) throws IOException { + if (getVersion() != version) { + throw new IllegalArgumentException( + String.format( + "The provided serializer version (%d) is not expected (expected : %s).", + version, getVersion())); + } + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + List remainingTableStreams = + BigQueryStateSerde.deserializeList(in, DataInput::readUTF); + List completedTableStreams = + BigQueryStateSerde.deserializeList(in, DataInput::readUTF); + List remainingScanSplits = + BigQueryStateSerde.deserializeList( + in, i -> deserializeBigQuerySourceSplit(version, i)); + + Map assignedScanSplits = + BigQueryStateSerde.deserializeMap( + in, + DataInput::readUTF, + i -> deserializeBigQuerySourceSplit(version, i)); + + boolean initialized = in.readBoolean(); + + return new BigQuerySourceEnumState( + remainingTableStreams, + completedTableStreams, + remainingScanSplits, + assignedScanSplits, + initialized); + } + } + + private static BigQuerySourceSplit deserializeBigQuerySourceSplit( + int version, DataInputStream in) throws IOException { + return BigQuerySourceSplitSerializer.INSTANCE.deserializeBigQuerySourceSplit(version, in); + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplit.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplit.java new file mode 100644 index 00000000..d8672037 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplit.java @@ -0,0 +1,85 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.split; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.source.SourceSplit; + +import java.io.Serializable; +import java.util.Objects; + +/** A {@link SourceSplit} implementation for a BigQuery Read API stream. */ +@PublicEvolving +public class BigQuerySourceSplit implements SourceSplit, Serializable { + + private final String streamName; + private final Integer offset; + + public BigQuerySourceSplit(String streamName) { + this.streamName = streamName; + this.offset = 0; + } + + public BigQuerySourceSplit(String streamName, Integer offset) { + this.streamName = streamName; + this.offset = offset; + } + + @Override + public String splitId() { + return streamName; + } + + public String getStreamName() { + return streamName; + } + + public Integer getOffset() { + return offset; + } + + @Override + public int hashCode() { + int hash = 7; + hash = 61 * hash + Objects.hashCode(this.streamName); + hash = 61 * hash + Objects.hashCode(this.offset); + return hash; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final BigQuerySourceSplit other = (BigQuerySourceSplit) obj; + if (!Objects.equals(this.streamName, other.streamName)) { + return false; + } + return Objects.equals(this.offset, other.offset); + } + + @Override + public String toString() { + return "BigQuerySourceSplit{" + "streamName=" + streamName + ", offset=" + offset + '}'; + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializer.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializer.java new file mode 100644 index 00000000..8c6bd83b --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializer.java @@ -0,0 +1,90 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.split; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +/** The {@link SimpleVersionedSerializer serializer} for {@link BigQuerySourceSplit}. */ +@Internal +public class BigQuerySourceSplitSerializer + implements SimpleVersionedSerializer { + + public static final BigQuerySourceSplitSerializer INSTANCE = + new BigQuerySourceSplitSerializer(); + // This version should be bumped after modifying the source split or the enum states. + public static final int CURRENT_VERSION = 0; + + private BigQuerySourceSplitSerializer() { + // singleton instance + } + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(BigQuerySourceSplit obj) throws IOException { + // VERSION 0 serialization + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + serializeBigQuerySourceSplit(out, obj); + out.flush(); + return baos.toByteArray(); + } + } + + @Override + public BigQuerySourceSplit deserialize(int version, byte[] serialized) throws IOException { + if (getVersion() != version) { + throw new IllegalArgumentException( + String.format( + "The provided serializer version (%d) is not expected (expected : %s).", + version, CURRENT_VERSION)); + } + // VERSION 0 deserialization + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + return deserializeBigQuerySourceSplit(version, in); + } + } + + public void serializeBigQuerySourceSplit(DataOutputStream out, BigQuerySourceSplit split) + throws IOException { + out.writeUTF(split.getStreamName()); + out.writeInt(split.getOffset()); + } + + public BigQuerySourceSplit deserializeBigQuerySourceSplit(int version, DataInputStream in) + throws IOException { + switch (version) { + case CURRENT_VERSION: + String streamName = in.readUTF(); + int offset = in.readInt(); + return new BigQuerySourceSplit(streamName, offset); + default: + throw new IOException("Unknown version: " + version); + } + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitState.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitState.java new file mode 100644 index 00000000..009ab58d --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitState.java @@ -0,0 +1,72 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.split; + +import org.apache.flink.annotation.Internal; + +import java.util.Objects; + +/** BigQuery source split state for {@link BigQuerySourceSplit}. */ +@Internal +public class BigQuerySourceSplitState { + private final BigQuerySourceSplit split; + private Integer offset; + + public BigQuerySourceSplitState(BigQuerySourceSplit split) { + this.split = split; + offset = split.getOffset(); + } + + public BigQuerySourceSplit toBigQuerySourceSplit() { + return new BigQuerySourceSplit(split.getStreamName(), offset); + } + + public void updateOffset() { + offset++; + } + + @Override + public String toString() { + return "BigQuerySourceSplitState{" + "split=" + split + ", offset=" + offset + '}'; + } + + @Override + public int hashCode() { + int hash = 7; + hash = 53 * hash + Objects.hashCode(this.split); + hash = 53 * hash + Objects.hashCode(this.offset); + return hash; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final BigQuerySourceSplitState other = (BigQuerySourceSplitState) obj; + if (!Objects.equals(this.split, other.split)) { + return false; + } + return Objects.equals(this.offset, other.offset); + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializerTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializerTest.java new file mode 100644 index 00000000..dfdda05e --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializerTest.java @@ -0,0 +1,96 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.enumerator; + +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplit; +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplitSerializer; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +/** */ +public class BigQuerySourceEnumStateSerializerTest { + + private BigQuerySourceEnumState create() { + + List remainingTableStreams = new ArrayList<>(); + + remainingTableStreams.add("third stream"); + remainingTableStreams.add("fourth stream"); + remainingTableStreams.add("fifth stream"); + + List completedTableStreams = new ArrayList<>(); + completedTableStreams.add("first stream"); + + List remainingSourceSplits = new ArrayList<>(); + remainingSourceSplits.add(new BigQuerySourceSplit("second stream", 0)); + + Map assignedSourceSplits = new TreeMap<>(); + assignedSourceSplits.put("key1", remainingSourceSplits.get(0)); + + return new BigQuerySourceEnumState( + remainingTableStreams, + completedTableStreams, + remainingSourceSplits, + assignedSourceSplits, + true); + } + + @Test + public void testEnumStateSerializerInitialState() throws IOException { + BigQuerySourceEnumState initialState = BigQuerySourceEnumState.initialState(); + + byte[] serialized = BigQuerySourceEnumStateSerializer.INSTANCE.serialize(initialState); + + BigQuerySourceEnumState enumState1 = + BigQuerySourceEnumStateSerializer.INSTANCE.deserialize( + BigQuerySourceSplitSerializer.CURRENT_VERSION, serialized); + + Assert.assertEquals(initialState, enumState1); + Assert.assertEquals(initialState.hashCode(), enumState1.hashCode()); + } + + @Test + public void testEnumStateSerializer() throws IOException { + BigQuerySourceEnumState enumState = create(); + + byte[] serialized = BigQuerySourceEnumStateSerializer.INSTANCE.serialize(enumState); + + BigQuerySourceEnumState enumState1 = + BigQuerySourceEnumStateSerializer.INSTANCE.deserialize( + BigQuerySourceSplitSerializer.CURRENT_VERSION, serialized); + + Assert.assertEquals(enumState, enumState1); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongSerializerVersion() throws IOException { + BigQuerySourceEnumState enumState = create(); + + byte[] serialized = BigQuerySourceEnumStateSerializer.INSTANCE.serialize(enumState); + + BigQuerySourceEnumStateSerializer.INSTANCE.deserialize(1000, serialized); + + // should never reach here + assert (true); + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializerTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializerTest.java new file mode 100644 index 00000000..a7af57fb --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializerTest.java @@ -0,0 +1,51 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.split; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +/** */ +public class BigQuerySourceSplitSerializerTest { + + @Test + public void testSplitSerializer() throws IOException { + BigQuerySourceSplit split = new BigQuerySourceSplit("some stream name", 10); + + byte[] serialized = BigQuerySourceSplitSerializer.INSTANCE.serialize(split); + + BigQuerySourceSplit split1 = + BigQuerySourceSplitSerializer.INSTANCE.deserialize( + BigQuerySourceSplitSerializer.CURRENT_VERSION, serialized); + + Assert.assertEquals(split, split1); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongSerializerVersion() throws IOException { + BigQuerySourceSplit split = new BigQuerySourceSplit("some stream name", 10); + + byte[] serialized = BigQuerySourceSplitSerializer.INSTANCE.serialize(split); + + BigQuerySourceSplitSerializer.INSTANCE.deserialize(1000, serialized); + + // should never reach here + assert (true); + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java new file mode 100644 index 00000000..c9725034 --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java @@ -0,0 +1,54 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.split; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +/** */ +public class BigQuerySourceSplitStateTest { + + @Test + public void testSplitStateTransformation() { + + String streamName = "somestream"; + BigQuerySourceSplit originalSplit = new BigQuerySourceSplit(streamName, 10); + Assertions.assertThat(originalSplit.splitId()).isEqualTo(streamName); + + BigQuerySourceSplitState splitState = new BigQuerySourceSplitState(originalSplit); + Assertions.assertThat(splitState.toBigQuerySourceSplit()).isEqualTo(originalSplit); + } + + @Test + public void testSplitStateMutation() { + + String streamName = "somestream"; + BigQuerySourceSplit originalSplit = new BigQuerySourceSplit(streamName, 10); + BigQuerySourceSplitState splitState = new BigQuerySourceSplitState(originalSplit); + + splitState.updateOffset(); + BigQuerySourceSplit otherSplit = new BigQuerySourceSplit(streamName, 11); + + Assertions.assertThat(splitState.toBigQuerySourceSplit()).isEqualTo(otherSplit); + Assertions.assertThat(splitState.toBigQuerySourceSplit().hashCode()) + .isEqualTo(otherSplit.hashCode()); + // should be different since they started from different splits + Assertions.assertThat(splitState).isNotEqualTo(new BigQuerySourceSplitState(otherSplit)); + Assertions.assertThat(splitState.hashCode()) + .isNotEqualTo(new BigQuerySourceSplitState(otherSplit).hashCode()); + } +} From 4d4b60f37d68a46b876f3491faaabba57a3875a7 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Wed, 28 Jun 2023 14:16:53 -0700 Subject: [PATCH 08/34] added configs, split reader and split assignment impls --- flink-connector-bigquery/pom.xml | 1 + .../source/config/BigQueryReadOptions.java | 264 ++++++++++++++ .../reader/BigQuerySourceReaderContext.java | 95 +++++ .../split/BigQuerySourceSplitAssigner.java | 250 +++++++++++++ .../reader/BigQuerySourceSplitReader.java | 335 +++++++++++++++++ .../bigquery/fakes/StorageClientFaker.java | 336 ++++++++++++++++++ .../services/QueryResultInfoTest.java | 8 + .../BigQuerySourceSplitAssignerTest.java | 79 ++++ .../reader/BigQuerySourceSplitReaderTest.java | 135 +++++++ 9 files changed, 1503 insertions(+) create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/BigQuerySourceReaderContext.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssigner.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReader.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssignerTest.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReaderTest.java diff --git a/flink-connector-bigquery/pom.xml b/flink-connector-bigquery/pom.xml index befb6af2..71e18607 100644 --- a/flink-connector-bigquery/pom.xml +++ b/flink-connector-bigquery/pom.xml @@ -204,6 +204,7 @@ under the License. **/com/google/cloud/flink/bigquery/common/config/* **/com/google/cloud/flink/bigquery/common/utils/GoogleCredentialsSupplier.class **/com/google/cloud/flink/bigquery/services/**Impl.class + **/com/google/cloud/flink/bigquery/source/config/* diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java new file mode 100644 index 00000000..8587a4b9 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java @@ -0,0 +1,264 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; + +import com.google.auto.value.AutoValue; +import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; +import com.google.cloud.flink.bigquery.common.config.CredentialsOptions; +import org.threeten.bp.Instant; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** The options available to read data from BigQuery. */ +@AutoValue +@PublicEvolving +public abstract class BigQueryReadOptions implements Serializable { + + public abstract ImmutableList getColumnNames(); + + public abstract String getRowRestriction(); + + @Nullable + public abstract Long getSnapshotTimestampInMillis(); + + @Nullable + public abstract String getQuery(); + + @Nullable + public abstract String getQueryExecutionProject(); + + public abstract Integer getMaxStreamCount(); + + public abstract Integer getMaxRecordsPerSplitFetch(); + + public abstract BigQueryConnectOptions getBigQueryConnectOptions(); + + @Override + public final int hashCode() { + int hash = 5; + hash = 61 * hash + Objects.hashCode(getColumnNames()); + hash = 61 * hash + Objects.hashCode(getRowRestriction()); + hash = 61 * hash + Objects.hashCode(getSnapshotTimestampInMillis()); + hash = 61 * hash + Objects.hashCode(getMaxStreamCount()); + hash = 61 * hash + Objects.hashCode(getBigQueryConnectOptions()); + return hash; + } + + @Override + public final boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final BigQueryReadOptions other = (BigQueryReadOptions) obj; + if (!Objects.equals(this.getColumnNames(), other.getColumnNames())) { + return false; + } + if (!Objects.equals(this.getRowRestriction(), other.getRowRestriction())) { + return false; + } + if (!Objects.equals( + this.getSnapshotTimestampInMillis(), other.getSnapshotTimestampInMillis())) { + return false; + } + if (!Objects.equals(this.getMaxStreamCount(), other.getMaxStreamCount())) { + return false; + } + return Objects.equals(this.getBigQueryConnectOptions(), other.getBigQueryConnectOptions()); + } + + /** + * Transforms the instance into a builder instance for property modification. + * + * @return A {@link Builder} instance for the type. + */ + public abstract Builder toBuilder(); + + /** + * Creates a builder instance with all the default values set. + * + * @return A {@link Builder} for the type. + */ + public static Builder builder() { + return new AutoValue_BigQueryReadOptions.Builder() + .setRowRestriction("") + .setColumnNames(new ArrayList<>()) + .setMaxStreamCount(0) + .setMaxRecordsPerSplitFetch(10000) + .setSnapshotTimestampInMillis(null); + } + + /** Builder class for {@link BigQueryReadOptions}. */ + @AutoValue.Builder + public abstract static class Builder { + + /** + * Prepares this builder to execute a query driven read using the default credentials + * configuration. + * + * @param query A BigQuery standard SQL query. + * @param projectId A GCP project where the query will run. + * @return This {@link Builder} instance. + * @throws IOException + */ + public Builder setQueryAndExecutionProject(String query, String projectId) + throws IOException { + return setQueryWithExecutionProjectAndCredentialsOptions( + query, projectId, CredentialsOptions.builder().build()); + } + + /** + * Prepares this builder to execute a query driven read. + * + * @param query A BigQuery standard SQL query. + * @param projectId A GCP project where the query will run. + * @param credentialsOptions The GCP credentials options. + * @return This {@link Builder} instance. + * @throws IOException + */ + public Builder setQueryWithExecutionProjectAndCredentialsOptions( + String query, String projectId, CredentialsOptions credentialsOptions) + throws IOException { + this.setQuery(query); + this.setQueryExecutionProject(projectId); + this.setBigQueryConnectOptions( + BigQueryConnectOptions.builderForQuerySource() + .setCredentialsOptions(credentialsOptions) + .build()); + return this; + } + + /** + * Sets a BigQuery query which will be run first, storing its result in a temporary table, + * and Flink will read the query results from that temporary table. This is an optional + * argument. + * + * @param query A BigQuery standard SQL query. + * @return This {@link Builder} instance. + */ + public abstract Builder setQuery(String query); + + /** + * Sets the GCP project where the configured query will be run. In case the query + * configuration is not set this configuration is discarded. + * + * @param projectId A GCP project. + * @return This {@link Builder} instance. + */ + public abstract Builder setQueryExecutionProject(String projectId); + + /** + * Sets the restriction the rows in the BigQuery table must comply to be returned by the + * source. + * + * @param rowRestriction A {@link String} containing the row restrictions. + * @return This {@link Builder} instance. + */ + public abstract Builder setRowRestriction(String rowRestriction); + + /** + * Sets the column names that will be projected from the table's retrieved data. + * + * @param colNames The names of the columns as a list of strings. + * @return This {@link Builder} instance. + */ + public abstract Builder setColumnNames(List colNames); + + /** + * Sets the snapshot time (in milliseconds since epoch) for the BigQuery table, if not set + * {@code now()} is used. + * + * @param snapshotTs The snapshot's time in milliseconds since epoch. + * @return This {@link Builder} instance. + */ + public abstract Builder setSnapshotTimestampInMillis(Long snapshotTs); + + /** + * Sets the maximum number of read streams that BigQuery should create to retrieve data from + * the table. BigQuery can return a lower number than the specified. + * + * @param maxStreamCount The maximum number of read streams. + * @return This {@link Builder} instance. + */ + public abstract Builder setMaxStreamCount(Integer maxStreamCount); + + /** + * Sets the maximum number of records to read from a streams once a fetch has been requested + * from a particular split. Configuring this number too high may cause memory pressure in + * the task manager, depending on the BigQuery record's size and total rows on the stream. + * + * @param maxStreamCount The maximum number records to read from a split at a time. + * @return This {@link Builder} instance. + */ + public abstract Builder setMaxRecordsPerSplitFetch(Integer maxStreamCount); + + /** + * Sets the {@link BigQueryConnectOptions} instance. + * + * @param connect The {@link BigQueryConnectOptions} instance. + * @return This {@link Builder} instance. + */ + public abstract Builder setBigQueryConnectOptions(BigQueryConnectOptions connect); + + abstract BigQueryReadOptions autoBuild(); + + /** + * A fully initialized {@link BigQueryReadOptions} instance. + * + * @return A {@link BigQueryReadOptions} instance. + */ + public final BigQueryReadOptions build() { + BigQueryReadOptions readOptions = autoBuild(); + Preconditions.checkState( + readOptions.getMaxStreamCount() >= 0, + "The max number of streams should be zero or positive."); + Preconditions.checkState( + !Optional.ofNullable(readOptions.getSnapshotTimestampInMillis()) + // see if the value is lower than the epoch + .filter(timeInMillis -> timeInMillis < Instant.EPOCH.toEpochMilli()) + // if present, then fail + .isPresent(), + "The oldest timestamp should be equal or bigger than epoch."); + Preconditions.checkState( + !Optional.ofNullable(readOptions.getQuery()) + // if the project was not configured + .filter(q -> readOptions.getQueryExecutionProject() == null) + // if present fail + .isPresent(), + "If a query is configured, then a GCP projec should be provided."); + + return readOptions; + } + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/BigQuerySourceReaderContext.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/BigQuerySourceReaderContext.java new file mode 100644 index 00000000..71454a0a --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/BigQuerySourceReaderContext.java @@ -0,0 +1,95 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.SourceReaderMetricGroup; +import org.apache.flink.util.UserCodeClassLoader; + +import java.util.concurrent.atomic.AtomicInteger; + +/** A {@link SourceReaderContext} proxy that adds limit and counts for state management. */ +@Internal +public class BigQuerySourceReaderContext implements SourceReaderContext { + + private final SourceReaderContext readerContext; + private final AtomicInteger readCount = new AtomicInteger(0); + private final Integer limit; + + public BigQuerySourceReaderContext(SourceReaderContext readerContext, Integer limit) { + this.readerContext = readerContext; + this.limit = limit; + } + + @Override + public SourceReaderMetricGroup metricGroup() { + return readerContext.metricGroup(); + } + + @Override + public Configuration getConfiguration() { + return readerContext.getConfiguration(); + } + + @Override + public String getLocalHostName() { + return readerContext.getLocalHostName(); + } + + @Override + public int getIndexOfSubtask() { + return readerContext.getIndexOfSubtask(); + } + + @Override + public void sendSplitRequest() { + readerContext.sendSplitRequest(); + } + + @Override + public void sendSourceEventToCoordinator(SourceEvent sourceEvent) { + readerContext.sendSourceEventToCoordinator(sourceEvent); + } + + @Override + public UserCodeClassLoader getUserCodeClassLoader() { + return readerContext.getUserCodeClassLoader(); + } + + public Integer updateReadCount(Integer newReads) { + return readCount.addAndGet(newReads); + } + + public Integer currentReadCount() { + return readCount.get(); + } + + public boolean isLimitPushedDown() { + return limit > 0; + } + + public boolean willItBeOverLimit(Integer newReads) { + return limit > 0 && (readCount.get() + newReads) >= limit; + } + + public int getLimit() { + return limit; + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssigner.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssigner.java new file mode 100644 index 00000000..871356b5 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssigner.java @@ -0,0 +1,250 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.split; + +import org.apache.flink.annotation.Internal; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import com.google.cloud.bigquery.storage.v1.CreateReadSessionRequest; +import com.google.cloud.bigquery.storage.v1.DataFormat; +import com.google.cloud.bigquery.storage.v1.ReadSession; +import com.google.cloud.bigquery.storage.v1.ReadSession.TableModifiers; +import com.google.cloud.bigquery.storage.v1.ReadSession.TableReadOptions; +import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; +import com.google.cloud.flink.bigquery.services.BigQueryServices; +import com.google.cloud.flink.bigquery.services.BigQueryServicesFactory; +import com.google.cloud.flink.bigquery.services.QueryResultInfo; +import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; +import com.google.cloud.flink.bigquery.source.enumerator.BigQuerySourceEnumState; +import com.google.protobuf.Timestamp; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkState; + +/** A simple split assigner based on the BigQuery {@link ReadSession} streams. */ +@Internal +public class BigQuerySourceSplitAssigner { + + private static final Logger LOG = LoggerFactory.getLogger(BigQuerySourceSplitAssigner.class); + + private final BigQueryReadOptions readOptions; + + private final ArrayDeque remainingTableStreams; + private final List alreadyProcessedTableStreams; + private final ArrayDeque remainingSourceSplits; + private final Map assignedSourceSplits; + private boolean initialized; + + public BigQuerySourceSplitAssigner( + BigQueryReadOptions readOptions, BigQuerySourceEnumState sourceEnumState) { + this.readOptions = readOptions; + this.remainingTableStreams = new ArrayDeque<>(sourceEnumState.getRemaniningTableStreams()); + this.alreadyProcessedTableStreams = sourceEnumState.getCompletedTableStreams(); + this.remainingSourceSplits = new ArrayDeque<>(sourceEnumState.getRemainingSourceSplits()); + this.assignedSourceSplits = sourceEnumState.getAssignedSourceSplits(); + this.initialized = sourceEnumState.isInitialized(); + } + + /** + * Reviews the read options argument and see if a query has been configured, in that case run + * that query and then return a modified version of the connect options pointing to the + * temporary location (project, dataset and table) of the query results. + * + * @param readOptions The configured read options. + * @return The BigQuery connect options with the right project, dataset and table given the + * specified configuration. + */ + BigQueryConnectOptions checkOptionsAndRunQueryIfNeededReturningModifiedOptions() { + return Optional.ofNullable(this.readOptions.getQuery()) + // if query is available, execute it using the configured GCP project and gather the + // results + .flatMap( + query -> + BigQueryServicesFactory.instance( + this.readOptions.getBigQueryConnectOptions()) + .queryClient() + .runQuery( + this.readOptions.getQueryExecutionProject(), query)) + // with the query results return the new connection options, fail if the query + // failed + .map( + result -> { + if (result.getStatus().equals(QueryResultInfo.Status.FAILED)) { + throw new IllegalStateException( + "The BigQuery query execution failed with errors: " + + result.getErrorMessages() + .orElse(Lists.newArrayList())); + } + String projectId = result.getDestinationProject().get(); + String dataset = result.getDestinationDataset().get(); + String table = result.getDestinationTable().get(); + LOG.info( + "After BigQuery query execution, switching connect options" + + " to read from table {}.{}.{}", + projectId, + dataset, + table); + return this.readOptions + .getBigQueryConnectOptions() + .toBuilder() + .setProjectId(projectId) + .setDataset(dataset) + .setTable(table) + .build(); + }) + // in case no query configured, just return the configured options. + .orElse(this.readOptions.getBigQueryConnectOptions()); + } + + public void open() { + LOG.info("BigQuery source split assigner is opening."); + if (!initialized) { + BigQueryConnectOptions connectionOptions = + checkOptionsAndRunQueryIfNeededReturningModifiedOptions(); + try (BigQueryServices.StorageReadClient client = + BigQueryServicesFactory.instance(connectionOptions).storageRead()) { + String parent = String.format("projects/%s", connectionOptions.getProjectId()); + + String srcTable = + String.format( + "projects/%s/datasets/%s/tables/%s", + connectionOptions.getProjectId(), + connectionOptions.getDataset(), + connectionOptions.getTable()); + + // We specify the columns to be projected by adding them to the selected fields, + // and set a simple filter to restrict which rows are transmitted. + TableReadOptions.Builder optionsBuilder = TableReadOptions.newBuilder(); + + readOptions + .getColumnNames() + .forEach(name -> optionsBuilder.addSelectedFields(name)); + optionsBuilder.setRowRestriction(readOptions.getRowRestriction()); + + TableReadOptions options = optionsBuilder.build(); + + // Start specifying the read session we want created. + ReadSession.Builder sessionBuilder = + ReadSession.newBuilder() + .setTable(srcTable) + .setDataFormat(DataFormat.AVRO) + .setReadOptions(options); + + // Optionally specify the snapshot time. When unspecified, snapshot time is "now". + if (readOptions.getSnapshotTimestampInMillis() != null) { + Timestamp t = + Timestamp.newBuilder() + .setSeconds(readOptions.getSnapshotTimestampInMillis() / 1000) + .setNanos( + (int) + ((readOptions.getSnapshotTimestampInMillis() + % 1000) + * 1000000)) + .build(); + TableModifiers modifiers = + TableModifiers.newBuilder().setSnapshotTime(t).build(); + sessionBuilder.setTableModifiers(modifiers); + } + + // Begin building the session creation request. + CreateReadSessionRequest.Builder builder = + CreateReadSessionRequest.newBuilder() + .setParent(parent) + .setReadSession(sessionBuilder) + .setMaxStreamCount(readOptions.getMaxStreamCount()); + + // request the session + ReadSession session = client.createReadSession(builder.build()); + LOG.info( + "BigQuery Storage Read session, name: {}," + + " estimated row count {}, estimated scanned bytes {}," + + " streams count {}, expired time {} (seconds after epoch).", + session.getName(), + session.getEstimatedRowCount(), + session.getEstimatedTotalBytesScanned(), + session.getStreamsCount(), + session.getExpireTime().getSeconds()); + // get all the stream names added to the initialized state + remainingTableStreams.addAll( + session.getStreamsList().stream() + .map(stream -> stream.getName()) + .collect(Collectors.toList())); + initialized = true; + } catch (IOException ex) { + throw new RuntimeException( + "Problems creating the BigQuery Storage Read session.", ex); + } + } + } + + public void addSplitsBack(List splits) { + for (BigQuerySourceSplit split : splits) { + remainingSourceSplits.add((BigQuerySourceSplit) split); + // we should remove the add-backed splits from the assigned list, + // because they are failed + assignedSourceSplits.remove(split.splitId()); + } + } + + public BigQuerySourceEnumState snapshotState(long checkpointId) { + return new BigQuerySourceEnumState( + Lists.newArrayList(remainingTableStreams), + alreadyProcessedTableStreams, + Lists.newArrayList(remainingSourceSplits), + assignedSourceSplits, + initialized); + } + + public void close() { + // so far not much to be done here + LOG.info("BigQuery source split assigner is closed."); + } + + public Optional getNext() { + if (!remainingSourceSplits.isEmpty()) { + // return remaining splits firstly + BigQuerySourceSplit split = remainingSourceSplits.poll(); + assignedSourceSplits.put(split.splitId(), split); + return Optional.of(split); + } else { + // it's turn for next collection + String nextStream = remainingTableStreams.poll(); + if (nextStream != null) { + BigQuerySourceSplit split = new BigQuerySourceSplit(nextStream); + remainingSourceSplits.add(split); + alreadyProcessedTableStreams.add(nextStream); + return getNext(); + } else { + return Optional.empty(); + } + } + } + + public boolean noMoreSplits() { + checkState(initialized, "The noMoreSplits method was called but not initialized."); + return remainingTableStreams.isEmpty() && remainingSourceSplits.isEmpty(); + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReader.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReader.java new file mode 100644 index 00000000..a5840925 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReader.java @@ -0,0 +1,335 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.split.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.RecordsBySplits; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.dropwizard.metrics.DropwizardHistogramWrapper; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.util.Preconditions; + +import com.codahale.metrics.SlidingWindowReservoir; +import com.google.cloud.bigquery.storage.v1.AvroRows; +import com.google.cloud.bigquery.storage.v1.ReadRowsRequest; +import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; +import com.google.cloud.flink.bigquery.services.BigQueryServices; +import com.google.cloud.flink.bigquery.services.BigQueryServicesFactory; +import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; +import com.google.cloud.flink.bigquery.source.reader.BigQuerySourceReaderContext; +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplit; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DecoderFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.Queue; +import java.util.stream.Collectors; + +/** A split reader for {@link BigQuerySourceSplit}. */ +@Internal +public class BigQuerySourceSplitReader implements SplitReader { + private static final Logger LOG = LoggerFactory.getLogger(BigQuerySourceSplitReader.class); + + private final BigQueryReadOptions readOptions; + private final BigQuerySourceReaderContext readerContext; + private final transient Optional readSplitTimeMetric; + private final Queue assignedSplits = new ArrayDeque<>(); + + private Boolean closed = false; + private Schema avroSchema = null; + private Integer readSoFar = 0; + private Long splitStartFetch; + private Iterator readStreamIterator = null; + + public BigQuerySourceSplitReader( + BigQueryReadOptions readOptions, BigQuerySourceReaderContext readerContext) { + this.readOptions = readOptions; + this.readerContext = readerContext; + this.readSplitTimeMetric = + Optional.ofNullable(readerContext.metricGroup()) + .map( + mgroup -> + mgroup.histogram( + "bq.split.read.time.ms", + new DropwizardHistogramWrapper( + new com.codahale.metrics.Histogram( + new SlidingWindowReservoir(500))))); + } + + Integer offsetToFetch(BigQuerySourceSplit split) { + // honor what is coming as checkpointed + if (split.getOffset() > 0) { + readSoFar = split.getOffset(); + splitStartFetch = System.currentTimeMillis(); + } else if (readSoFar == 0) { + // will start reading the stream from the beginning + splitStartFetch = System.currentTimeMillis(); + } + LOG.debug( + "[subtask #{}] Offset to fetch from {} for stream {}.", + readerContext.getIndexOfSubtask(), + readSoFar, + split.getStreamName()); + return readSoFar; + } + + BigQueryServices.BigQueryServerStream retrieveReadStream( + BigQuerySourceSplit split) throws IOException { + try (BigQueryServices.StorageReadClient client = + BigQueryServicesFactory.instance(readOptions.getBigQueryConnectOptions()) + .storageRead()) { + ReadRowsRequest readRequest = + ReadRowsRequest.newBuilder() + .setReadStream(split.getStreamName()) + .setOffset(offsetToFetch(split)) + .build(); + + return client.readRows(readRequest); + } catch (Exception ex) { + throw new IOException( + String.format( + "[subtask #%d] Problems while opening the stream %s from BigQuery" + + " with connection info %s. Current split offset %d," + + " reader offset %d.", + readerContext.getIndexOfSubtask(), + Optional.ofNullable(split.getStreamName()).orElse("NA"), + readOptions.toString(), + split.getOffset(), + readSoFar), + ex); + } + } + + @Override + public RecordsWithSplitIds fetch() throws IOException { + if (closed) { + throw new IllegalStateException("Can't fetch records from a closed split reader."); + } + + RecordsBySplits.Builder respBuilder = new RecordsBySplits.Builder<>(); + + // nothing to read has been assigned + if (assignedSplits.isEmpty()) { + return respBuilder.build(); + } + + // return when current read count is already over limit + if (readerContext.willItBeOverLimit(0)) { + LOG.info( + "Completing reading because we are over limit (context reader count {}).", + readerContext.currentReadCount()); + respBuilder.addFinishedSplits( + assignedSplits.stream() + .map(split -> split.splitId()) + .collect(Collectors.toList())); + assignedSplits.clear(); + return respBuilder.build(); + } + + BigQuerySourceSplit assignedSplit = assignedSplits.peek(); + int maxRecordsPerSplitFetch = readOptions.getMaxRecordsPerSplitFetch(); + int read = 0; + Long fetchStartTime = System.currentTimeMillis(); + Boolean truncated = false; + + try { + if (readStreamIterator == null) { + readStreamIterator = retrieveReadStream(assignedSplit).iterator(); + } + Long itStartTime = System.currentTimeMillis(); + while (readStreamIterator.hasNext()) { + ReadRowsResponse response = readStreamIterator.next(); + if (!response.hasAvroRows()) { + LOG.info( + "[subtask #{}] The response contained no avro records for stream {}.", + readerContext.getIndexOfSubtask(), + assignedSplit.getStreamName()); + } + if (avroSchema == null && response.hasAvroSchema()) { + // this will happen only the first time we read from a particular stream + avroSchema = new Schema.Parser().parse(response.getAvroSchema().getSchema()); + } else if (avroSchema == null && !response.hasAvroSchema()) { + throw new IllegalArgumentException( + "Avro schema not initialized and not available in the response."); + } + Long decodeStart = System.currentTimeMillis(); + List recordList = + GenericRecordReader.create(avroSchema).processRows(response.getAvroRows()); + Long decodeTimeMS = System.currentTimeMillis() - decodeStart; + LOG.debug( + "[subtask #{}] Iteration decoded records in {}ms from stream {}.", + readerContext.getIndexOfSubtask(), + decodeTimeMS, + assignedSplit.getStreamName()); + + for (GenericRecord record : recordList) { + respBuilder.add(assignedSplit, record); + read++; + // check if the read count will be over the limit + if (readerContext.willItBeOverLimit(read)) { + break; + } + } + // check if the read count will be over the limit + if (readerContext.willItBeOverLimit(read)) { + break; + } + Long itTimeMs = System.currentTimeMillis() - itStartTime; + LOG.debug( + "[subtask #{}] Completed reading iteration in {}ms," + + " so far read {} from stream {}.", + readerContext.getIndexOfSubtask(), + itTimeMs, + readSoFar + read, + assignedSplit.getStreamName()); + itStartTime = System.currentTimeMillis(); + /** + * Assuming the record list from the read session have the same size (true in most + * cases but the last one in the response stream) we check if we will be going over + * the per fetch limit, in that case we break the loop and return the partial + * results (enabling the checkpointing of the partial retrieval if wanted by the + * runtime). The read response record count has been observed to have 1024 elements. + */ + if (read + recordList.size() > maxRecordsPerSplitFetch) { + truncated = true; + break; + } + } + readSoFar += read; + // check if we finished to read the stream to finalize the split + if (!truncated) { + readerContext.updateReadCount(read); + Long splitTimeMs = System.currentTimeMillis() - splitStartFetch; + this.readSplitTimeMetric.ifPresent(m -> m.update(splitTimeMs)); + LOG.info( + "[subtask #{}] Completed reading split, {} records in {}ms on stream {}.", + readerContext.getIndexOfSubtask(), + readSoFar, + splitTimeMs, + assignedSplit.splitId()); + readSoFar = 0; + assignedSplits.poll(); + readStreamIterator = null; + respBuilder.addFinishedSplit(assignedSplit.splitId()); + } else { + Long fetchTimeMs = System.currentTimeMillis() - fetchStartTime; + LOG.debug( + "[subtask #{}] Completed a partial fetch in {}ms," + + " so far read {} from stream {}.", + readerContext.getIndexOfSubtask(), + fetchTimeMs, + readSoFar, + assignedSplit.getStreamName()); + } + return respBuilder.build(); + } catch (Exception ex) { + throw new IOException( + String.format( + "[subtask #%d] Problems while reading stream %s from BigQuery" + + " with connection info %s. Current split offset %d," + + " reader offset %d.", + readerContext.getIndexOfSubtask(), + Optional.ofNullable(assignedSplit.getStreamName()).orElse("NA"), + readOptions.toString(), + assignedSplit.getOffset(), + readSoFar), + ex); + } + } + + @Override + public void handleSplitsChanges(SplitsChange splitsChanges) { + LOG.debug("Handle split changes {}.", splitsChanges); + + if (!(splitsChanges instanceof SplitsAddition)) { + throw new UnsupportedOperationException( + String.format( + "The SplitChange type of %s is not supported.", + splitsChanges.getClass())); + } + + assignedSplits.addAll(splitsChanges.splits()); + } + + @Override + public void wakeUp() { + LOG.debug("[subtask #{}] Wake up called.", readerContext.getIndexOfSubtask()); + // do nothing, for now + } + + @Override + public void close() throws Exception { + LOG.debug( + "[subtask #{}] Close called, assigned splits {}.", + readerContext.getIndexOfSubtask(), + assignedSplits.toString()); + if (!closed) { + closed = true; + readSoFar = 0; + readStreamIterator = null; + // complete closing with what may be needed + } + } + + static class GenericRecordReader { + + private final Schema schema; + + private GenericRecordReader(Schema schema) { + Preconditions.checkNotNull(schema, "The provided avro schema reference is null."); + this.schema = schema; + } + + public static GenericRecordReader create(Schema schema) { + return new GenericRecordReader(schema); + } + + /** + * Method for processing AVRO rows which only validates decoding. + * + * @param avroRows object returned from the ReadRowsResponse. + */ + public List processRows(AvroRows avroRows) throws IOException { + BinaryDecoder decoder = + DecoderFactory.get() + .binaryDecoder(avroRows.getSerializedBinaryRows().toByteArray(), null); + DatumReader datumReader = new GenericDatumReader<>(schema); + List records = new ArrayList<>(); + GenericRecord row; + while (!decoder.isEnd()) { + // Reusing object row + row = datumReader.read(null, decoder); + records.add(row); + } + return records; + } + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java new file mode 100644 index 00000000..aa0b09a6 --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java @@ -0,0 +1,336 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.fakes; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.util.function.SerializableFunction; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.StandardSQLTypeName; +import com.google.cloud.bigquery.storage.v1.AvroRows; +import com.google.cloud.bigquery.storage.v1.AvroSchema; +import com.google.cloud.bigquery.storage.v1.CreateReadSessionRequest; +import com.google.cloud.bigquery.storage.v1.DataFormat; +import com.google.cloud.bigquery.storage.v1.ReadRowsRequest; +import com.google.cloud.bigquery.storage.v1.ReadRowsResponse; +import com.google.cloud.bigquery.storage.v1.ReadSession; +import com.google.cloud.bigquery.storage.v1.ReadStream; +import com.google.cloud.bigquery.storage.v1.StreamStats; +import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; +import com.google.cloud.flink.bigquery.common.config.CredentialsOptions; +import com.google.cloud.flink.bigquery.services.BigQueryServices; +import com.google.cloud.flink.bigquery.services.QueryResultInfo; +import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; +import com.google.protobuf.ByteString; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.Encoder; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.util.RandomData; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.Random; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** Utility class to generate mocked objects for the BQ storage client. */ +public class StorageClientFaker { + + /** Implementation for the BigQuery services for testing purposes. */ + public static class FakeBigQueryServices implements BigQueryServices { + + private final FakeBigQueryStorageReadClient storageReadClient; + + public FakeBigQueryServices(FakeBigQueryStorageReadClient storageReadClient) { + this.storageReadClient = storageReadClient; + } + + @Override + public StorageReadClient getStorageClient(CredentialsOptions readOptions) + throws IOException { + return storageReadClient; + } + + @Override + public QueryDataClient getQueryDataClient(CredentialsOptions readOptions) { + return new QueryDataClient() { + @Override + public List retrieveTablePartitions( + String project, String dataset, String table) { + return new ArrayList<>(); + } + + @Override + public Optional> retrievePartitionColumnName( + String project, String dataset, String table) { + return Optional.empty(); + } + + @Override + public TableSchema getTableSchema(String project, String dataset, String table) { + return new TableSchema(); + } + + @Override + public Optional runQuery(String projectId, String query) { + return Optional.of(QueryResultInfo.succeed("", "", "")); + } + + @Override + public Job dryRunQuery(String projectId, String query) { + return null; + } + }; + } + + /** Implementation of the server stream for testing purposes. */ + public static class FakeBigQueryServerStream + implements BigQueryServices.BigQueryServerStream { + + private final List toReturn; + + public FakeBigQueryServerStream( + SerializableFunction> dataGenerator, + String schema, + String dataPrefix, + Long size, + Long offset) { + this.toReturn = + createResponse( + schema, + dataGenerator + .apply(new RecordGenerationParams(schema, size.intValue())) + .stream() + .skip(offset) + .collect(Collectors.toList()), + 0, + size); + } + + @Override + public Iterator iterator() { + return toReturn.iterator(); + } + + @Override + public void cancel() {} + } + + /** Implementation for the storage read client for testing purposes. */ + public static class FakeBigQueryStorageReadClient implements StorageReadClient { + + private final ReadSession session; + private final SerializableFunction> + dataGenerator; + + public FakeBigQueryStorageReadClient( + ReadSession session, + SerializableFunction> + dataGenerator) { + this.session = session; + this.dataGenerator = dataGenerator; + } + + @Override + public ReadSession createReadSession(CreateReadSessionRequest request) { + return session; + } + + @Override + public BigQueryServerStream readRows(ReadRowsRequest request) { + try { + // introduce some random delay + Thread.sleep(new Random().nextInt(500)); + } catch (InterruptedException ex) { + } + return new FakeBigQueryServerStream( + dataGenerator, + session.getAvroSchema().getSchema(), + request.getReadStream(), + session.getEstimatedRowCount(), + request.getOffset()); + } + + @Override + public void close() {} + } + } + + public static final String SIMPLE_AVRO_SCHEMA_STRING = + "{\"namespace\": \"example.avro\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"RowRecord\",\n" + + " \"fields\": [\n" + + " {\"name\": \"name\", \"type\": \"string\"},\n" + + " {\"name\": \"number\", \"type\": \"long\"}\n" + + " ]\n" + + "}"; + + public static final Schema SIMPLE_AVRO_SCHEMA = + new Schema.Parser().parse(SIMPLE_AVRO_SCHEMA_STRING); + + /** Represents the parameters needed for the Avro data generation. */ + public static class RecordGenerationParams implements Serializable { + private final String avroSchemaString; + private final Integer recordCount; + + public RecordGenerationParams(String avroSchemaString, Integer recordCount) { + this.avroSchemaString = avroSchemaString; + this.recordCount = recordCount; + } + + public String getAvroSchemaString() { + return avroSchemaString; + } + + public Integer getRecordCount() { + return recordCount; + } + } + + public static ReadSession fakeReadSession( + Integer expectedRowCount, Integer expectedReadStreamCount, String avroSchemaString) { + // setup the response for read session request + List readStreams = + IntStream.range(0, expectedReadStreamCount) + .mapToObj(i -> ReadStream.newBuilder().setName("stream" + i).build()) + .collect(Collectors.toList()); + return ReadSession.newBuilder() + .addAllStreams(readStreams) + .setEstimatedRowCount(expectedRowCount) + .setDataFormat(DataFormat.AVRO) + .setAvroSchema(AvroSchema.newBuilder().setSchema(avroSchemaString)) + .build(); + } + + public static List createRecordList(RecordGenerationParams params) { + Schema schema = new Schema.Parser().parse(params.getAvroSchemaString()); + return IntStream.range(0, params.getRecordCount()) + .mapToObj(i -> createRecord(schema)) + .collect(Collectors.toList()); + } + + public static GenericRecord createRecord(Schema schema) { + return (GenericRecord) new RandomData(schema, 0).iterator().next(); + } + + private static final EncoderFactory ENCODER_FACTORY = EncoderFactory.get(); + + @SuppressWarnings("deprecation") + public static List createResponse( + String schemaString, + List genericRecords, + double progressAtResponseStart, + double progressAtResponseEnd) { + // BigQuery delivers the data in 1024 elements chunks, so we partition the generated list + // into multiple ones with that size max. + List> responsesData = Lists.partition(genericRecords, 1024); + + return responsesData.stream() + // for each data response chunk we generate a read response object + .map( + genRecords -> { + try { + Schema schema = new Schema.Parser().parse(schemaString); + GenericDatumWriter writer = + new GenericDatumWriter<>(schema); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + Encoder binaryEncoder = + ENCODER_FACTORY.binaryEncoder(outputStream, null); + for (GenericRecord genericRecord : genRecords) { + writer.write(genericRecord, binaryEncoder); + } + + binaryEncoder.flush(); + + return ReadRowsResponse.newBuilder() + .setAvroRows( + AvroRows.newBuilder() + .setSerializedBinaryRows( + ByteString.copyFrom( + outputStream.toByteArray())) + .setRowCount(genRecords.size())) + .setAvroSchema( + AvroSchema.newBuilder() + .setSchema(schema.toString()) + .build()) + .setRowCount(genRecords.size()) + .setStats( + StreamStats.newBuilder() + .setProgress( + StreamStats.Progress.newBuilder() + .setAtResponseStart( + progressAtResponseStart) + .setAtResponseEnd( + progressAtResponseEnd))) + .build(); + } catch (Exception ex) { + throw new RuntimeException( + "Problems generating faked response.", ex); + } + }) + .collect(Collectors.toList()); + } + + public static BigQueryReadOptions createReadOptions( + Integer expectedRowCount, Integer expectedReadStreamCount, String avroSchemaString) + throws IOException { + return createReadOptions( + expectedRowCount, + expectedReadStreamCount, + avroSchemaString, + params -> StorageClientFaker.createRecordList(params)); + } + + public static BigQueryReadOptions createReadOptions( + Integer expectedRowCount, + Integer expectedReadStreamCount, + String avroSchemaString, + SerializableFunction> dataGenerator) + throws IOException { + return BigQueryReadOptions.builder() + .setBigQueryConnectOptions( + BigQueryConnectOptions.builder() + .setDataset("dataset") + .setProjectId("project") + .setTable("table") + .setCredentialsOptions(null) + .setTestingBigQueryServices( + () -> { + return new StorageClientFaker.FakeBigQueryServices( + new StorageClientFaker.FakeBigQueryServices + .FakeBigQueryStorageReadClient( + StorageClientFaker.fakeReadSession( + expectedRowCount, + expectedReadStreamCount, + avroSchemaString), + dataGenerator)); + }) + .build()) + .build(); + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/QueryResultInfoTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/QueryResultInfoTest.java index e3f7a87d..90260de1 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/QueryResultInfoTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/QueryResultInfoTest.java @@ -50,4 +50,12 @@ public void testNotEquals() { QueryResultInfo failed = QueryResultInfo.failed(Lists.newArrayList()); Assertions.assertThat(succeed).isNotEqualTo(failed); } + + @Test + public void testEquals() { + QueryResultInfo succeed = QueryResultInfo.succeed("", "", ""); + QueryResultInfo another = QueryResultInfo.succeed("", "", ""); + Assertions.assertThat(succeed).isEqualTo(another); + Assertions.assertThat(succeed.hashCode()).isEqualTo(another.hashCode()); + } } diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssignerTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssignerTest.java new file mode 100644 index 00000000..23aa6b56 --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssignerTest.java @@ -0,0 +1,79 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.split; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import com.google.cloud.flink.bigquery.fakes.StorageClientFaker; +import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; +import com.google.cloud.flink.bigquery.source.enumerator.BigQuerySourceEnumState; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Optional; + +/** */ +public class BigQuerySourceSplitAssignerTest { + + private BigQueryReadOptions readOptions; + + @Before + public void beforeTest() throws IOException { + this.readOptions = + StorageClientFaker.createReadOptions( + 0, 2, StorageClientFaker.SIMPLE_AVRO_SCHEMA_STRING); + } + + @Test + public void testAssignment() { + // initialize the assigner with default options since we are faking the bigquery services + BigQuerySourceSplitAssigner assigner = + new BigQuerySourceSplitAssigner( + this.readOptions, BigQuerySourceEnumState.initialState()); + // request the retrieval of the bigquery table info + assigner.open(); + + // should retrieve the first split representing the firt stream + Optional maybeSplit = assigner.getNext(); + Assert.assertTrue(maybeSplit.isPresent()); + // should retrieve the second split representing the second stream + maybeSplit = assigner.getNext(); + Assert.assertTrue(maybeSplit.isPresent()); + BigQuerySourceSplit split = maybeSplit.get(); + // no more splits should be available + maybeSplit = assigner.getNext(); + Assert.assertTrue(!maybeSplit.isPresent()); + Assert.assertTrue(assigner.noMoreSplits()); + // lets check on the enum state + BigQuerySourceEnumState state = assigner.snapshotState(0); + Assert.assertTrue(state.getRemaniningTableStreams().isEmpty()); + Assert.assertTrue(state.getRemainingSourceSplits().isEmpty()); + // add some splits back + assigner.addSplitsBack(Lists.newArrayList(split)); + // check again on the enum state + state = assigner.snapshotState(0); + Assert.assertTrue(state.getRemaniningTableStreams().isEmpty()); + Assert.assertTrue(!state.getRemainingSourceSplits().isEmpty()); + // empty it again and check + assigner.getNext(); + maybeSplit = assigner.getNext(); + Assert.assertTrue(!maybeSplit.isPresent()); + Assert.assertTrue(assigner.noMoreSplits()); + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReaderTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReaderTest.java new file mode 100644 index 00000000..fda7634c --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReaderTest.java @@ -0,0 +1,135 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.split.reader; + +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import com.google.cloud.flink.bigquery.fakes.StorageClientFaker; +import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; +import com.google.cloud.flink.bigquery.source.reader.BigQuerySourceReaderContext; +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplit; +import org.apache.avro.generic.GenericRecord; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; + +/** */ +public class BigQuerySourceSplitReaderTest { + + @Test + public void testSplitReaderSmall() throws IOException { + // init the read options for BQ + BigQueryReadOptions readOptions = + StorageClientFaker.createReadOptions( + 10, 2, StorageClientFaker.SIMPLE_AVRO_SCHEMA_STRING); + SourceReaderContext readerContext = Mockito.mock(SourceReaderContext.class); + BigQuerySourceReaderContext context = new BigQuerySourceReaderContext(readerContext, 10); + BigQuerySourceSplitReader reader = new BigQuerySourceSplitReader(readOptions, context); + // wake the thing up + reader.wakeUp(); + + String splitName = "stream1"; + BigQuerySourceSplit split = new BigQuerySourceSplit(splitName, 0); + BigQuerySourceSplit split2 = new BigQuerySourceSplit("stream2", 0); + SplitsAddition change = + new SplitsAddition<>(Lists.newArrayList(split, split2)); + + // send an assignment + reader.handleSplitsChanges(change); + + // this should fetch us some data + RecordsWithSplitIds records = reader.fetch(); + // there is one finished split and is named stream1 + Assert.assertTrue(records.finishedSplits().size() == 1); + + String firstSplit = records.nextSplit(); + Assert.assertNotNull(firstSplit); + Assert.assertTrue(firstSplit.equals(splitName)); + + int i = 0; + while (records.nextRecordFromSplit() != null) { + i++; + } + // there were 10 generic records read + Assert.assertTrue(i == 10); + // there are no more splits + Assert.assertNull(records.nextSplit()); + + // now there should be another split to process + records = reader.fetch(); + Assert.assertTrue(!records.finishedSplits().isEmpty()); + + // after processing no more splits can be retrieved + records = reader.fetch(); + Assert.assertTrue(records.finishedSplits().isEmpty()); + } + + @Test + public void testSplitReaderMultipleFetch() throws IOException { + Integer totalRecordCount = 15000; + // init the read options for BQ + BigQueryReadOptions readOptions = + StorageClientFaker.createReadOptions( + totalRecordCount, 1, StorageClientFaker.SIMPLE_AVRO_SCHEMA_STRING); + SourceReaderContext readerContext = Mockito.mock(SourceReaderContext.class); + // no limits in the read + BigQuerySourceReaderContext context = new BigQuerySourceReaderContext(readerContext, -1); + BigQuerySourceSplitReader reader = new BigQuerySourceSplitReader(readOptions, context); + // wake the thing up + reader.wakeUp(); + + String splitName = "stream1"; + BigQuerySourceSplit split = new BigQuerySourceSplit(splitName, 0); + SplitsAddition change = + new SplitsAddition<>(Lists.newArrayList(split)); + + // send an assignment + reader.handleSplitsChanges(change); + + // this should fetch us some data + RecordsWithSplitIds records = reader.fetch(); + // there shouldn't be a finished split + Assert.assertTrue(records.finishedSplits().isEmpty()); + + String firstPartialSplit = records.nextSplit(); + Assert.assertNotNull(firstPartialSplit); + Assert.assertTrue(firstPartialSplit.equals(splitName)); + + int i = 0; + while (records.nextRecordFromSplit() != null) { + i++; + } + // there were less than 10000 generic records read, the max per fetch + Assert.assertTrue(i <= 10000); + // there are no more splits + Assert.assertNull(records.nextSplit()); + + // now there should be more data in the split and now should be able to finalize it + records = reader.fetch(); + Assert.assertTrue(!records.finishedSplits().isEmpty()); + + // after processing no more splits can be retrieved + records = reader.fetch(); + Assert.assertTrue(records.finishedSplits().isEmpty()); + } +} From 0567b586574686f85c37675aa7a137792bab6a17 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Wed, 28 Jun 2023 15:24:30 -0700 Subject: [PATCH 09/34] applying recommendations from sonartype-lift --- .../cloud/flink/bigquery/common/utils/SchemaTransform.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java index 4400fa51..6b769442 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java @@ -105,7 +105,7 @@ private static boolean hasNamespaceCollision(List fieldSchemas List fieldsToCheck = new ArrayList<>(); for (fieldsToCheck.addAll(fieldSchemas); !fieldsToCheck.isEmpty(); ) { TableFieldSchema field = fieldsToCheck.remove(0); - if ("STRUCT".equals(field.getType()) || "RECORD".equals(field.getType())) { + if (field.getType().equals("STRUCT") || field.getType().equals("RECORD")) { if (recordTypeFieldNames.contains(field.getName())) { return true; } @@ -141,11 +141,11 @@ private static Schema.Field convertField(TableFieldSchema bigQueryField, String elementSchema = handleAvroLogicalTypes(bigQueryField, avroType); } Schema fieldSchema; - if (bigQueryField.getMode() == null || "NULLABLE".equals(bigQueryField.getMode())) { + if (bigQueryField.getMode() == null || bigQueryField.getMode().equals("NULLABLE")) { fieldSchema = Schema.createUnion(Schema.create(Schema.Type.NULL), elementSchema); } else if (Objects.equals(bigQueryField.getMode(), "REQUIRED")) { fieldSchema = elementSchema; - } else if ("REPEATED".equals(bigQueryField.getMode())) { + } else if (bigQueryField.getMode().equals("REPEATED")) { fieldSchema = Schema.createArray(elementSchema); } else { throw new IllegalArgumentException( From 9006714f2aedc18e2af0ea77081364d118d026a1 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Wed, 28 Jun 2023 16:55:14 -0700 Subject: [PATCH 10/34] adding the Datastream source implementation for BigQuery --- flink-connector-bigquery/pom.xml | 2 +- .../flink/bigquery/source/BigQuerySource.java | 331 ++++++++++++++++++ .../source/emitter/BigQueryRecordEmitter.java | 73 ++++ .../enumerator/BigQuerySourceEnumerator.java | 125 +++++++ .../source/reader/BigQuerySourceReader.java | 90 +++++ .../AvroDeserializationSchema.java | 51 +++ .../AvroToRowDataDeserializationSchema.java | 51 +++ .../BigQueryDeserializationSchema.java | 63 ++++ .../bigquery/fakes/StorageClientFaker.java | 19 +- .../bigquery/source/BigQuerySourceITCase.java | 172 +++++++++ .../src/test/resources/log4j2-test.properties | 28 ++ pom.xml | 4 +- 12 files changed, 1003 insertions(+), 6 deletions(-) create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/BigQuerySource.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/emitter/BigQueryRecordEmitter.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumerator.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/BigQuerySourceReader.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/AvroDeserializationSchema.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/AvroToRowDataDeserializationSchema.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/BigQueryDeserializationSchema.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceITCase.java create mode 100644 flink-connector-bigquery/src/test/resources/log4j2-test.properties diff --git a/flink-connector-bigquery/pom.xml b/flink-connector-bigquery/pom.xml index 71e18607..2f809db1 100644 --- a/flink-connector-bigquery/pom.xml +++ b/flink-connector-bigquery/pom.xml @@ -216,7 +216,7 @@ under the License. report - prepare-package + install report diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/BigQuerySource.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/BigQuerySource.java new file mode 100644 index 00000000..d13882a5 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/BigQuerySource.java @@ -0,0 +1,331 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import com.google.api.services.bigquery.model.TableSchema; +import com.google.auto.value.AutoValue; +import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; +import com.google.cloud.flink.bigquery.common.utils.SchemaTransform; +import com.google.cloud.flink.bigquery.services.BigQueryServicesFactory; +import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; +import com.google.cloud.flink.bigquery.source.emitter.BigQueryRecordEmitter; +import com.google.cloud.flink.bigquery.source.enumerator.BigQuerySourceEnumState; +import com.google.cloud.flink.bigquery.source.enumerator.BigQuerySourceEnumStateSerializer; +import com.google.cloud.flink.bigquery.source.enumerator.BigQuerySourceEnumerator; +import com.google.cloud.flink.bigquery.source.reader.BigQuerySourceReader; +import com.google.cloud.flink.bigquery.source.reader.BigQuerySourceReaderContext; +import com.google.cloud.flink.bigquery.source.reader.deserializer.AvroDeserializationSchema; +import com.google.cloud.flink.bigquery.source.reader.deserializer.BigQueryDeserializationSchema; +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplit; +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplitAssigner; +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplitSerializer; +import com.google.cloud.flink.bigquery.source.split.reader.BigQuerySourceSplitReader; +import org.apache.avro.generic.GenericRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.function.Supplier; + +/** + * The DataStream {@link Source} implementation for Google BigQuery. It can be used to read data + * directly from a BigQuery table or read data from a BigQuery query execution. + * + *

The following example demonstrates how to configure the source to read {@link GenericRecord} + * data from a BigQuery table. + * + *

{@code
+ * BigQuerySource source =
+ *       BigQuerySource.readAvros(
+ *           BigQueryReadOptions.builder()
+ *             .setColumnNames(Lists.newArrayList("col1", "col2"))
+ *             .setRowRestriction(
+ *               "col2 BETWEEN '2023-06-01' AND '2023-06-02'")
+ *             .setBigQueryConnectOptions(
+ *               BigQueryConnectOptions.builder()
+ *                 .setProjectId("some-gcp-project")
+ *                 .setDataset("some-bq-dataset")
+ *                 .setTable("some-bq-table")
+ *                 .build())
+ *             .build(), 1000);
+ * }
+ * + *

Review the option classes and their builders for more details on the configurable options. + * + *

The following example demonstrates how to configure the Source to read {@link GenericRecord} + * data from the results of a BigQuery query execution. + * + *

{@code
+ * BigQuerySource bqSource =
+ *         BigQuerySource.readAvrosFromQuery(
+ *                 "SELECT * FROM some_dataset.INFORMATION_SCHEMA.PARTITIONS",
+ *                 "some_gcp_project");
+ * }
+ * + * @param The type of the data returned by this source implementation. + */ +@AutoValue +@PublicEvolving +public abstract class BigQuerySource + implements Source, + ResultTypeQueryable { + private static final Logger LOG = LoggerFactory.getLogger(BigQuerySource.class); + + public abstract BigQueryDeserializationSchema getDeserializationSchema(); + + public abstract BigQueryReadOptions getReadOptions(); + + @Nullable + public abstract Integer getLimit(); + + @Override + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return BigQuerySourceSplitSerializer.INSTANCE; + } + + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return BigQuerySourceEnumStateSerializer.INSTANCE; + } + + @Override + public TypeInformation getProducedType() { + return getDeserializationSchema().getProducedType(); + } + + @Override + public SourceReader createReader(SourceReaderContext readerContext) + throws Exception { + FutureCompletingBlockingQueue> elementsQueue = + new FutureCompletingBlockingQueue<>(); + + BigQuerySourceReaderContext bqReaderContext = + new BigQuerySourceReaderContext(readerContext, getLimit()); + + Supplier> splitReaderSupplier = + () -> new BigQuerySourceSplitReader(getReadOptions(), bqReaderContext); + + return new BigQuerySourceReader<>( + elementsQueue, + splitReaderSupplier, + new BigQueryRecordEmitter<>(getDeserializationSchema()), + bqReaderContext); + } + + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) throws Exception { + BigQuerySourceEnumState initialState = BigQuerySourceEnumState.initialState(); + BigQuerySourceSplitAssigner assigner = + new BigQuerySourceSplitAssigner(getReadOptions(), initialState); + return new BigQuerySourceEnumerator(getBoundedness(), enumContext, assigner); + } + + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, + BigQuerySourceEnumState checkpoint) + throws Exception { + LOG.debug("Restoring enumerator with state {}", checkpoint); + BigQuerySourceSplitAssigner splitAssigner = + new BigQuerySourceSplitAssigner(getReadOptions(), checkpoint); + return new BigQuerySourceEnumerator(getBoundedness(), enumContext, splitAssigner); + } + + /** + * Transforms the instance into a builder instance for property modification. + * + * @return A {@link Builder} instance for the type. + */ + public abstract Builder toBuilder(); + + /** + * Creates an instance of this class builder. + * + * @param The expected return type of this source. + * @return the BigQuerySource builder instance. + */ + public static Builder builder() { + return new AutoValue_BigQuerySource.Builder().setLimit(-1); + } + + /** + * Creates an instance of the source, setting Avro {@link GenericRecord} as the return type for + * the data (mimicking the table's schema), limiting the record retrieval to the provided limit + * and reading data from the provided query which will be executed using the provided GCP + * project. + * + * @param query A BigQuery standard SQL query. + * @param gcpProject The GCP project where the provided query will execute. + * @param limit the max quantity of records to be returned. + * @return A fully initialized instance of the source, ready to read {@link GenericRecord} from + * the BigQuery query results. + * @throws IOException + */ + public static BigQuerySource readAvrosFromQuery( + String query, String gcpProject, Integer limit) throws IOException { + BigQueryReadOptions readOptions = + BigQueryReadOptions.builder() + .setQueryAndExecutionProject(query, gcpProject) + .build(); + + BigQueryConnectOptions connectOptions = readOptions.getBigQueryConnectOptions(); + TableSchema tableSchema = + BigQueryServicesFactory.instance(connectOptions) + .queryClient() + .dryRunQuery(readOptions.getQueryExecutionProject(), readOptions.getQuery()) + .getStatistics() + .getQuery() + .getSchema(); + return BigQuerySource.builder() + .setDeserializationSchema( + new AvroDeserializationSchema( + SchemaTransform.toGenericAvroSchema( + "queryresultschema", tableSchema.getFields()) + .toString())) + .setLimit(limit) + .setReadOptions(readOptions) + .build(); + } + + /** + * Creates an instance of the source, setting Avro {@link GenericRecord} as the return type for + * the data (mimicking the table's schema) and reading data from the provided query which will + * be executed using the provided GCP project. + * + * @param query A BigQuery standard SQL query. + * @param gcpProject The GCP project where the provided query will execute. + * @return A fully initialized instance of the source, ready to read {@link GenericRecord} from + * the BigQuery query results. + * @throws IOException + */ + public static BigQuerySource readAvrosFromQuery(String query, String gcpProject) + throws IOException { + return readAvrosFromQuery(query, gcpProject, -1); + } + + /** + * Creates an instance of the source, limiting the record retrieval to the provided limit and + * setting Avro {@link GenericRecord} as the return type for the data (mimicking the table's + * schema). In case of projecting the columns of the table a new de-serialization schema should + * be provided (considering the new result projected schema). + * + * @param readOptions The read options for this source + * @param limit the max quantity of records to be returned. + * @return A fully initialized instance of the source, ready to read {@link GenericRecord} from + * the underlying table. + */ + public static BigQuerySource readAvros( + BigQueryReadOptions readOptions, Integer limit) { + BigQueryConnectOptions connectOptions = readOptions.getBigQueryConnectOptions(); + TableSchema tableSchema = + BigQueryServicesFactory.instance(connectOptions) + .queryClient() + .getTableSchema( + connectOptions.getProjectId(), + connectOptions.getDataset(), + connectOptions.getTable()); + return BigQuerySource.builder() + .setDeserializationSchema( + new AvroDeserializationSchema( + SchemaTransform.toGenericAvroSchema( + String.format( + "%s.%s.%s", + connectOptions.getProjectId(), + connectOptions.getDataset(), + connectOptions.getTable()), + tableSchema.getFields()) + .toString())) + .setLimit(limit) + .setReadOptions(readOptions) + .build(); + } + + /** + * Creates an instance of the source, setting Avro {@link GenericRecord} as the return type for + * the data (mimicking the table's schema). In case of projecting the columns of the table a new + * de-serialization schema should be provided (considering the new result projected schema). + * + * @param readOptions The read options for this source + * @return A fully initialized instance of the source, ready to read {@link GenericRecord} from + * the underlying table. + */ + public static BigQuerySource readAvros(BigQueryReadOptions readOptions) { + return readAvros(readOptions, -1); + } + + /** + * Builder class for {@link BigQuerySource}. + * + * @param The type of the data returned by this source implementation. + */ + @AutoValue.Builder + public abstract static class Builder { + /** + * Sets the de-serialization schema for BigQuery returned data (AVRO by default). + * + * @param deserSchema the de-serialization schema for BigQuery return type. + * @return the BigQuerySource builder instance. + */ + public abstract Builder setDeserializationSchema( + BigQueryDeserializationSchema deserSchema); + + /** + * Sets the BigQuery read options that configures the source's instance. + * + * @param options The instance of the BigQuery read options. + * @return the BigQuerySource builder instance. + */ + public abstract Builder setReadOptions(BigQueryReadOptions options); + + /** + * Sets the max element count returned by this source. + * + * @param limit The max element count returned by the source. + * @return the BigQuerySource builder instance. + */ + public abstract Builder setLimit(Integer limit); + + /** + * Creates an instance of the {@link BigQuerySource}. + * + * @return A fully initialized instance of the source. + */ + public abstract BigQuerySource build(); + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/emitter/BigQueryRecordEmitter.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/emitter/BigQueryRecordEmitter.java new file mode 100644 index 00000000..55801aae --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/emitter/BigQueryRecordEmitter.java @@ -0,0 +1,73 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.emitter; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.flink.util.Collector; + +import com.google.cloud.flink.bigquery.source.reader.deserializer.BigQueryDeserializationSchema; +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplitState; +import org.apache.avro.generic.GenericRecord; + +/** + * The {@link RecordEmitter} implementation for {@link BigQuerySourceReader} .We would always update + * the last consumed message id in this emitter. + * + * @param the emitted type. + */ +@Internal +public class BigQueryRecordEmitter + implements RecordEmitter { + + private final BigQueryDeserializationSchema deserializationSchema; + private final SourceOutputWrapper sourceOutputWrapper; + + public BigQueryRecordEmitter( + BigQueryDeserializationSchema deserializationSchema) { + this.deserializationSchema = deserializationSchema; + this.sourceOutputWrapper = new SourceOutputWrapper<>(); + } + + @Override + public void emitRecord( + GenericRecord record, SourceOutput output, BigQuerySourceSplitState splitState) + throws Exception { + // Update current offset. + splitState.updateOffset(); + // Sink the record to source output. + sourceOutputWrapper.setSourceOutput(output); + deserializationSchema.deserialize(record, sourceOutputWrapper); + } + + private static class SourceOutputWrapper implements Collector { + private SourceOutput sourceOutput; + + @Override + public void collect(T record) { + sourceOutput.collect(record); + } + + @Override + public void close() {} + + private void setSourceOutput(SourceOutput sourceOutput) { + this.sourceOutput = sourceOutput; + } + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumerator.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumerator.java new file mode 100644 index 00000000..d5f2c82e --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumerator.java @@ -0,0 +1,125 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; + +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplit; +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplitAssigner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.TreeSet; + +/** The enumerator class for {@link BigQuerySource}. */ +@Internal +public class BigQuerySourceEnumerator + implements SplitEnumerator { + + private static final Logger LOG = LoggerFactory.getLogger(BigQuerySourceEnumerator.class); + + private final Boundedness boundedness; + private final SplitEnumeratorContext context; + private final BigQuerySourceSplitAssigner splitAssigner; + private final TreeSet readersAwaitingSplit; + + public BigQuerySourceEnumerator( + Boundedness boundedness, + SplitEnumeratorContext context, + BigQuerySourceSplitAssigner splitAssigner) { + this.boundedness = boundedness; + this.context = context; + this.splitAssigner = splitAssigner; + this.readersAwaitingSplit = new TreeSet<>(); + } + + @Override + public void start() { + splitAssigner.open(); + } + + @Override + public void handleSplitRequest(int subtaskId, String requesterHostname) { + if (!context.registeredReaders().containsKey(subtaskId)) { + // reader failed between sending the request and now. skip this request. + return; + } + + readersAwaitingSplit.add(subtaskId); + assignSplits(); + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + LOG.debug("BigQuery Source Enumerator adds splits back: {}", splits); + splitAssigner.addSplitsBack(splits); + } + + @Override + public void addReader(int subtaskId) { + LOG.debug("Adding reader {} to BigQuerySourceEnumerator.", subtaskId); + } + + @Override + public BigQuerySourceEnumState snapshotState(long checkpointId) throws Exception { + BigQuerySourceEnumState state = splitAssigner.snapshotState(checkpointId); + LOG.debug("Checkpointing state {}", state); + return state; + } + + @Override + public void close() throws IOException { + splitAssigner.close(); + } + + private void assignSplits() { + final Iterator awaitingReader = readersAwaitingSplit.iterator(); + + while (awaitingReader.hasNext()) { + int nextAwaiting = awaitingReader.next(); + // if the reader that requested another split has failed in the meantime, remove + // it from the list of waiting readers + if (!context.registeredReaders().containsKey(nextAwaiting)) { + awaitingReader.remove(); + continue; + } + + Optional split = splitAssigner.getNext(); + if (split.isPresent()) { + final BigQuerySourceSplit bqSplit = split.get(); + context.assignSplit(bqSplit, nextAwaiting); + awaitingReader.remove(); + LOG.info("Assign split {} to subtask {}", bqSplit, nextAwaiting); + break; + } else if (splitAssigner.noMoreSplits() && boundedness == Boundedness.BOUNDED) { + LOG.info("All splits have been assigned"); + context.registeredReaders().keySet().forEach(context::signalNoMoreSplits); + break; + } else { + // there is no available splits by now, skip assigning + break; + } + } + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/BigQuerySourceReader.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/BigQuerySourceReader.java new file mode 100644 index 00000000..47661a80 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/BigQuerySourceReader.java @@ -0,0 +1,90 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; + +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplit; +import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplitState; +import org.apache.avro.generic.GenericRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.function.Supplier; + +/** + * The common BigQuery source reader for both ordered & unordered message consuming. + * + * @param The output message type for Flink. + */ +@Internal +public class BigQuerySourceReader + extends SingleThreadMultiplexSourceReaderBase< + GenericRecord, OUT, BigQuerySourceSplit, BigQuerySourceSplitState> { + private static final Logger LOG = LoggerFactory.getLogger(BigQuerySourceReader.class); + + public BigQuerySourceReader( + FutureCompletingBlockingQueue> elementsQueue, + Supplier> splitReaderSupplier, + RecordEmitter recordEmitter, + Configuration config, + SourceReaderContext context) { + super(elementsQueue, splitReaderSupplier, recordEmitter, config, context); + } + + public BigQuerySourceReader( + FutureCompletingBlockingQueue> elementsQueue, + Supplier> splitReaderSupplier, + RecordEmitter recordEmitter, + SourceReaderContext context) { + super(elementsQueue, splitReaderSupplier, recordEmitter, new Configuration(), context); + } + + @Override + public void start() { + if (getNumberOfCurrentlyAssignedSplits() == 0) { + context.sendSplitRequest(); + } + } + + @Override + protected void onSplitFinished(Map finishedSplitIds) { + for (BigQuerySourceSplitState splitState : finishedSplitIds.values()) { + BigQuerySourceSplit sourceSplit = splitState.toBigQuerySourceSplit(); + LOG.info("Read for split {} is completed.", sourceSplit.splitId()); + } + context.sendSplitRequest(); + } + + @Override + protected BigQuerySourceSplitState initializedState(BigQuerySourceSplit split) { + return new BigQuerySourceSplitState(split); + } + + @Override + protected BigQuerySourceSplit toSplitType(String string, BigQuerySourceSplitState sst) { + return sst.toBigQuerySourceSplit(); + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/AvroDeserializationSchema.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/AvroDeserializationSchema.java new file mode 100644 index 00000000..c94b7755 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/AvroDeserializationSchema.java @@ -0,0 +1,51 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.reader.deserializer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; + +import java.io.IOException; + +/** + * A simple Identity de-serialization for pipelines that just want {@link GenericRecord} as response + * from BigQuery. + */ +@Internal +public class AvroDeserializationSchema + implements BigQueryDeserializationSchema { + + private final String avroSchemaString; + + public AvroDeserializationSchema(String avroSchemaString) { + this.avroSchemaString = avroSchemaString; + } + + @Override + public GenericRecord deserialize(GenericRecord record) throws IOException { + return record; + } + + @Override + public TypeInformation getProducedType() { + return new GenericRecordAvroTypeInfo(new Schema.Parser().parse(avroSchemaString)); + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/AvroToRowDataDeserializationSchema.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/AvroToRowDataDeserializationSchema.java new file mode 100644 index 00000000..c0abe256 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/AvroToRowDataDeserializationSchema.java @@ -0,0 +1,51 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.reader.deserializer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.AvroToRowDataConverters; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.avro.generic.GenericRecord; + +import java.io.IOException; + +/** Simple implementation for the Deserialization schema (from Avro GenericRecord to RowData). */ +@Internal +public class AvroToRowDataDeserializationSchema + implements BigQueryDeserializationSchema { + private final AvroToRowDataConverters.AvroToRowDataConverter converter; + private final TypeInformation typeInfo; + + public AvroToRowDataDeserializationSchema(RowType rowType, TypeInformation typeInfo) { + this.converter = AvroToRowDataConverters.createRowConverter(rowType); + this.typeInfo = typeInfo; + } + + @Override + public RowData deserialize(GenericRecord record) throws IOException { + return (GenericRowData) converter.convert(record); + } + + @Override + public TypeInformation getProducedType() { + return typeInfo; + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/BigQueryDeserializationSchema.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/BigQueryDeserializationSchema.java new file mode 100644 index 00000000..ed56f28f --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/reader/deserializer/BigQueryDeserializationSchema.java @@ -0,0 +1,63 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source.reader.deserializer; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.io.Serializable; + +/** + * A schema bridge for de-serializing the BigQuery's return types ({@code GenericRecord} or {@link + * ArrowRecord}) into a flink managed instance. + * + * @param The input type to de-serialize. + * @param The output record type for to sink for downstream processing. + */ +@PublicEvolving +public interface BigQueryDeserializationSchema + extends Serializable, ResultTypeQueryable { + + /** + * De-serializes the IN type record. + * + * @param record The BSON document to de-serialize. + * @return The de-serialized message as an object (null if the message cannot be de-serialized). + * @throws java.io.IOException In case of problems while de-serializing. + */ + OUT deserialize(IN record) throws IOException; + + /** + * De-serializes the IN type record. + * + *

Can output multiple records through the {@link Collector}. Note that number and size of + * the produced records should be relatively small. Depending on the source implementation + * records can be buffered in memory or collecting records might delay emitting checkpoint + * barrier. + * + * @param record The IN document to de-serialize. + * @param out The collector to put the resulting messages. + */ + default void deserialize(IN record, Collector out) throws IOException { + OUT deserialize = deserialize(record); + if (deserialize != null) { + out.collect(deserialize); + } + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java index aa0b09a6..452740e2 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java @@ -22,6 +22,7 @@ import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableSchema; import com.google.cloud.bigquery.StandardSQLTypeName; import com.google.cloud.bigquery.storage.v1.AvroRows; @@ -49,7 +50,6 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.Serializable; -import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Optional; @@ -81,7 +81,7 @@ public QueryDataClient getQueryDataClient(CredentialsOptions readOptions) { @Override public List retrieveTablePartitions( String project, String dataset, String table) { - return new ArrayList<>(); + return Lists.newArrayList(); } @Override @@ -92,7 +92,7 @@ public Optional> retrievePartitionColumnName @Override public TableSchema getTableSchema(String project, String dataset, String table) { - return new TableSchema(); + return SIMPLE_BQ_TABLE_SCHEMA; } @Override @@ -193,6 +193,19 @@ public void close() {} public static final Schema SIMPLE_AVRO_SCHEMA = new Schema.Parser().parse(SIMPLE_AVRO_SCHEMA_STRING); + public static final TableSchema SIMPLE_BQ_TABLE_SCHEMA = + new TableSchema() + .setFields( + Lists.newArrayList( + new TableFieldSchema() + .setName("name") + .setType("STRING") + .setMode("REQUIRED"), + new TableFieldSchema() + .setName("number") + .setType("INTEGER") + .setMode("REQUIRED"))); + /** Represents the parameters needed for the Avro data generation. */ public static class RecordGenerationParams implements Serializable { private final String avroSchemaString; diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceITCase.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceITCase.java new file mode 100644 index 00000000..407bd7c9 --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceITCase.java @@ -0,0 +1,172 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.testutils.junit.SharedObjectsExtension; +import org.apache.flink.testutils.junit.SharedReference; +import org.apache.flink.util.CollectionUtil; + +import com.google.cloud.flink.bigquery.fakes.StorageClientFaker; +import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; +import com.google.cloud.flink.bigquery.source.reader.deserializer.AvroToRowDataDeserializationSchema; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.TestInstance.Lifecycle; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +/** */ +@TestInstance(Lifecycle.PER_CLASS) +public class BigQuerySourceITCase { + + private static final int PARALLELISM = 2; + private static final Integer TOTAL_ROW_COUNT_PER_STREAM = 10000; + + @RegisterExtension + static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(PARALLELISM) + .build()); + + @RegisterExtension final SharedObjectsExtension sharedObjects = SharedObjectsExtension.create(); + + private BigQueryReadOptions readOptions; + + @BeforeAll + public void beforeTest() throws Exception { + // init the read options for BQ + readOptions = + StorageClientFaker.createReadOptions( + TOTAL_ROW_COUNT_PER_STREAM, + 2, + StorageClientFaker.SIMPLE_AVRO_SCHEMA_STRING); + } + + private BigQuerySource.Builder defaultSourceBuilder() { + RowType rowType = defaultSourceRowType(); + TypeInformation typeInfo = InternalTypeInfo.of(rowType); + + return BigQuerySource.builder() + .setReadOptions(readOptions) + .setDeserializationSchema( + new AvroToRowDataDeserializationSchema(rowType, typeInfo)); + } + + private static RowType defaultSourceRowType() { + ResolvedSchema schema = + ResolvedSchema.of( + Column.physical("name", DataTypes.STRING()), + Column.physical("number", DataTypes.BIGINT())); + return (RowType) schema.toPhysicalRowDataType().getLogicalType(); + } + + @Test + public void testReadCount() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + BigQuerySource bqSource = defaultSourceBuilder().build(); + + List results = + CollectionUtil.iteratorToList( + env.fromSource( + bqSource, + WatermarkStrategy.noWatermarks(), + "BigQuery-Source") + .executeAndCollect()); + + // we only create 2 streams as response + Assertions.assertThat(results).hasSize(TOTAL_ROW_COUNT_PER_STREAM * PARALLELISM); + } + + @Test + public void testLimit() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + final int limitSize = 10; + BigQuerySource bqSource = defaultSourceBuilder().setLimit(limitSize).build(); + + List results = + env.fromSource(bqSource, WatermarkStrategy.noWatermarks(), "BigQuery-Source") + .executeAndCollect(TOTAL_ROW_COUNT_PER_STREAM); + + Assertions.assertThat(results).hasSize(limitSize * PARALLELISM); + } + + @Test + public void testRecovery() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(300L); + + BigQuerySource bqSource = defaultSourceBuilder().build(); + final SharedReference failed = sharedObjects.add(new AtomicBoolean(false)); + + List results = + CollectionUtil.iteratorToList( + env.fromSource( + bqSource, + WatermarkStrategy.noWatermarks(), + "BigQuery-Source") + .map(new FailingMapper(failed)) + .executeAndCollect()); + + Assertions.assertThat(results).hasSize(TOTAL_ROW_COUNT_PER_STREAM * PARALLELISM); + } + + private static class FailingMapper + implements MapFunction, CheckpointListener { + + private final SharedReference failed; + private int emittedRecords = 0; + + private FailingMapper(SharedReference failed) { + this.failed = failed; + } + + @Override + public RowData map(RowData value) { + emittedRecords++; + return value; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + if (failed.get().get() || emittedRecords == 0) { + return; + } + failed.get().set(true); + throw new Exception("Expected failure"); + } + } +} diff --git a/flink-connector-bigquery/src/test/resources/log4j2-test.properties b/flink-connector-bigquery/src/test/resources/log4j2-test.properties new file mode 100644 index 00000000..835c2ec9 --- /dev/null +++ b/flink-connector-bigquery/src/test/resources/log4j2-test.properties @@ -0,0 +1,28 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = OFF +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/pom.xml b/pom.xml index 9a343107..7b9f9568 100644 --- a/pom.xml +++ b/pom.xml @@ -58,8 +58,8 @@ under the License. 3.0.0-1.16 0.8.10 - 2.0.7 - 2.20.0 + 1.7.36 + 2.17.2 flink-connector-bigquery-parent From d5d95bf7a84cd9e27777ac10aa86ee1f103b7e78 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Thu, 29 Jun 2023 00:20:08 -0700 Subject: [PATCH 11/34] added Table API implementation for BigQuery --- flink-connector-bigquery/pom.xml | 2 +- .../flink/bigquery/source/BigQuerySource.java | 22 ++ .../table/BigQueryDynamicTableFactory.java | 112 +++++++ .../table/BigQueryDynamicTableSource.java | 241 +++++++++++++++ .../config/BigQueryConnectorOptions.java | 96 ++++++ .../config/BigQueryTableConfiguration.java | 94 ++++++ .../table/restrictions/BigQueryPartition.java | 127 ++++++++ .../restrictions/BigQueryRestriction.java | 288 ++++++++++++++++++ .../org.apache.flink.table.factories.Factory | 16 + .../bigquery/fakes/StorageClientFaker.java | 37 ++- .../bigquery/source/BigQuerySourceTest.java | 61 ++++ .../BigQueryDynamicTableFactoryTest.java | 144 +++++++++ .../BigQueryDynamicTableSourceITCase.java | 267 ++++++++++++++++ 13 files changed, 1496 insertions(+), 11 deletions(-) create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableFactory.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/config/BigQueryConnectorOptions.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/config/BigQueryTableConfiguration.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryPartition.java create mode 100644 flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryRestriction.java create mode 100644 flink-connector-bigquery/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceTest.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableFactoryTest.java create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSourceITCase.java diff --git a/flink-connector-bigquery/pom.xml b/flink-connector-bigquery/pom.xml index 2f809db1..1e90c672 100644 --- a/flink-connector-bigquery/pom.xml +++ b/flink-connector-bigquery/pom.xml @@ -191,7 +191,7 @@ under the License. test - ${argLine} -XX:+UseG1GC -Xms256m -Xmx2048m + -XX:+UseG1GC -Xms256m -Xmx2048m diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/BigQuerySource.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/BigQuerySource.java index d13882a5..4af8d088 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/BigQuerySource.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/BigQuerySource.java @@ -17,6 +17,7 @@ package com.google.cloud.flink.bigquery.source; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; @@ -204,6 +205,27 @@ public static BigQuerySource readAvrosFromQuery( .setQueryAndExecutionProject(query, gcpProject) .build(); + return readAvrosFromQuery(readOptions, query, gcpProject, limit); + } + + /** + * Creates an instance of the source, setting Avro {@link GenericRecord} as the return type for + * the data (mimicking the table's schema), limiting the record retrieval to the provided limit + * and reading data from the provided query which will be executed using the provided GCP + * project. + * + * @param readOptions The BigQuery read options to execute + * @param query A BigQuery standard SQL query. + * @param gcpProject The GCP project where the provided query will execute. + * @param limit the max quantity of records to be returned. + * @return A fully initialized instance of the source, ready to read {@link GenericRecord} from + * the BigQuery query results. + * @throws IOException + */ + @VisibleForTesting + static BigQuerySource readAvrosFromQuery( + BigQueryReadOptions readOptions, String query, String gcpProject, Integer limit) + throws IOException { BigQueryConnectOptions connectOptions = readOptions.getBigQueryConnectOptions(); TableSchema tableSchema = BigQueryServicesFactory.instance(connectOptions) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableFactory.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableFactory.java new file mode 100644 index 00000000..18bb2d6e --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableFactory.java @@ -0,0 +1,112 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.util.function.SerializableSupplier; + +import com.google.cloud.flink.bigquery.services.BigQueryServices; +import com.google.cloud.flink.bigquery.table.config.BigQueryConnectorOptions; +import com.google.cloud.flink.bigquery.table.config.BigQueryTableConfiguration; + +import java.util.HashSet; +import java.util.Set; + +/** Factory class to create configured instances of {@link BigQueryDynamicTableSource}. */ +@Internal +public class BigQueryDynamicTableFactory implements DynamicTableSourceFactory { + + public static final String IDENTIFIER = "bigquery"; + + private static SerializableSupplier testingServices = null; + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + final Set> requiredOptions = new HashSet<>(); + + requiredOptions.add(BigQueryConnectorOptions.PROJECT); + requiredOptions.add(BigQueryConnectorOptions.DATASET); + requiredOptions.add(BigQueryConnectorOptions.TABLE); + + return requiredOptions; + } + + @Override + public Set> optionalOptions() { + final Set> additionalOptions = new HashSet<>(); + + additionalOptions.add(BigQueryConnectorOptions.LIMIT); + additionalOptions.add(BigQueryConnectorOptions.ROW_RESTRICTION); + additionalOptions.add(BigQueryConnectorOptions.COLUMNS_PROJECTION); + additionalOptions.add(BigQueryConnectorOptions.MAX_STREAM_COUNT); + additionalOptions.add(BigQueryConnectorOptions.SNAPSHOT_TIMESTAMP); + additionalOptions.add(BigQueryConnectorOptions.CREDENTIALS_ACCESS_TOKEN); + additionalOptions.add(BigQueryConnectorOptions.CREDENTIALS_FILE); + additionalOptions.add(BigQueryConnectorOptions.CREDENTIALS_KEY); + additionalOptions.add(BigQueryConnectorOptions.TEST_MODE); + + return additionalOptions; + } + + @Override + public Set> forwardOptions() { + final Set> forwardOptions = new HashSet<>(); + + forwardOptions.add(BigQueryConnectorOptions.PROJECT); + forwardOptions.add(BigQueryConnectorOptions.DATASET); + forwardOptions.add(BigQueryConnectorOptions.TABLE); + forwardOptions.add(BigQueryConnectorOptions.LIMIT); + forwardOptions.add(BigQueryConnectorOptions.ROW_RESTRICTION); + forwardOptions.add(BigQueryConnectorOptions.COLUMNS_PROJECTION); + forwardOptions.add(BigQueryConnectorOptions.MAX_STREAM_COUNT); + forwardOptions.add(BigQueryConnectorOptions.SNAPSHOT_TIMESTAMP); + forwardOptions.add(BigQueryConnectorOptions.CREDENTIALS_ACCESS_TOKEN); + forwardOptions.add(BigQueryConnectorOptions.CREDENTIALS_FILE); + forwardOptions.add(BigQueryConnectorOptions.CREDENTIALS_KEY); + + return forwardOptions; + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); + + BigQueryTableConfiguration config = new BigQueryTableConfiguration(helper.getOptions()); + helper.validate(); + + if (config.isTestModeEnabled()) { + config = config.withTestingServices(testingServices); + } + + return new BigQueryDynamicTableSource( + config.toBigQueryReadOptions(), context.getPhysicalRowDataType()); + } + + static void setTestingServices(SerializableSupplier testingServices) { + BigQueryDynamicTableFactory.testingServices = testingServices; + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java new file mode 100644 index 00000000..bc7cd092 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java @@ -0,0 +1,241 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.SourceProvider; +import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsPartitionPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; +import com.google.cloud.flink.bigquery.services.BigQueryServices; +import com.google.cloud.flink.bigquery.services.BigQueryServicesFactory; +import com.google.cloud.flink.bigquery.source.BigQuerySource; +import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; +import com.google.cloud.flink.bigquery.source.reader.deserializer.AvroToRowDataDeserializationSchema; +import com.google.cloud.flink.bigquery.table.restrictions.BigQueryPartition; +import com.google.cloud.flink.bigquery.table.restrictions.BigQueryRestriction; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; + +/** A {@link DynamicTableSource} for Google BigQuery. */ +@Internal +public class BigQueryDynamicTableSource + implements ScanTableSource, + SupportsProjectionPushDown, + SupportsLimitPushDown, + SupportsFilterPushDown, + SupportsPartitionPushDown { + + private BigQueryReadOptions readOptions; + private DataType producedDataType; + private Integer limit = -1; + + public BigQueryDynamicTableSource(BigQueryReadOptions readOptions, DataType producedDataType) { + this.readOptions = readOptions; + this.producedDataType = producedDataType; + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { + final RowType rowType = (RowType) producedDataType.getLogicalType(); + final TypeInformation typeInfo = + runtimeProviderContext.createTypeInformation(producedDataType); + + BigQuerySource bqSource = + BigQuerySource.builder() + .setLimit(limit) + .setReadOptions(readOptions) + .setDeserializationSchema( + new AvroToRowDataDeserializationSchema(rowType, typeInfo)) + .build(); + + return SourceProvider.of(bqSource); + } + + @Override + public DynamicTableSource copy() { + return new BigQueryDynamicTableSource(readOptions, producedDataType); + } + + @Override + public String asSummaryString() { + return "BigQuery"; + } + + @Override + public boolean supportsNestedProjection() { + return false; + } + + @Override + public void applyProjection(int[][] projectedFields, DataType producedDataType) { + this.producedDataType = producedDataType; + this.readOptions = + this.readOptions + .toBuilder() + .setColumnNames(DataType.getFieldNames(producedDataType)) + .build(); + } + + @Override + public void applyLimit(long limit) { + this.limit = (int) limit; + } + + @Override + public Result applyFilters(List filters) { + Map>> translatedFilters = + filters.stream() + .map( + exp -> + Tuple2.>of( + exp, BigQueryRestriction.convert(exp))) + .map( + transExp -> + Tuple3.of( + transExp.f1.isPresent(), + transExp.f1.orElse(""), + transExp.f0)) + .collect( + Collectors.groupingBy( + (Tuple3 t) -> t.f0)); + String rowRestriction = + translatedFilters.getOrDefault(true, Lists.newArrayList()).stream() + .map(t -> t.f1) + .collect(Collectors.joining(" AND ")); + this.readOptions = this.readOptions.toBuilder().setRowRestriction(rowRestriction).build(); + return Result.of( + translatedFilters.getOrDefault(true, Lists.newArrayList()).stream() + .map(t -> t.f2) + .collect(Collectors.toList()), + filters); + } + + @Override + public int hashCode() { + int hash = 5; + hash = 61 * hash + Objects.hashCode(this.readOptions); + hash = 61 * hash + Objects.hashCode(this.producedDataType); + hash = 61 * hash + Objects.hashCode(this.limit); + return hash; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final BigQueryDynamicTableSource other = (BigQueryDynamicTableSource) obj; + if (!Objects.equals(this.readOptions, other.readOptions)) { + return false; + } + if (!Objects.equals(this.producedDataType, other.producedDataType)) { + return false; + } + return Objects.equals(this.limit, other.limit); + } + + @Override + public Optional>> listPartitions() { + BigQueryConnectOptions connectOptions = readOptions.getBigQueryConnectOptions(); + BigQueryServices.QueryDataClient dataClient = + BigQueryServicesFactory.instance(connectOptions).queryClient(); + return dataClient + // get the column name that is a partition, maybe none. + .retrievePartitionColumnName( + connectOptions.getProjectId(), + connectOptions.getDataset(), + connectOptions.getTable()) + .map( + tuple -> { + // we retrieve the existing partition ids and transform them into valid + // values given the column data type + return BigQueryPartition.partitionValuesFromIdAndDataType( + dataClient.retrieveTablePartitions( + connectOptions.getProjectId(), + connectOptions.getDataset(), + connectOptions.getTable()), + tuple.f1) + .stream() + // for each of those valid partition values we create an map + // with the column name and the value + .map( + pValue -> { + Map partitionColAndValue = + new HashMap<>(); + partitionColAndValue.put(tuple.f0, pValue); + return partitionColAndValue; + }) + .collect(Collectors.toList()); + }); + } + + @Override + public void applyPartitions(List> remainingPartitions) { + this.readOptions = + this.readOptions + .toBuilder() + .setRowRestriction( + // lets set the row restriction concating previously set restriction + // (coming from the table definition) with the partition restriction + // sent by Flink planner. + this.readOptions.getRowRestriction() + + " AND " + + remainingPartitions.stream() + .flatMap(map -> map.entrySet().stream()) + .map( + entry -> + "(" + + entry.getKey() + + "=" + + entry.getValue() + + ")") + .collect(Collectors.joining(" AND "))) + .build(); + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/config/BigQueryConnectorOptions.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/config/BigQueryConnectorOptions.java new file mode 100644 index 00000000..ddc53a30 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/config/BigQueryConnectorOptions.java @@ -0,0 +1,96 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.table.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** + * Base options for the BigQuery connector. Needs to be public so that the {@link + * org.apache.flink.table.api.TableDescriptor} can access it. + */ +@PublicEvolving +public class BigQueryConnectorOptions { + + private BigQueryConnectorOptions() {} + + public static final ConfigOption PROJECT = + ConfigOptions.key("project") + .stringType() + .noDefaultValue() + .withDescription("Specifies the GCP project for BigQuery."); + public static final ConfigOption DATASET = + ConfigOptions.key("dataset") + .stringType() + .noDefaultValue() + .withDescription("Specifies the BigQuery dataset name."); + public static final ConfigOption TABLE = + ConfigOptions.key("table") + .stringType() + .noDefaultValue() + .withDescription("Specifies the BigQuery table name."); + public static final ConfigOption LIMIT = + ConfigOptions.key("read.limit") + .intType() + .defaultValue(-1) + .withDescription("Specifies the limit number of rows retrieved."); + public static final ConfigOption ROW_RESTRICTION = + ConfigOptions.key("read.row.restriction") + .stringType() + .defaultValue("") + .withDescription("Specifies the row restriction for data retrieval."); + public static final ConfigOption COLUMNS_PROJECTION = + ConfigOptions.key("read.columns.projection") + .stringType() + .noDefaultValue() + .withDescription( + "Specifies, as a comma separated list of values, " + + "the columns to be included as part of the data retrieved."); + public static final ConfigOption MAX_STREAM_COUNT = + ConfigOptions.key("read.streams.maxcount") + .intType() + .defaultValue(0) + .withDescription( + "The max number of streams used to read from the underlying table," + + " BigQuery can decide for less than this number."); + public static final ConfigOption SNAPSHOT_TIMESTAMP = + ConfigOptions.key("read.snapshot.timestamp") + .longType() + .noDefaultValue() + .withDescription("The millis since epoch for the underlying table snapshot."); + public static final ConfigOption CREDENTIALS_ACCESS_TOKEN = + ConfigOptions.key("credentials.accesstoken") + .stringType() + .noDefaultValue() + .withDescription("Specifies the GCP access token to use as credentials."); + public static final ConfigOption CREDENTIALS_FILE = + ConfigOptions.key("credentials.file") + .stringType() + .noDefaultValue() + .withDescription("Specifies the GCP credentials file to use."); + public static final ConfigOption CREDENTIALS_KEY = + ConfigOptions.key("credentials.key") + .stringType() + .noDefaultValue() + .withDescription("Specifies the GCP credentials key to use."); + public static final ConfigOption TEST_MODE = + ConfigOptions.key("test.enabled") + .booleanType() + .defaultValue(false) + .withDescription("Specifies if the connector should run in test mode."); +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/config/BigQueryTableConfiguration.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/config/BigQueryTableConfiguration.java new file mode 100644 index 00000000..6a14ee2f --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/config/BigQueryTableConfiguration.java @@ -0,0 +1,94 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.table.config; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.util.function.SerializableSupplier; + +import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; +import com.google.cloud.flink.bigquery.common.config.CredentialsOptions; +import com.google.cloud.flink.bigquery.services.BigQueryServices; +import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Optional; + +/** + * A BigQuery Configuration class which can easily be used to transform to the option objects the + * source implementation expects. + */ +@Internal +public class BigQueryTableConfiguration { + private final ReadableConfig config; + private Optional> testingServices = Optional.empty(); + + public BigQueryTableConfiguration(ReadableConfig config) { + this.config = config; + } + + public BigQueryTableConfiguration withTestingServices( + SerializableSupplier testingServices) { + this.testingServices = Optional.of(testingServices); + return this; + } + + public Boolean isTestModeEnabled() { + return config.get(BigQueryConnectorOptions.TEST_MODE); + } + + public BigQueryReadOptions toBigQueryReadOptions() { + try { + return BigQueryReadOptions.builder() + .setSnapshotTimestampInMillis( + config.get(BigQueryConnectorOptions.SNAPSHOT_TIMESTAMP)) + .setMaxStreamCount(config.get(BigQueryConnectorOptions.MAX_STREAM_COUNT)) + .setRowRestriction(config.get(BigQueryConnectorOptions.ROW_RESTRICTION)) + .setColumnNames( + Optional.ofNullable( + config.get(BigQueryConnectorOptions.COLUMNS_PROJECTION)) + .map(cols -> Arrays.asList(cols.split(","))) + .orElse(new ArrayList<>())) + .setBigQueryConnectOptions(translateBigQueryConnectOptions()) + .build(); + } catch (Exception ex) { + throw new RuntimeException( + "Problems while trying to translate table configuration.", ex); + } + } + + private BigQueryConnectOptions translateBigQueryConnectOptions() throws IOException { + return BigQueryConnectOptions.builder() + .setProjectId(config.get(BigQueryConnectorOptions.PROJECT)) + .setDataset(config.get(BigQueryConnectorOptions.DATASET)) + .setTable(config.get(BigQueryConnectorOptions.TABLE)) + .setTestingBigQueryServices(testingServices.orElse(null)) + .setCredentialsOptions( + CredentialsOptions.builder() + .setAccessToken( + config.get( + BigQueryConnectorOptions.CREDENTIALS_ACCESS_TOKEN)) + .setCredentialsFile( + config.get(BigQueryConnectorOptions.CREDENTIALS_FILE)) + .setCredentialsKey( + config.get(BigQueryConnectorOptions.CREDENTIALS_KEY)) + .build()) + .build(); + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryPartition.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryPartition.java new file mode 100644 index 00000000..d4b2a806 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryPartition.java @@ -0,0 +1,127 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.table.restrictions; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.Preconditions; + +import com.google.cloud.bigquery.StandardSQLTypeName; + +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +/** Utility class to handle the BigQuery partition conversions to Flink types and structures. */ +@Internal +public class BigQueryPartition { + + private BigQueryPartition() {} + + static List partitionIdToDateFormat( + List partitions, String fromFormat, String toFormat) { + SimpleDateFormat parseFormat = new SimpleDateFormat(fromFormat); + SimpleDateFormat printFormat = new SimpleDateFormat(toFormat); + + return partitions.stream() + .map( + id -> { + try { + return parseFormat.parse(id); + } catch (ParseException ex) { + throw new RuntimeException( + "Problems parsing the temporal value: " + id); + } + }) + .map(date -> printFormat.format(date)) + .map(strDate -> String.format("'%s'", strDate)) + .collect(Collectors.toList()); + } + + public static List partitionValuesFromIdAndDataType( + List partitionIds, StandardSQLTypeName dataType) { + List partitionValues = new ArrayList<>(); + switch (dataType) { + // integer range partition + case INT64: + // we add them as they are + partitionValues.addAll(partitionIds); + break; + // time based partitioning (hour, date, month, year) + case DATE: + case DATETIME: + case TIMESTAMP: + // lets first check that all the partition ids have the same length + String firstId = partitionIds.get(0); + Preconditions.checkState( + partitionIds.stream().allMatch(pid -> pid.length() == firstId.length()), + "Some elements in the partition id list have a different length: " + + partitionIds.toString()); + switch (firstId.length()) { + case 4: + // we have yearly partitions + partitionValues.addAll( + partitionIds.stream() + .map(id -> String.format("'%s'", id)) + .collect(Collectors.toList())); + break; + case 6: + // we have monthly partitions + partitionValues.addAll( + partitionIdToDateFormat(partitionIds, "yyyyMM", "yyyy-MM")); + break; + case 8: + // we have daily partitions + partitionValues.addAll( + partitionIdToDateFormat(partitionIds, "yyyyMMdd", "yyyy-MM-dd")); + break; + case 10: + // we have hourly partitions + partitionValues.addAll( + partitionIdToDateFormat( + partitionIds, "yyyyMMddHH", "yyyy-MM-dd HH:mm:ss")); + break; + default: + throw new IllegalArgumentException( + "The lenght of the partition id is not one of the expected ones: " + + firstId); + } + break; + // non supported data types for partitions + case ARRAY: + case STRUCT: + case JSON: + case GEOGRAPHY: + case BIGNUMERIC: + case BOOL: + case BYTES: + case FLOAT64: + case STRING: + case TIME: + case INTERVAL: + case NUMERIC: + default: + throw new IllegalArgumentException( + String.format( + "The provided SQL type name (%s) is not supported" + + " as a partition column.", + dataType.name())); + } + return partitionValues; + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryRestriction.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryRestriction.java new file mode 100644 index 00000000..49860a12 --- /dev/null +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryRestriction.java @@ -0,0 +1,288 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.table.restrictions; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; + +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap; + +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.BiFunction; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Utility class to transform a flink {@link ResolvedExpression} in a string restriction that can be + * send to BigQuery as a row restriction. Heavily based in the Iceberg expression translation + * implementation. + */ +@Internal +public class BigQueryRestriction { + + private BigQueryRestriction() {} + + private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%"); + + /** Represents the possible BQ expressions supported for the correspondent flink ones. */ + public enum Operation { + EQ, + NOT_EQ, + GT, + GT_EQ, + LT, + LT_EQ, + IS_NULL, + NOT_NULL, + AND, + OR, + NOT, + STARTS_WITH + } + + private static final Map FILTERS = + ImmutableMap.builder() + .put(BuiltInFunctionDefinitions.EQUALS, Operation.EQ) + .put(BuiltInFunctionDefinitions.NOT_EQUALS, Operation.NOT_EQ) + .put(BuiltInFunctionDefinitions.GREATER_THAN, Operation.GT) + .put(BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, Operation.GT_EQ) + .put(BuiltInFunctionDefinitions.LESS_THAN, Operation.LT) + .put(BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, Operation.LT_EQ) + .put(BuiltInFunctionDefinitions.IS_NULL, Operation.IS_NULL) + .put(BuiltInFunctionDefinitions.IS_NOT_NULL, Operation.NOT_NULL) + .put(BuiltInFunctionDefinitions.AND, Operation.AND) + .put(BuiltInFunctionDefinitions.OR, Operation.OR) + .put(BuiltInFunctionDefinitions.NOT, Operation.NOT) + .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH) + .build(); + + /** + * Convert a flink expression into a BigQuery row restriction. + * + *

the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the + * BETWEEN will be converted to (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ + * OR GT_EQ), the IN will be converted to OR, so we do not add the conversion here + * + * @param flinkExpression the flink expression + * @return An {@link Optional} potentially containing the resolved row restriction for BigQuery. + */ + public static Optional convert(Expression flinkExpression) { + if (!(flinkExpression instanceof CallExpression)) { + return Optional.empty(); + } + + CallExpression call = (CallExpression) flinkExpression; + Operation op = FILTERS.get(call.getFunctionDefinition()); + if (op != null) { + switch (op) { + case IS_NULL: + return onlyChildAs(call, FieldReferenceExpression.class) + .map(FieldReferenceExpression::getName) + .map(field -> field + " IS NULL"); + + case NOT_NULL: + return onlyChildAs(call, FieldReferenceExpression.class) + .map(FieldReferenceExpression::getName) + .map(field -> "NOT " + field + " IS NULL"); + + case LT: + return convertFieldAndLiteral( + (left, right) -> "(" + left + " < " + right + ")", + (left, right) -> "(" + left + " > " + right + ")", + call); + + case LT_EQ: + return convertFieldAndLiteral( + (left, right) -> "(" + left + " <= " + right + ")", + (left, right) -> "(" + left + " >= " + right + ")", + call); + + case GT: + return convertFieldAndLiteral( + (left, right) -> "(" + left + " > " + right + ")", + (left, right) -> "(" + left + " < " + right + ")", + call); + + case GT_EQ: + return convertFieldAndLiteral( + (left, right) -> "(" + left + " >= " + right + ")", + (left, right) -> "(" + left + " <= " + right + ")", + call); + + case EQ: + return convertFieldAndLiteral((ref, lit) -> ref + " = " + lit, call); + + case NOT_EQ: + return convertFieldAndLiteral((ref, lit) -> ref + " <> " + lit, call); + + case NOT: + return onlyChildAs(call, CallExpression.class) + .flatMap(BigQueryRestriction::convert) + .map(field -> "NOT " + field); + + case AND: + return convertLogicExpression( + (left, right) -> "(" + left + " AND " + right + ")", call); + + case OR: + return convertLogicExpression( + (left, right) -> "(" + left + " OR " + right + ")", call); + + case STARTS_WITH: + return convertLike(call); + } + } + + return Optional.empty(); + } + + private static Optional onlyChildAs( + CallExpression call, Class expectedChildClass) { + List children = call.getResolvedChildren(); + if (children.size() != 1) { + return Optional.empty(); + } + + ResolvedExpression child = children.get(0); + if (!expectedChildClass.isInstance(child)) { + return Optional.empty(); + } + + return Optional.of(expectedChildClass.cast(child)); + } + + private static Optional convertLike(CallExpression call) { + List args = call.getResolvedChildren(); + if (args.size() != 2) { + return Optional.empty(); + } + + Expression left = args.get(0); + Expression right = args.get(1); + + if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { + String name = ((FieldReferenceExpression) left).getName(); + return convertLiteral((ValueLiteralExpression) right) + .flatMap( + lit -> { + if (lit instanceof String) { + String pattern = (String) lit; + Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern); + // exclude special char of LIKE + // '_' is the wildcard of the SQL LIKE + if (!pattern.contains("_") && matcher.matches()) { + return Optional.of( + name + " LIKE '" + matcher.group(1) + "'"); + } + } + + return Optional.empty(); + }); + } + + return Optional.empty(); + } + + private static Optional convertLogicExpression( + BiFunction function, CallExpression call) { + List args = call.getResolvedChildren(); + if (args == null || args.size() != 2) { + return Optional.empty(); + } + + Optional left = convert(args.get(0)); + Optional right = convert(args.get(1)); + if (left.isPresent() && right.isPresent()) { + return Optional.of(function.apply(left.get(), right.get())); + } + + return Optional.empty(); + } + + private static String addSingleQuotes(String input) { + return "'" + input + "'"; + } + + private static Optional convertLiteral(ValueLiteralExpression expression) { + Optional value = + expression.getValueAs( + expression.getOutputDataType().getLogicalType().getDefaultConversion()); + return value.map( + o -> { + if (o instanceof LocalDateTime) { + return addSingleQuotes(((LocalDateTime) o).toString()); + } else if (o instanceof Instant) { + return addSingleQuotes(((Instant) o).toString()); + } else if (o instanceof LocalTime) { + return addSingleQuotes(((LocalTime) o).toString()); + } else if (o instanceof LocalDate) { + return addSingleQuotes(((LocalDate) o).toString()); + } else if (o instanceof String) { + return addSingleQuotes((String) o); + } + + return o; + }); + } + + private static Optional convertFieldAndLiteral( + BiFunction expr, CallExpression call) { + return convertFieldAndLiteral(expr, expr, call); + } + + private static Optional convertFieldAndLiteral( + BiFunction convertLR, + BiFunction convertRL, + CallExpression call) { + List args = call.getResolvedChildren(); + if (args.size() != 2) { + return Optional.empty(); + } + + Expression left = args.get(0); + Expression right = args.get(1); + + if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { + String name = ((FieldReferenceExpression) left).getName(); + Optional lit = convertLiteral((ValueLiteralExpression) right); + if (lit.isPresent()) { + return Optional.of(convertLR.apply(name, lit.get())); + } + } else if (left instanceof ValueLiteralExpression + && right instanceof FieldReferenceExpression) { + Optional lit = convertLiteral((ValueLiteralExpression) left); + String name = ((FieldReferenceExpression) right).getName(); + if (lit.isPresent()) { + return Optional.of(convertRL.apply(name, lit.get())); + } + } + + return Optional.empty(); + } +} diff --git a/flink-connector-bigquery/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connector-bigquery/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 00000000..bb64c806 --- /dev/null +++ b/flink-connector-bigquery/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +com.google.cloud.flink.bigquery.table.BigQueryDynamicTableFactory diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java index 452740e2..a22aa268 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java @@ -22,6 +22,8 @@ import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.JobStatistics; +import com.google.api.services.bigquery.model.JobStatistics2; import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableSchema; import com.google.cloud.bigquery.StandardSQLTypeName; @@ -81,13 +83,13 @@ public QueryDataClient getQueryDataClient(CredentialsOptions readOptions) { @Override public List retrieveTablePartitions( String project, String dataset, String table) { - return Lists.newArrayList(); + return Lists.newArrayList("2023062811"); } @Override public Optional> retrievePartitionColumnName( String project, String dataset, String table) { - return Optional.empty(); + return Optional.of(Tuple2.of("number", StandardSQLTypeName.INT64)); } @Override @@ -102,7 +104,12 @@ public Optional runQuery(String projectId, String query) { @Override public Job dryRunQuery(String projectId, String query) { - return null; + return new Job() + .setStatistics( + new JobStatistics() + .setQuery( + new JobStatistics2() + .setSchema(SIMPLE_BQ_TABLE_SCHEMA))); } }; } @@ -180,16 +187,26 @@ public void close() {} } } - public static final String SIMPLE_AVRO_SCHEMA_STRING = - "{\"namespace\": \"example.avro\",\n" - + " \"type\": \"record\",\n" - + " \"name\": \"RowRecord\",\n" - + " \"fields\": [\n" + public static final String SIMPLE_AVRO_SCHEMA_FIELDS_STRING = + " \"fields\": [\n" + " {\"name\": \"name\", \"type\": \"string\"},\n" + " {\"name\": \"number\", \"type\": \"long\"}\n" - + " ]\n" + + " ]\n"; + public static final String SIMPLE_AVRO_SCHEMA_STRING = + "{\"namespace\": \"project.dataset\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"table\",\n" + + " \"doc\": \"Translated Avro Schema for project.dataset.table\",\n" + + SIMPLE_AVRO_SCHEMA_FIELDS_STRING + + "}"; + public static final String SIMPLE_AVRO_SCHEMA_FORQUERY_STRING = + "{\"namespace\": \"project.dataset\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"queryresultschema\",\n" + + " \"namespace\": \"org.apache.flink.connector.bigquery\",\n" + + " \"doc\": \"Translated Avro Schema for queryresultschema\",\n" + + SIMPLE_AVRO_SCHEMA_FIELDS_STRING + "}"; - public static final Schema SIMPLE_AVRO_SCHEMA = new Schema.Parser().parse(SIMPLE_AVRO_SCHEMA_STRING); diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceTest.java new file mode 100644 index 00000000..19e919e6 --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceTest.java @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.source; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; + +import com.google.cloud.flink.bigquery.fakes.StorageClientFaker; +import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import java.io.IOException; + +/** */ +public class BigQuerySourceTest { + + @Test + public void testReadAvros() throws IOException { + BigQueryReadOptions readOptions = + StorageClientFaker.createReadOptions( + 10, 2, StorageClientFaker.SIMPLE_AVRO_SCHEMA_STRING); + BigQuerySource source = BigQuerySource.readAvros(readOptions); + TypeInformation expected = + new GenericRecordAvroTypeInfo(StorageClientFaker.SIMPLE_AVRO_SCHEMA); + Assertions.assertThat(source.getDeserializationSchema().getProducedType()) + .isEqualTo(expected); + } + + @Test + public void testReadAvrosFromQuery() throws IOException { + BigQueryReadOptions readOptions = + StorageClientFaker.createReadOptions( + 10, 2, StorageClientFaker.SIMPLE_AVRO_SCHEMA_STRING); + BigQuerySource source = + BigQuerySource.readAvrosFromQuery(readOptions, "SELECT 1", "someproject", -1); + + TypeInformation expected = + new GenericRecordAvroTypeInfo( + new Schema.Parser() + .parse(StorageClientFaker.SIMPLE_AVRO_SCHEMA_FORQUERY_STRING)); + Assertions.assertThat(source.getDeserializationSchema().getProducedType()) + .isEqualTo(expected); + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableFactoryTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableFactoryTest.java new file mode 100644 index 00000000..e0ad3afd --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableFactoryTest.java @@ -0,0 +1,144 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.table; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.utils.FactoryMocks; + +import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; +import com.google.cloud.flink.bigquery.common.config.CredentialsOptions; +import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; +import com.google.cloud.flink.bigquery.table.config.BigQueryConnectorOptions; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.time.Instant; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** Tests for the {@link BigQueryDynamicTableSource} factory class. */ +public class BigQueryDynamicTableFactoryTest { + + private static final ResolvedSchema SCHEMA = + new ResolvedSchema( + Arrays.asList( + Column.physical("aaa", DataTypes.INT().notNull()), + Column.physical("bbb", DataTypes.STRING().notNull()), + Column.physical("ccc", DataTypes.DOUBLE()), + Column.physical("ddd", DataTypes.DECIMAL(31, 18)), + Column.physical("eee", DataTypes.TIMESTAMP(3))), + Collections.emptyList(), + UniqueConstraint.primaryKey("name", Arrays.asList("bbb", "aaa"))); + + @Test + public void testBigQuerySourceCommonProperties() throws IOException { + DynamicTableSource actualSource = + FactoryMocks.createTableSource(SCHEMA, getRequiredOptions()); + + BigQueryDynamicTableSource expectedSource = + new BigQueryDynamicTableSource( + getConnectorOptions(), SCHEMA.toPhysicalRowDataType()); + Assert.assertEquals("The sources are not equals.", actualSource, expectedSource); + } + + @Test + public void testBigQueryReadProperties() throws IOException { + Map properties = getRequiredOptions(); + properties.put(BigQueryConnectorOptions.COLUMNS_PROJECTION.key(), "aaa,bbb"); + properties.put(BigQueryConnectorOptions.MAX_STREAM_COUNT.key(), "100"); + properties.put( + BigQueryConnectorOptions.ROW_RESTRICTION.key(), "aaa > 10 AND NOT bbb IS NULL"); + properties.put( + BigQueryConnectorOptions.SNAPSHOT_TIMESTAMP.key(), + "" + Instant.EPOCH.toEpochMilli()); + + DynamicTableSource actual = FactoryMocks.createTableSource(SCHEMA, properties); + + BigQueryReadOptions connectorOptions = getConnectorOptions(); + BigQueryReadOptions readOptions = + BigQueryReadOptions.builder() + .setColumnNames(Arrays.asList("aaa", "bbb")) + .setMaxStreamCount(100) + .setRowRestriction("aaa > 10 AND NOT bbb IS NULL") + .setSnapshotTimestampInMillis(Instant.EPOCH.toEpochMilli()) + .setBigQueryConnectOptions(connectorOptions.getBigQueryConnectOptions()) + .build(); + + BigQueryDynamicTableSource expected = + new BigQueryDynamicTableSource(readOptions, SCHEMA.toPhysicalRowDataType()); + + Assertions.assertThat(actual).isEqualTo(expected); + } + + @Test + public void testBigQuerySourceValidation() { + // max num of streams should be positive + assertSourceValidationRejects( + BigQueryConnectorOptions.MAX_STREAM_COUNT.key(), + "-5", + "The max number of streams should be zero or positive."); + // the snapshot timestamp in millis should at least be equal to epoch + assertSourceValidationRejects( + BigQueryConnectorOptions.SNAPSHOT_TIMESTAMP.key(), + "-1000", + "The oldest timestamp should be equal or bigger than epoch."); + } + + private void assertSourceValidationRejects(String key, String value, String errorMessage) { + Assertions.assertThatThrownBy( + () -> + FactoryMocks.createTableSource( + SCHEMA, getRequiredOptionsWithSetting(key, value))) + .hasStackTraceContaining(errorMessage); + } + + private static Map getRequiredOptionsWithSetting(String key, String value) { + Map requiredOptions = getRequiredOptions(); + requiredOptions.put(key, value); + return requiredOptions; + } + + private static Map getRequiredOptions() { + Map options = new HashMap<>(); + options.put(FactoryUtil.CONNECTOR.key(), "bigquery"); + options.put(BigQueryConnectorOptions.PROJECT.key(), "project"); + options.put(BigQueryConnectorOptions.DATASET.key(), "dataset"); + options.put(BigQueryConnectorOptions.TABLE.key(), "table"); + return options; + } + + private static BigQueryReadOptions getConnectorOptions() throws IOException { + return BigQueryReadOptions.builder() + .setBigQueryConnectOptions( + BigQueryConnectOptions.builder() + .setDataset("dataset") + .setProjectId("project") + .setTable("table") + .setCredentialsOptions(CredentialsOptions.builder().build()) + .build()) + .build(); + } +} diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSourceITCase.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSourceITCase.java new file mode 100644 index 00000000..69c1ad38 --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSourceITCase.java @@ -0,0 +1,267 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.table; + +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.function.SerializableFunction; + +import com.google.cloud.flink.bigquery.fakes.StorageClientFaker; +import com.google.cloud.flink.bigquery.table.config.BigQueryConnectorOptions; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.nio.ByteBuffer; +import java.time.Instant; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +/** An integration test for the SQL interface of the BigQuery connector. */ +public class BigQueryDynamicTableSourceITCase { + + private static final int PARALLELISM = 1; + private static final Integer TOTAL_ROW_COUNT_PER_STREAM = 10000; + private static final Integer STREAM_COUNT = 2; + private static final Schema AVRO_SCHEMA = + SchemaBuilder.record("testRecord") + .fields() + .requiredInt("id") + .optionalDouble("optDouble") + .optionalString("optString") + .name("reqSubRecord") + .type( + SchemaBuilder.record("reqSubRecord") + .fields() + .requiredBoolean("reqBoolean") + .name("reqTs") + .type( + LogicalTypes.timestampMillis() + .addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord()) + .noDefault() + .name("optArraySubRecords") + .type() + .nullable() + .array() + .items( + SchemaBuilder.record("myOptionalArraySubRecordType") + .fields() + .requiredLong("reqLong") + .optionalBytes("optBytes") + .endRecord()) + .noDefault() + .endRecord(); + + @RegisterExtension + static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(PARALLELISM) + .build()); + + @BeforeAll + public static void beforeTest() throws Exception { + // create a data generator based on the test schema + SerializableFunction> + dataGenerator = + params -> { + Schema schema = new Schema.Parser().parse(params.getAvroSchemaString()); + return IntStream.range(0, params.getRecordCount()) + .mapToObj( + i -> { + GenericRecord record = + new GenericData.Record(schema); + record.put("id", i); + record.put("optDouble", (double) i * 2); + record.put("optString", "s" + i); + + GenericData.Record reqSubRecord = + new GenericData.Record( + schema.getField("reqSubRecord") + .schema()); + reqSubRecord.put("reqBoolean", false); + reqSubRecord.put( + "reqTs", Instant.now().toEpochMilli()); + record.put("reqSubRecord", reqSubRecord); + + GenericData.Record optArraySubRecords = + new GenericData.Record( + schema.getField( + "optArraySubRecords") + .schema() + .getTypes() + .get(0) + .getElementType()); + optArraySubRecords.put("reqLong", (long) i * 100); + optArraySubRecords.put( + "optBytes", + ByteBuffer.wrap(new byte[4]).putInt(i)); + record.put( + "optArraySubRecords", + Arrays.asList( + optArraySubRecords, + optArraySubRecords)); + + return record; + }) + .collect(Collectors.toList()); + }; + + // init the testing services and inject them into the table factory + BigQueryDynamicTableFactory.setTestingServices( + StorageClientFaker.createReadOptions( + TOTAL_ROW_COUNT_PER_STREAM, + STREAM_COUNT, + AVRO_SCHEMA.toString(), + dataGenerator) + .getBigQueryConnectOptions() + .getTestingBigQueryServices()); + } + + public static StreamExecutionEnvironment env; + public static StreamTableEnvironment tEnv; + + @BeforeEach + public void before() { + env = StreamExecutionEnvironment.getExecutionEnvironment(); + tEnv = StreamTableEnvironment.create(env); + } + + @Test + public void testSource() { + tEnv.executeSql(createTestDDl(null)); + + Iterator collected = tEnv.executeSql("SELECT * FROM bigquery_source").collect(); + List result = + CollectionUtil.iteratorToList(collected).stream() + .map(Row::toString) + .sorted() + .collect(Collectors.toList()); + + // we check the data count correlates with the generated total + Assertions.assertThat(result).hasSize(TOTAL_ROW_COUNT_PER_STREAM * STREAM_COUNT); + } + + @Test + public void testLimit() { + tEnv.executeSql(createTestDDl(null)); + + Iterator collected = + tEnv.executeSql("SELECT * FROM bigquery_source LIMIT 1").collect(); + List result = + CollectionUtil.iteratorToList(collected).stream() + .map(Row::toString) + .sorted() + .collect(Collectors.toList()); + + Assertions.assertThat(result).hasSize(1); + } + + @Test + public void testProject() { + tEnv.executeSql(createTestDDl(null)); + + Iterator collected = + tEnv.executeSql("SELECT id, optDouble, optString FROM bigquery_source LIMIT 1") + .collect(); + List result = + CollectionUtil.iteratorToList(collected).stream() + .map(Row::toString) + .sorted() + .collect(Collectors.toList()); + + List expected = Stream.of("+I[0, 0.0, s0]").sorted().collect(Collectors.toList()); + + Assertions.assertThat(result).isEqualTo(expected); + } + + @Test + public void testRestriction() { + String sqlFilter = "id = 0 AND NOT optString IS NULL"; + tEnv.executeSql(createTestDDl(null)); + + Iterator collected = + tEnv.executeSql( + "SELECT id, optDouble, optString " + + "FROM bigquery_source " + + "WHERE " + + sqlFilter + + " " + + "LIMIT 1") + .collect(); + List result = + CollectionUtil.iteratorToList(collected).stream() + .map(Row::toString) + .sorted() + .collect(Collectors.toList()); + + List expected = Stream.of("+I[0, 0.0, s0]").sorted().collect(Collectors.toList()); + + Assertions.assertThat(result).isEqualTo(expected); + } + + private static String createTestDDl(Map extraOptions) { + Map options = new HashMap<>(); + options.put(FactoryUtil.CONNECTOR.key(), "bigquery"); + options.put(BigQueryConnectorOptions.PROJECT.key(), "project"); + options.put(BigQueryConnectorOptions.DATASET.key(), "dataset"); + options.put(BigQueryConnectorOptions.TABLE.key(), "table"); + options.put(BigQueryConnectorOptions.TEST_MODE.key(), "true"); + if (extraOptions != null) { + options.putAll(extraOptions); + } + + String optionString = + options.entrySet().stream() + .map(e -> String.format("'%s' = '%s'", e.getKey(), e.getValue())) + .collect(Collectors.joining(",\n")); + + return String.join( + "\n", + Arrays.asList( + "CREATE TABLE bigquery_source", + "(", + " id INTEGER,", + " optDouble DOUBLE,", + " optString VARCHAR,", + " reqSubRecord ROW,", + " optArraySubRecords ARRAY>", + ") WITH (", + optionString, + ")")); + } +} From 12637685f974fcef229eea4face5ad26383ece9c Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Thu, 29 Jun 2023 11:15:26 -0700 Subject: [PATCH 12/34] adding the example and shaded distro jar, fixes a NPE when the provided table is expired and no longer accessible. --- flink-connector-bigquery-examples/pom.xml | 131 ++++++++++++++++ .../gcp/bigquery/BigQueryExample.java | 140 ++++++++++++++++++ .../services/BigQueryServiceImpl.java | 19 ++- flink-sql-connector-bigquery/pom.xml | 80 ++++++++++ .../src/main/resources/META-INF/NOTICE | 5 + pom.xml | 2 + 6 files changed, 371 insertions(+), 6 deletions(-) create mode 100644 flink-connector-bigquery-examples/pom.xml create mode 100644 flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java create mode 100644 flink-sql-connector-bigquery/pom.xml create mode 100644 flink-sql-connector-bigquery/src/main/resources/META-INF/NOTICE diff --git a/flink-connector-bigquery-examples/pom.xml b/flink-connector-bigquery-examples/pom.xml new file mode 100644 index 00000000..cc183d30 --- /dev/null +++ b/flink-connector-bigquery-examples/pom.xml @@ -0,0 +1,131 @@ + + + + 4.0.0 + + + org.apache.flink + flink-connector-bigquery-parent + 1.1-SNAPSHOT + + + flink-connector-bigquery-examples + Flink : Connectors : Google BigQuery Examples + jar + + + true + + + + + org.apache.flink + flink-connector-bigquery + ${project.version} + + + org.apache.flink + flink-streaming-java + ${flink.version} + + + + + BigQueryExample + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + false + + + src/test/resources + + + + + org.apache.flink.examples.gcp.bigquery.BigQueryExample + + + log4j2-bqexample.properties + src/main/resources/log4j2-bqexample.properties + + + + + org.apache.flink:flink-connector-bigquery + org.apache.flink:flink-avro + org.apache.flink:flink-metrics-dropwizard + com.google.cloud:google-cloud-bigquerystorage + com.google.*:* + commons-codec:commons-codec + dev.failsafe:* + org.apache.avro:* + org.apache.httpcomponents:* + org.codehaus.mojo:animal-sniffer-annotations + org.conscrypt:* + com.fasterxml.jackson.*:* + org.threeten:* + org.checkerframework:* + io.dropwizard.metrics:* + io.grpc:* + io.opencensus:* + io.perfmark:* + + + + + com.google + org.apache.flink.examples.gcp.bigquery.shaded.com.google + + + + + + org.apache.flink:flink-connector-bigquery-examples* + + org/apache/flink/examples/gcp/bigquery/** + + + + + + + + + + \ No newline at end of file diff --git a/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java b/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java new file mode 100644 index 00000000..37874080 --- /dev/null +++ b/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.examples.gcp.bigquery; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Collector; + +import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; +import com.google.cloud.flink.bigquery.source.BigQuerySource; +import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; +import org.apache.avro.generic.GenericRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A simple BigQuery table read example with Flink's DataStream API. + * + *

The Flink pipeline will try to read the specified BigQuery table, potentially limiting the + * element count to the specified row restriction and limit count, returning {@link GenericRecord} + * representing the rows, to finally prints out some aggregated values given the provided payload's + * field. + * + *

Note on row restriction: In case of including a restriction with a temporal reference, + * something like {@code "TIMESTAMP_TRUNC(ingestion_timestamp, HOUR) = '2023-06-20 19:00:00'"}, and + * launching the job from Flink's Rest API is known the single quotes are not supported and will + * make the pipeline fail. As a workaround for that case using \u0027 as a replacement will make it + * work, example {@code "TIMESTAMP_TRUNC(ingestion_timestamp, HOUR) = \u00272023-06-20 + * 19:00:00\u0027"}. + */ +public class BigQueryExample { + + private static final Logger LOG = LoggerFactory.getLogger(BigQueryExample.class); + + public static void main(String[] args) throws Exception { + // parse input arguments + final ParameterTool parameterTool = ParameterTool.fromArgs(args); + + if (parameterTool.getNumberOfParameters() < 4) { + LOG.error( + "Missing parameters!\n" + + "Usage: flink run BigQuery.jar" + + " --gcp-project --bq-dataset " + + " --bq-table --agg-prop " + + " --restriction " + + " --limit "); + return; + } + + String projectName = parameterTool.getRequired("gcp-project"); + String datasetName = parameterTool.getRequired("bq-dataset"); + String tableName = parameterTool.getRequired("bq-table"); + String rowRestriction = parameterTool.get("restriction", "").replace("\\u0027", "'"); + Integer recordLimit = parameterTool.getInt("limit", -1); + String recordPropertyToAggregate = parameterTool.getRequired("agg-prop"); + + runFlinkJob( + projectName, + datasetName, + tableName, + recordPropertyToAggregate, + rowRestriction, + recordLimit); + } + + private static void runFlinkJob( + String projectName, + String datasetName, + String tableName, + String recordPropertyToAggregate, + String rowRestriction, + Integer limit) + throws Exception { + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(60000L); + + /** + * we will be reading avro generic records from BigQuery, and in this case we are assuming + * the GOOGLE_APPLICATION_CREDENTIALS env variable will be present in the execution runtime. + * In case of needing authenticate differently, the credentials builder (part of the + * BigQueryConnectOptions) should enable capturing the credentials from various sources. + */ + BigQuerySource bqSource = + BigQuerySource.readAvros( + BigQueryReadOptions.builder() + .setBigQueryConnectOptions( + BigQueryConnectOptions.builder() + .setProjectId(projectName) + .setDataset(datasetName) + .setTable(tableName) + .build()) + .setRowRestriction(rowRestriction) + .build(), + limit); + + env.fromSource(bqSource, WatermarkStrategy.noWatermarks(), "BigQuerySource") + .flatMap(new FlatMapper(recordPropertyToAggregate)) + .keyBy(t -> t.f0) + .max("f1") + .print(); + + env.execute("Flink BigQuery Example"); + } + + static class FlatMapper implements FlatMapFunction> { + + private final String recordPropertyToAggregate; + + public FlatMapper(String recordPropertyToAggregate) { + this.recordPropertyToAggregate = recordPropertyToAggregate; + } + + @Override + public void flatMap(GenericRecord record, Collector> out) + throws Exception { + out.collect( + Tuple2.of( + (String) record.get(recordPropertyToAggregate).toString(), 1)); + } + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServiceImpl.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServiceImpl.java index 9e6fd32e..18d89d68 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServiceImpl.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServiceImpl.java @@ -206,7 +206,8 @@ public List retrieveTablePartitions(String project, String dataset, Stri } catch (Exception ex) { throw new RuntimeException( String.format( - "Problems while trying to retrieve table partitions (table: %s.%s.%s).", + "Problems while trying to retrieve table partitions" + + " (table: %s.%s.%s).", project, dataset, table), ex); } @@ -251,7 +252,8 @@ public Optional> retrievePartitionColumnName } catch (Exception ex) { throw new RuntimeException( String.format( - "Problems while trying to retrieve table partition's column name (table: %s.%s.%s).", + "Problems while trying to retrieve table partition's" + + " column name (table: %s.%s.%s).", project, dataset, table), ex); } @@ -259,10 +261,15 @@ public Optional> retrievePartitionColumnName @Override public TableSchema getTableSchema(String project, String dataset, String table) { - return SchemaTransform.bigQuerySchemaToTableSchema( - bigQuery.getTable(TableId.of(project, dataset, table)) - .getDefinition() - .getSchema()); + return Optional.ofNullable(bigQuery.getTable(TableId.of(project, dataset, table))) + .map(t -> t.getDefinition().getSchema()) + .map(schema -> SchemaTransform.bigQuerySchemaToTableSchema(schema)) + .orElseThrow( + () -> + new IllegalArgumentException( + String.format( + "The provided table %s.%s.%s does not exists.", + project, dataset, table))); } @Override diff --git a/flink-sql-connector-bigquery/pom.xml b/flink-sql-connector-bigquery/pom.xml new file mode 100644 index 00000000..48f8af4f --- /dev/null +++ b/flink-sql-connector-bigquery/pom.xml @@ -0,0 +1,80 @@ + + + + + 4.0.0 + + + flink-connector-bigquery-parent + org.apache.flink + 1.1-SNAPSHOT + + + flink-sql-connector-bigquery + Flink : Connectors : SQL : Google BigQuery + + jar + + + true + + + + + org.apache.flink + flink-connector-bigquery + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + false + + + *:* + + + + + com.google + org.apache.flink.connector.bigquery.shaded.com.google + + + + + + + + + + diff --git a/flink-sql-connector-bigquery/src/main/resources/META-INF/NOTICE b/flink-sql-connector-bigquery/src/main/resources/META-INF/NOTICE new file mode 100644 index 00000000..27b8ca73 --- /dev/null +++ b/flink-sql-connector-bigquery/src/main/resources/META-INF/NOTICE @@ -0,0 +1,5 @@ +flink-sql-connector-bigquery +Copyright 2023 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). diff --git a/pom.xml b/pom.xml index 7b9f9568..3d509416 100644 --- a/pom.xml +++ b/pom.xml @@ -66,6 +66,8 @@ under the License. flink-connector-bigquery + flink-sql-connector-bigquery + flink-connector-bigquery-examples From cab9115ab1dff6baec1cf7f422adb80f01e218af Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Thu, 6 Jul 2023 13:36:13 -0700 Subject: [PATCH 13/34] fixing the package name for the schema namespace --- .../cloud/flink/bigquery/common/utils/SchemaTransform.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java index 6b769442..c7d6e3b2 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java @@ -42,7 +42,7 @@ */ public class SchemaTransform { - static final String NAMESPACE = "org.apache.flink.connector.bigquery"; + static final String NAMESPACE = "com.google.cloud.flink.bigquery"; /** * Defines the valid mapping between BigQuery types and native Avro types. * From 6b08119b29c515f810c4601f34de2172315f6f5e Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 11 Jul 2023 15:13:18 -0700 Subject: [PATCH 14/34] fixing lift recommendations and spotless --- .../cloud/flink/bigquery/services/BigQueryServices.java | 4 ++-- .../flink/bigquery/source/config/BigQueryReadOptions.java | 4 ++-- .../bigquery/source/split/BigQuerySourceSplitAssigner.java | 1 - 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServices.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServices.java index 6eb3cd08..ef768e50 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServices.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServices.java @@ -75,9 +75,9 @@ interface BigQueryServerStream extends Iterable, Serializable { /** An interface representing a client object for making calls to the BigQuery Storage API. */ interface StorageReadClient extends AutoCloseable { /** - * Create a new BigQuery storage read session against an existing table. + * Create a new BigQuery storage read session against an existing table. * - * @param request the create session request object. + * @param request the create session request object. * @return A BigQuery storage read session. */ ReadSession createReadSession(CreateReadSessionRequest request); diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java index 8587a4b9..a241373d 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java @@ -130,7 +130,7 @@ public abstract static class Builder { * @param query A BigQuery standard SQL query. * @param projectId A GCP project where the query will run. * @return This {@link Builder} instance. - * @throws IOException + * @throws IOException In case of problems while setting up the credentials options. */ public Builder setQueryAndExecutionProject(String query, String projectId) throws IOException { @@ -145,7 +145,7 @@ public Builder setQueryAndExecutionProject(String query, String projectId) * @param projectId A GCP project where the query will run. * @param credentialsOptions The GCP credentials options. * @return This {@link Builder} instance. - * @throws IOException + * @throws IOException In case of problems while setting up the credentials options. */ public Builder setQueryWithExecutionProjectAndCredentialsOptions( String query, String projectId, CredentialsOptions credentialsOptions) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssigner.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssigner.java index 871356b5..59f941f1 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssigner.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssigner.java @@ -73,7 +73,6 @@ public BigQuerySourceSplitAssigner( * that query and then return a modified version of the connect options pointing to the * temporary location (project, dataset and table) of the query results. * - * @param readOptions The configured read options. * @return The BigQuery connect options with the right project, dataset and table given the * specified configuration. */ From 3ffb582d6d288aecae12200602c50d134a095fab Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 11 Jul 2023 16:09:53 -0700 Subject: [PATCH 15/34] fixes namespace error in test and spotless --- .../flink/bigquery/common/utils/SchemaTransform.java | 8 +++++--- .../flink/bigquery/common/utils/SchemaTransformTest.java | 4 ++-- .../cloud/flink/bigquery/fakes/StorageClientFaker.java | 5 ++++- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java index c7d6e3b2..744e55f2 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java @@ -42,7 +42,7 @@ */ public class SchemaTransform { - static final String NAMESPACE = "com.google.cloud.flink.bigquery"; + public static final String DEFAULT_NAMESPACE = "com.google.cloud.flink.bigquery"; /** * Defines the valid mapping between BigQuery types and native Avro types. * @@ -86,7 +86,7 @@ public static Schema toGenericAvroSchema( return Schema.createRecord( schemaName, "Translated Avro Schema for " + schemaName, - namespace == null ? NAMESPACE : namespace, + namespace == null ? DEFAULT_NAMESPACE : namespace, false, avroFields); } @@ -94,7 +94,9 @@ public static Schema toGenericAvroSchema( public static Schema toGenericAvroSchema( String schemaName, List fieldSchemas) { return toGenericAvroSchema( - schemaName, fieldSchemas, hasNamespaceCollision(fieldSchemas) ? NAMESPACE : null); + schemaName, + fieldSchemas, + hasNamespaceCollision(fieldSchemas) ? DEFAULT_NAMESPACE : null); } // To maintain backwards compatibility we only disambiguate collisions in the field namespaces diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransformTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransformTest.java index c0976952..848f0d29 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransformTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransformTest.java @@ -178,7 +178,7 @@ public void testConvertBigQuerySchemaToAvroSchema() { Schema.createRecord( "scion", "Translated Avro Schema for scion", - SchemaTransform.NAMESPACE, + SchemaTransform.DEFAULT_NAMESPACE, false, ImmutableList.of( new Schema.Field( @@ -194,7 +194,7 @@ public void testConvertBigQuerySchemaToAvroSchema() { Schema.createRecord( "associates", "Translated Avro Schema for associates", - SchemaTransform.NAMESPACE, + SchemaTransform.DEFAULT_NAMESPACE, false, ImmutableList.of( new Schema.Field( diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java index a22aa268..3eef94dd 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java @@ -38,6 +38,7 @@ import com.google.cloud.bigquery.storage.v1.StreamStats; import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; import com.google.cloud.flink.bigquery.common.config.CredentialsOptions; +import com.google.cloud.flink.bigquery.common.utils.SchemaTransform; import com.google.cloud.flink.bigquery.services.BigQueryServices; import com.google.cloud.flink.bigquery.services.QueryResultInfo; import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; @@ -203,7 +204,9 @@ public void close() {} "{\"namespace\": \"project.dataset\",\n" + " \"type\": \"record\",\n" + " \"name\": \"queryresultschema\",\n" - + " \"namespace\": \"org.apache.flink.connector.bigquery\",\n" + + " \"namespace\": \"" + + SchemaTransform.DEFAULT_NAMESPACE + + "\",\n" + " \"doc\": \"Translated Avro Schema for queryresultschema\",\n" + SIMPLE_AVRO_SCHEMA_FIELDS_STRING + "}"; From f473d5745a253a7d4c2120af7068790d437cc0a8 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Thu, 27 Jul 2023 14:47:31 -0700 Subject: [PATCH 16/34] addressing comments from review --- .../bigquery/services/BigQueryServices.java | 4 +-- .../enumerator/BigQuerySourceEnumState.java | 24 ++++++++--------- .../BigQuerySourceEnumStateSerializer.java | 2 +- .../source/split/BigQuerySourceSplit.java | 8 +++--- .../split/BigQuerySourceSplitSerializer.java | 12 ++++----- .../split/BigQuerySourceSplitState.java | 2 +- ...BigQuerySourceEnumStateSerializerTest.java | 6 ++--- .../BigQuerySourceSplitSerializerTest.java | 6 ++--- .../split/BigQuerySourceSplitStateTest.java | 26 ++++++++++++++++--- 9 files changed, 55 insertions(+), 35 deletions(-) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServices.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServices.java index 6eb3cd08..ef768e50 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServices.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServices.java @@ -75,9 +75,9 @@ interface BigQueryServerStream extends Iterable, Serializable { /** An interface representing a client object for making calls to the BigQuery Storage API. */ interface StorageReadClient extends AutoCloseable { /** - * Create a new BigQuery storage read session against an existing table. + * Create a new BigQuery storage read session against an existing table. * - * @param request the create session request object. + * @param request the create session request object. * @return A BigQuery storage read session. */ ReadSession createReadSession(CreateReadSessionRequest request); diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumState.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumState.java index 81614eac..25c0db29 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumState.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumState.java @@ -114,17 +114,17 @@ public boolean equals(Object obj) { @Override public String toString() { - return "BigQuerySourceEnumState{" - + "remaniningTableStreams=" - + remaniningTableStreams - + ", completedTableStreams=" - + completedTableStreams - + ", remainingSourceSplits=" - + remainingSourceSplits - + ", assignedSourceSplits=" - + assignedSourceSplits - + ", initialized=" - + initialized - + '}'; + return String.format( + "BigQuerySourceEnumState{" + + "remaniningTableStreams=%s" + + ", completedTableStreams=%s" + + ", remainingSourceSplits=%s" + + ", assignedSourceSplits=%s" + + ", initialized=%s}", + remaniningTableStreams, + completedTableStreams, + remainingSourceSplits, + assignedSourceSplits, + initialized); } } diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializer.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializer.java index 3551aaa9..1352c226 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializer.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializer.java @@ -45,7 +45,7 @@ private BigQuerySourceEnumStateSerializer() { @Override public int getVersion() { - return BigQuerySourceSplitSerializer.CURRENT_VERSION; + return BigQuerySourceSplitSerializer.VERSION; } @Override diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplit.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplit.java index d8672037..8fd2c0ac 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplit.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplit.java @@ -27,14 +27,14 @@ public class BigQuerySourceSplit implements SourceSplit, Serializable { private final String streamName; - private final Integer offset; + private final Long offset; public BigQuerySourceSplit(String streamName) { this.streamName = streamName; - this.offset = 0; + this.offset = 0L; } - public BigQuerySourceSplit(String streamName, Integer offset) { + public BigQuerySourceSplit(String streamName, Long offset) { this.streamName = streamName; this.offset = offset; } @@ -48,7 +48,7 @@ public String getStreamName() { return streamName; } - public Integer getOffset() { + public Long getOffset() { return offset; } diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializer.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializer.java index 8c6bd83b..fb80b181 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializer.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializer.java @@ -33,7 +33,7 @@ public class BigQuerySourceSplitSerializer public static final BigQuerySourceSplitSerializer INSTANCE = new BigQuerySourceSplitSerializer(); // This version should be bumped after modifying the source split or the enum states. - public static final int CURRENT_VERSION = 0; + public static final int VERSION = 0; private BigQuerySourceSplitSerializer() { // singleton instance @@ -41,7 +41,7 @@ private BigQuerySourceSplitSerializer() { @Override public int getVersion() { - return CURRENT_VERSION; + return VERSION; } @Override @@ -61,7 +61,7 @@ public BigQuerySourceSplit deserialize(int version, byte[] serialized) throws IO throw new IllegalArgumentException( String.format( "The provided serializer version (%d) is not expected (expected : %s).", - version, CURRENT_VERSION)); + version, VERSION)); } // VERSION 0 deserialization try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); @@ -73,15 +73,15 @@ public BigQuerySourceSplit deserialize(int version, byte[] serialized) throws IO public void serializeBigQuerySourceSplit(DataOutputStream out, BigQuerySourceSplit split) throws IOException { out.writeUTF(split.getStreamName()); - out.writeInt(split.getOffset()); + out.writeLong(split.getOffset()); } public BigQuerySourceSplit deserializeBigQuerySourceSplit(int version, DataInputStream in) throws IOException { switch (version) { - case CURRENT_VERSION: + case VERSION: String streamName = in.readUTF(); - int offset = in.readInt(); + long offset = in.readLong(); return new BigQuerySourceSplit(streamName, offset); default: throw new IOException("Unknown version: " + version); diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitState.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitState.java index 009ab58d..b5527359 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitState.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitState.java @@ -24,7 +24,7 @@ @Internal public class BigQuerySourceSplitState { private final BigQuerySourceSplit split; - private Integer offset; + private Long offset; public BigQuerySourceSplitState(BigQuerySourceSplit split) { this.split = split; diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializerTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializerTest.java index dfdda05e..9e10da5b 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializerTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializerTest.java @@ -42,7 +42,7 @@ private BigQuerySourceEnumState create() { completedTableStreams.add("first stream"); List remainingSourceSplits = new ArrayList<>(); - remainingSourceSplits.add(new BigQuerySourceSplit("second stream", 0)); + remainingSourceSplits.add(new BigQuerySourceSplit("second stream", 0L)); Map assignedSourceSplits = new TreeMap<>(); assignedSourceSplits.put("key1", remainingSourceSplits.get(0)); @@ -63,7 +63,7 @@ public void testEnumStateSerializerInitialState() throws IOException { BigQuerySourceEnumState enumState1 = BigQuerySourceEnumStateSerializer.INSTANCE.deserialize( - BigQuerySourceSplitSerializer.CURRENT_VERSION, serialized); + BigQuerySourceSplitSerializer.VERSION, serialized); Assert.assertEquals(initialState, enumState1); Assert.assertEquals(initialState.hashCode(), enumState1.hashCode()); @@ -77,7 +77,7 @@ public void testEnumStateSerializer() throws IOException { BigQuerySourceEnumState enumState1 = BigQuerySourceEnumStateSerializer.INSTANCE.deserialize( - BigQuerySourceSplitSerializer.CURRENT_VERSION, serialized); + BigQuerySourceSplitSerializer.VERSION, serialized); Assert.assertEquals(enumState, enumState1); } diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializerTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializerTest.java index a7af57fb..7c8a92e3 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializerTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializerTest.java @@ -26,20 +26,20 @@ public class BigQuerySourceSplitSerializerTest { @Test public void testSplitSerializer() throws IOException { - BigQuerySourceSplit split = new BigQuerySourceSplit("some stream name", 10); + BigQuerySourceSplit split = new BigQuerySourceSplit("some stream name", 10L); byte[] serialized = BigQuerySourceSplitSerializer.INSTANCE.serialize(split); BigQuerySourceSplit split1 = BigQuerySourceSplitSerializer.INSTANCE.deserialize( - BigQuerySourceSplitSerializer.CURRENT_VERSION, serialized); + BigQuerySourceSplitSerializer.VERSION, serialized); Assert.assertEquals(split, split1); } @Test(expected = IllegalArgumentException.class) public void testWrongSerializerVersion() throws IOException { - BigQuerySourceSplit split = new BigQuerySourceSplit("some stream name", 10); + BigQuerySourceSplit split = new BigQuerySourceSplit("some stream name", 10L); byte[] serialized = BigQuerySourceSplitSerializer.INSTANCE.serialize(split); diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java index c9725034..39ec5e59 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java @@ -26,22 +26,42 @@ public class BigQuerySourceSplitStateTest { public void testSplitStateTransformation() { String streamName = "somestream"; - BigQuerySourceSplit originalSplit = new BigQuerySourceSplit(streamName, 10); + BigQuerySourceSplit originalSplit = new BigQuerySourceSplit(streamName, 10L); Assertions.assertThat(originalSplit.splitId()).isEqualTo(streamName); BigQuerySourceSplitState splitState = new BigQuerySourceSplitState(originalSplit); Assertions.assertThat(splitState.toBigQuerySourceSplit()).isEqualTo(originalSplit); } + @Test + public void testSplitsEquals() { + + String streamName1 = "somestream"; + BigQuerySourceSplit split1 = new BigQuerySourceSplit(streamName1, 10L); + String streamName2 = "somestream"; + BigQuerySourceSplit split2 = new BigQuerySourceSplit(streamName2, 10L); + Assertions.assertThat(split1).isEqualTo(split2); + + BigQuerySourceSplitState splitState1 = new BigQuerySourceSplitState(split1); + BigQuerySourceSplitState splitState2 = new BigQuerySourceSplitState(split2); + Assertions.assertThat(splitState1).isEqualTo(splitState2); + + BigQuerySourceSplit split3 = new BigQuerySourceSplit(streamName2, 11L); + Assertions.assertThat(split1).isNotEqualTo(split3); + + BigQuerySourceSplitState splitState3 = new BigQuerySourceSplitState(split3); + Assertions.assertThat(splitState1).isNotEqualTo(splitState3); + } + @Test public void testSplitStateMutation() { String streamName = "somestream"; - BigQuerySourceSplit originalSplit = new BigQuerySourceSplit(streamName, 10); + BigQuerySourceSplit originalSplit = new BigQuerySourceSplit(streamName, 10L); BigQuerySourceSplitState splitState = new BigQuerySourceSplitState(originalSplit); splitState.updateOffset(); - BigQuerySourceSplit otherSplit = new BigQuerySourceSplit(streamName, 11); + BigQuerySourceSplit otherSplit = new BigQuerySourceSplit(streamName, 11L); Assertions.assertThat(splitState.toBigQuerySourceSplit()).isEqualTo(otherSplit); Assertions.assertThat(splitState.toBigQuerySourceSplit().hashCode()) From ceabb12d4a1662f901686492d6b679239467bddf Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 1 Aug 2023 15:21:27 -0700 Subject: [PATCH 17/34] fixed type reference Int -> Long --- .../source/split/reader/BigQuerySourceSplitReader.java | 8 ++++---- .../split/reader/BigQuerySourceSplitReaderTest.java | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReader.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReader.java index a5840925..8b67fa85 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReader.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReader.java @@ -65,7 +65,7 @@ public class BigQuerySourceSplitReader implements SplitReader readStreamIterator = null; @@ -84,7 +84,7 @@ public BigQuerySourceSplitReader( new SlidingWindowReservoir(500))))); } - Integer offsetToFetch(BigQuerySourceSplit split) { + Long offsetToFetch(BigQuerySourceSplit split) { // honor what is coming as checkpointed if (split.getOffset() > 0) { readSoFar = split.getOffset(); @@ -235,7 +235,7 @@ public RecordsWithSplitIds fetch() throws IOException { readSoFar, splitTimeMs, assignedSplit.splitId()); - readSoFar = 0; + readSoFar = 0L; assignedSplits.poll(); readStreamIterator = null; respBuilder.addFinishedSplit(assignedSplit.splitId()); @@ -293,7 +293,7 @@ public void close() throws Exception { assignedSplits.toString()); if (!closed) { closed = true; - readSoFar = 0; + readSoFar = 0L; readStreamIterator = null; // complete closing with what may be needed } diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReaderTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReaderTest.java index fda7634c..72aadf34 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReaderTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReaderTest.java @@ -49,8 +49,8 @@ public void testSplitReaderSmall() throws IOException { reader.wakeUp(); String splitName = "stream1"; - BigQuerySourceSplit split = new BigQuerySourceSplit(splitName, 0); - BigQuerySourceSplit split2 = new BigQuerySourceSplit("stream2", 0); + BigQuerySourceSplit split = new BigQuerySourceSplit(splitName, 0L); + BigQuerySourceSplit split2 = new BigQuerySourceSplit("stream2", 0L); SplitsAddition change = new SplitsAddition<>(Lists.newArrayList(split, split2)); @@ -99,7 +99,7 @@ public void testSplitReaderMultipleFetch() throws IOException { reader.wakeUp(); String splitName = "stream1"; - BigQuerySourceSplit split = new BigQuerySourceSplit(splitName, 0); + BigQuerySourceSplit split = new BigQuerySourceSplit(splitName, 0L); SplitsAddition change = new SplitsAddition<>(Lists.newArrayList(split)); From e96ff59a6aaacfbe4a792284dbe2ccaf2477ce65 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 8 Aug 2023 16:09:51 -0700 Subject: [PATCH 18/34] addressing comments from review --- .../split/reader/BigQuerySourceSplitReader.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReader.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReader.java index 8b67fa85..b1070477 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReader.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReader.java @@ -173,12 +173,15 @@ public RecordsWithSplitIds fetch() throws IOException { readerContext.getIndexOfSubtask(), assignedSplit.getStreamName()); } - if (avroSchema == null && response.hasAvroSchema()) { - // this will happen only the first time we read from a particular stream - avroSchema = new Schema.Parser().parse(response.getAvroSchema().getSchema()); - } else if (avroSchema == null && !response.hasAvroSchema()) { - throw new IllegalArgumentException( - "Avro schema not initialized and not available in the response."); + if (avroSchema == null) { + if (response.hasAvroSchema()) { + // this will happen only the first time we read from a particular stream + avroSchema = + new Schema.Parser().parse(response.getAvroSchema().getSchema()); + } else { + throw new IllegalArgumentException( + "Avro schema not initialized and not available in the response."); + } } Long decodeStart = System.currentTimeMillis(); List recordList = From c492f0227325aecee412c7ffca601a6a86cd53c3 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 8 Aug 2023 16:38:43 -0700 Subject: [PATCH 19/34] improved hashcode and equals readability --- .../source/config/BigQueryReadOptions.java | 34 +++++++------------ 1 file changed, 13 insertions(+), 21 deletions(-) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java index a241373d..1ba084a6 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java @@ -61,13 +61,12 @@ public abstract class BigQueryReadOptions implements Serializable { @Override public final int hashCode() { - int hash = 5; - hash = 61 * hash + Objects.hashCode(getColumnNames()); - hash = 61 * hash + Objects.hashCode(getRowRestriction()); - hash = 61 * hash + Objects.hashCode(getSnapshotTimestampInMillis()); - hash = 61 * hash + Objects.hashCode(getMaxStreamCount()); - hash = 61 * hash + Objects.hashCode(getBigQueryConnectOptions()); - return hash; + return Objects.hash( + getColumnNames(), + getRowRestriction(), + getSnapshotTimestampInMillis(), + getMaxStreamCount(), + getBigQueryConnectOptions()); } @Override @@ -82,20 +81,13 @@ public final boolean equals(Object obj) { return false; } final BigQueryReadOptions other = (BigQueryReadOptions) obj; - if (!Objects.equals(this.getColumnNames(), other.getColumnNames())) { - return false; - } - if (!Objects.equals(this.getRowRestriction(), other.getRowRestriction())) { - return false; - } - if (!Objects.equals( - this.getSnapshotTimestampInMillis(), other.getSnapshotTimestampInMillis())) { - return false; - } - if (!Objects.equals(this.getMaxStreamCount(), other.getMaxStreamCount())) { - return false; - } - return Objects.equals(this.getBigQueryConnectOptions(), other.getBigQueryConnectOptions()); + return Objects.equals(this.getColumnNames(), other.getColumnNames()) + && Objects.equals(this.getRowRestriction(), other.getRowRestriction()) + && Objects.equals( + this.getSnapshotTimestampInMillis(), other.getSnapshotTimestampInMillis()) + && Objects.equals(this.getMaxStreamCount(), other.getMaxStreamCount()) + && Objects.equals( + this.getBigQueryConnectOptions(), other.getBigQueryConnectOptions()); } /** From dd5be94e8b19ad8fe2856e8ca46c0aaf391fd7ce Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 8 Aug 2023 16:46:14 -0700 Subject: [PATCH 20/34] improve readibility for hashcode and equals --- .../bigquery/services/QueryResultInfo.java | 31 +++++++------------ .../table/BigQueryDynamicTableSource.java | 16 +++------- 2 files changed, 15 insertions(+), 32 deletions(-) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/QueryResultInfo.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/QueryResultInfo.java index da6993a5..7f0605ed 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/QueryResultInfo.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/QueryResultInfo.java @@ -101,13 +101,12 @@ public Optional getDestinationTable() { @Override public int hashCode() { - int hash = 7; - hash = 97 * hash + Objects.hashCode(this.status); - hash = 97 * hash + Objects.hashCode(this.errorMessages); - hash = 97 * hash + Objects.hashCode(this.destinationProject); - hash = 97 * hash + Objects.hashCode(this.destinationDataset); - hash = 97 * hash + Objects.hashCode(this.destinationTable); - return hash; + return Objects.hash( + getStatus(), + getErrorMessages(), + getDestinationProject(), + getDestinationDataset(), + getDestinationTable()); } @Override @@ -122,19 +121,11 @@ public boolean equals(Object obj) { return false; } final QueryResultInfo other = (QueryResultInfo) obj; - if (!Objects.equals(this.destinationProject, other.destinationProject)) { - return false; - } - if (!Objects.equals(this.destinationDataset, other.destinationDataset)) { - return false; - } - if (!Objects.equals(this.destinationTable, other.destinationTable)) { - return false; - } - if (this.status != other.status) { - return false; - } - return Objects.equals(this.errorMessages, other.errorMessages); + return Objects.equals(this.destinationProject, other.destinationProject) + && Objects.equals(this.destinationDataset, other.destinationDataset) + && Objects.equals(this.destinationTable, other.destinationTable) + && Objects.equals(this.status, other.status) + && Objects.equals(this.errorMessages, other.errorMessages); } @Override diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java index bc7cd092..40096300 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java @@ -152,11 +152,7 @@ public Result applyFilters(List filters) { @Override public int hashCode() { - int hash = 5; - hash = 61 * hash + Objects.hashCode(this.readOptions); - hash = 61 * hash + Objects.hashCode(this.producedDataType); - hash = 61 * hash + Objects.hashCode(this.limit); - return hash; + return Objects.hash(this.readOptions, this.producedDataType, this.limit); } @Override @@ -171,13 +167,9 @@ public boolean equals(Object obj) { return false; } final BigQueryDynamicTableSource other = (BigQueryDynamicTableSource) obj; - if (!Objects.equals(this.readOptions, other.readOptions)) { - return false; - } - if (!Objects.equals(this.producedDataType, other.producedDataType)) { - return false; - } - return Objects.equals(this.limit, other.limit); + return Objects.equals(this.readOptions, other.readOptions) + && Objects.equals(this.producedDataType, other.producedDataType) + && Objects.equals(this.limit, other.limit); } @Override From 9aae0afaf4ff6335bc306e4675567ffa26d22eb4 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 8 Aug 2023 20:12:03 -0700 Subject: [PATCH 21/34] changed tests to use google-truth instead of junit or assertj assertion types --- .../common/utils/BigQueryStateSerdeTest.java | 7 ++-- .../common/utils/SchemaTransformTest.java | 35 ++++++++++--------- .../services/BigQueryServicesTest.java | 15 ++++---- .../services/QueryResultInfoTest.java | 26 +++++++------- ...BigQuerySourceEnumStateSerializerTest.java | 11 +++--- .../BigQuerySourceSplitAssignerTest.java | 24 +++++++------ .../BigQuerySourceSplitSerializerTest.java | 7 ++-- .../split/BigQuerySourceSplitStateTest.java | 24 ++++++------- .../reader/BigQuerySourceSplitReaderTest.java | 31 ++++++++-------- pom.xml | 12 +++---- tools/maven/suppressions.xml | 2 ++ 11 files changed, 103 insertions(+), 91 deletions(-) diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/BigQueryStateSerdeTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/BigQueryStateSerdeTest.java index 9382b7cf..46604b3c 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/BigQueryStateSerdeTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/BigQueryStateSerdeTest.java @@ -19,7 +19,6 @@ import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; import org.apache.flink.shaded.guava30.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; import org.junit.Test; import java.io.ByteArrayInputStream; @@ -31,6 +30,8 @@ import java.util.List; import java.util.Map; +import static com.google.common.truth.Truth.assertThat; + /** */ public class BigQueryStateSerdeTest { @@ -50,7 +51,7 @@ public void testListSerDe() throws IOException { List deserialized = BigQueryStateSerde.deserializeList(in, DataInput::readUTF); - Assertions.assertThat(original).isEqualTo(deserialized); + assertThat(original).isEqualTo(deserialized); } } } @@ -76,7 +77,7 @@ public void testMapSerDe() throws IOException { BigQueryStateSerde.deserializeMap( in, DataInput::readUTF, DataInput::readUTF); - Assertions.assertThat(original).isEqualTo(deserialized); + assertThat(original).isEqualTo(deserialized); } } } diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransformTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransformTest.java index c0976952..9fc22daf 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransformTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransformTest.java @@ -26,11 +26,12 @@ import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; -import org.assertj.core.api.Assertions; import org.junit.Test; import java.util.List; +import static com.google.common.truth.Truth.assertThat; + /** */ public class SchemaTransformTest { private final List subFields = @@ -108,70 +109,70 @@ public void testConvertBigQuerySchemaToAvroSchema() { Schema avroSchema = SchemaTransform.toGenericAvroSchema("testSchema", tableSchema.getFields()); - Assertions.assertThat(avroSchema.getField("number").schema()) + assertThat(avroSchema.getField("number").schema()) .isEqualTo(Schema.create(Schema.Type.LONG)); - Assertions.assertThat(avroSchema.getField("species").schema()) + assertThat(avroSchema.getField("species").schema()) .isEqualTo( Schema.createUnion( Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING))); - Assertions.assertThat(avroSchema.getField("quality").schema()) + assertThat(avroSchema.getField("quality").schema()) .isEqualTo( Schema.createUnion( Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.DOUBLE))); - Assertions.assertThat(avroSchema.getField("quantity").schema()) + assertThat(avroSchema.getField("quantity").schema()) .isEqualTo( Schema.createUnion( Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.LONG))); - Assertions.assertThat(avroSchema.getField("birthday").schema()) + assertThat(avroSchema.getField("birthday").schema()) .isEqualTo( Schema.createUnion( Schema.create(Schema.Type.NULL), LogicalTypes.timestampMicros() .addToSchema(Schema.create(Schema.Type.LONG)))); - Assertions.assertThat(avroSchema.getField("birthdayMoney").schema()) + assertThat(avroSchema.getField("birthdayMoney").schema()) .isEqualTo( Schema.createUnion( Schema.create(Schema.Type.NULL), LogicalTypes.decimal(38, 9) .addToSchema(Schema.create(Schema.Type.BYTES)))); - Assertions.assertThat(avroSchema.getField("lotteryWinnings").schema()) + assertThat(avroSchema.getField("lotteryWinnings").schema()) .isEqualTo( Schema.createUnion( Schema.create(Schema.Type.NULL), LogicalTypes.decimal(77, 38) .addToSchema(Schema.create(Schema.Type.BYTES)))); - Assertions.assertThat(avroSchema.getField("flighted").schema()) + assertThat(avroSchema.getField("flighted").schema()) .isEqualTo( Schema.createUnion( Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.BOOLEAN))); - Assertions.assertThat(avroSchema.getField("sound").schema()) + assertThat(avroSchema.getField("sound").schema()) .isEqualTo( Schema.createUnion( Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.BYTES))); - Assertions.assertThat(avroSchema.getField("anniversaryDate").schema()) + assertThat(avroSchema.getField("anniversaryDate").schema()) .isEqualTo( Schema.createUnion( Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING))); - Assertions.assertThat(avroSchema.getField("anniversaryDatetime").schema()) + assertThat(avroSchema.getField("anniversaryDatetime").schema()) .isEqualTo( Schema.createUnion( Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING))); - Assertions.assertThat(avroSchema.getField("anniversaryTime").schema()) + assertThat(avroSchema.getField("anniversaryTime").schema()) .isEqualTo( Schema.createUnion( Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING))); Schema geoSchema = Schema.create(Schema.Type.STRING); geoSchema.addProp(LogicalType.LOGICAL_TYPE_PROP, "geography_wkt"); - Assertions.assertThat(avroSchema.getField("geoPositions").schema()) + assertThat(avroSchema.getField("geoPositions").schema()) .isEqualTo(Schema.createUnion(Schema.create(Schema.Type.NULL), geoSchema)); - Assertions.assertThat(avroSchema.getField("scion").schema()) + assertThat(avroSchema.getField("scion").schema()) .isEqualTo( Schema.createUnion( Schema.create(Schema.Type.NULL), @@ -188,7 +189,7 @@ public void testConvertBigQuerySchemaToAvroSchema() { Schema.create(Schema.Type.STRING)), null, (Object) null))))); - Assertions.assertThat(avroSchema.getField("associates").schema()) + assertThat(avroSchema.getField("associates").schema()) .isEqualTo( Schema.createArray( Schema.createRecord( @@ -226,6 +227,6 @@ public void testBQSchemaToTableSchema() { TableSchema transformed = SchemaTransform.bigQuerySchemaToTableSchema(schema); - Assertions.assertThat(transformed).isEqualTo(expected); + assertThat(transformed).isEqualTo(expected); } } diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/BigQueryServicesTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/BigQueryServicesTest.java index 6a2ceb6f..6e818258 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/BigQueryServicesTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/BigQueryServicesTest.java @@ -20,11 +20,12 @@ import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; import com.google.cloud.flink.bigquery.common.config.CredentialsOptions; -import org.assertj.core.api.Assertions; import org.junit.Test; import java.io.IOException; +import static com.google.common.truth.Truth.assertThat; + /** */ public class BigQueryServicesTest { @Test @@ -50,14 +51,14 @@ public BigQueryServices.StorageReadClient getStorageClient( .setTestingBigQueryServices(dummyServices) .build()); - Assertions.assertThat(original.getIsTestingEnabled()).isTrue(); - Assertions.assertThat(original.getTestingServices()).isNotNull(); - Assertions.assertThat(original.queryClient()).isNull(); - Assertions.assertThat(original.storageRead()).isNull(); + assertThat(original.getIsTestingEnabled()).isTrue(); + assertThat(original.getTestingServices()).isNotNull(); + assertThat(original.queryClient()).isNull(); + assertThat(original.storageRead()).isNull(); original.defaultImplementation(); - Assertions.assertThat(original.getIsTestingEnabled()).isFalse(); - Assertions.assertThat(original.getTestingServices()).isNull(); + assertThat(original.getIsTestingEnabled()).isFalse(); + assertThat(original.getTestingServices()).isNull(); } } diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/QueryResultInfoTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/QueryResultInfoTest.java index e3f7a87d..929c8c50 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/QueryResultInfoTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/services/QueryResultInfoTest.java @@ -18,36 +18,38 @@ import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; +import com.google.common.truth.Truth8; import org.junit.Test; +import static com.google.common.truth.Truth.assertThat; + /** */ public class QueryResultInfoTest { @Test public void testQueryResultInfoFailed() { QueryResultInfo qri = QueryResultInfo.failed(Lists.newArrayList()); - Assertions.assertThat(qri.getStatus()).isEqualTo(QueryResultInfo.Status.FAILED); - Assertions.assertThat(qri.getDestinationProject()).isEmpty(); - Assertions.assertThat(qri.getDestinationDataset()).isEmpty(); - Assertions.assertThat(qri.getDestinationTable()).isEmpty(); - Assertions.assertThat(qri.getErrorMessages()).isNotEmpty(); + assertThat(qri.getStatus()).isEqualTo(QueryResultInfo.Status.FAILED); + Truth8.assertThat(qri.getDestinationProject()).isEmpty(); + Truth8.assertThat(qri.getDestinationDataset()).isEmpty(); + Truth8.assertThat(qri.getDestinationTable()).isEmpty(); + Truth8.assertThat(qri.getErrorMessages()).isPresent(); } @Test public void testQueryResultInfoSucceeded() { QueryResultInfo qri = QueryResultInfo.succeed("", "", ""); - Assertions.assertThat(qri.getStatus()).isEqualTo(QueryResultInfo.Status.SUCCEED); - Assertions.assertThat(qri.getDestinationProject()).isNotEmpty(); - Assertions.assertThat(qri.getDestinationDataset()).isNotEmpty(); - Assertions.assertThat(qri.getDestinationTable()).isNotEmpty(); - Assertions.assertThat(qri.getErrorMessages().get()).isEmpty(); + assertThat(qri.getStatus()).isEqualTo(QueryResultInfo.Status.SUCCEED); + Truth8.assertThat(qri.getDestinationProject()).isPresent(); + Truth8.assertThat(qri.getDestinationDataset()).isPresent(); + Truth8.assertThat(qri.getDestinationTable()).isPresent(); + assertThat(qri.getErrorMessages().get()).isEmpty(); } @Test public void testNotEquals() { QueryResultInfo succeed = QueryResultInfo.succeed("", "", ""); QueryResultInfo failed = QueryResultInfo.failed(Lists.newArrayList()); - Assertions.assertThat(succeed).isNotEqualTo(failed); + assertThat(succeed).isNotEqualTo(failed); } } diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializerTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializerTest.java index ae96a191..e116cf74 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializerTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/enumerator/BigQuerySourceEnumStateSerializerTest.java @@ -18,7 +18,6 @@ import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplit; import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplitSerializer; -import org.junit.Assert; import org.junit.Test; import java.io.IOException; @@ -27,6 +26,8 @@ import java.util.Map; import java.util.TreeMap; +import static com.google.common.truth.Truth.assertThat; + /** */ public class BigQuerySourceEnumStateSerializerTest { @@ -65,8 +66,8 @@ public void testEnumStateSerializerInitialState() throws IOException { BigQuerySourceEnumStateSerializer.INSTANCE.deserialize( BigQuerySourceSplitSerializer.VERSION, serialized); - Assert.assertEquals(initialState, enumState1); - Assert.assertEquals(initialState.hashCode(), enumState1.hashCode()); + assertThat(initialState).isEqualTo(enumState1); + assertThat(initialState.hashCode()).isEqualTo(enumState1.hashCode()); } @Test @@ -79,7 +80,7 @@ public void testEnumStateSerializer() throws IOException { BigQuerySourceEnumStateSerializer.INSTANCE.deserialize( BigQuerySourceSplitSerializer.VERSION, serialized); - Assert.assertEquals(enumState, enumState1); + assertThat(enumState).isEqualTo(enumState1); } @Test(expected = IllegalArgumentException.class) @@ -91,6 +92,6 @@ public void testWrongSerializerVersion() throws IOException { BigQuerySourceEnumStateSerializer.INSTANCE.deserialize(1000, serialized); // should never reach here - Assert.fail(); + assertThat(false).isTrue(); } } diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssignerTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssignerTest.java index 23aa6b56..fa77a546 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssignerTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitAssignerTest.java @@ -21,13 +21,15 @@ import com.google.cloud.flink.bigquery.fakes.StorageClientFaker; import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; import com.google.cloud.flink.bigquery.source.enumerator.BigQuerySourceEnumState; -import org.junit.Assert; +import com.google.common.truth.Truth8; import org.junit.Before; import org.junit.Test; import java.io.IOException; import java.util.Optional; +import static com.google.common.truth.Truth.assertThat; + /** */ public class BigQuerySourceSplitAssignerTest { @@ -51,29 +53,29 @@ public void testAssignment() { // should retrieve the first split representing the firt stream Optional maybeSplit = assigner.getNext(); - Assert.assertTrue(maybeSplit.isPresent()); + Truth8.assertThat(maybeSplit).isPresent(); // should retrieve the second split representing the second stream maybeSplit = assigner.getNext(); - Assert.assertTrue(maybeSplit.isPresent()); + Truth8.assertThat(maybeSplit).isPresent(); BigQuerySourceSplit split = maybeSplit.get(); // no more splits should be available maybeSplit = assigner.getNext(); - Assert.assertTrue(!maybeSplit.isPresent()); - Assert.assertTrue(assigner.noMoreSplits()); + Truth8.assertThat(maybeSplit).isEmpty(); + assertThat(assigner.noMoreSplits()).isTrue(); // lets check on the enum state BigQuerySourceEnumState state = assigner.snapshotState(0); - Assert.assertTrue(state.getRemaniningTableStreams().isEmpty()); - Assert.assertTrue(state.getRemainingSourceSplits().isEmpty()); + assertThat(state.getRemaniningTableStreams()).isEmpty(); + assertThat(state.getRemainingSourceSplits()).isEmpty(); // add some splits back assigner.addSplitsBack(Lists.newArrayList(split)); // check again on the enum state state = assigner.snapshotState(0); - Assert.assertTrue(state.getRemaniningTableStreams().isEmpty()); - Assert.assertTrue(!state.getRemainingSourceSplits().isEmpty()); + assertThat(state.getRemaniningTableStreams()).isEmpty(); + assertThat(state.getRemainingSourceSplits()).isNotEmpty(); // empty it again and check assigner.getNext(); maybeSplit = assigner.getNext(); - Assert.assertTrue(!maybeSplit.isPresent()); - Assert.assertTrue(assigner.noMoreSplits()); + Truth8.assertThat(maybeSplit).isEmpty(); + assertThat(assigner.noMoreSplits()).isTrue(); } } diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializerTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializerTest.java index 7c8a92e3..c3811bc1 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializerTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitSerializerTest.java @@ -16,11 +16,12 @@ package com.google.cloud.flink.bigquery.source.split; -import org.junit.Assert; import org.junit.Test; import java.io.IOException; +import static com.google.common.truth.Truth.assertThat; + /** */ public class BigQuerySourceSplitSerializerTest { @@ -34,7 +35,7 @@ public void testSplitSerializer() throws IOException { BigQuerySourceSplitSerializer.INSTANCE.deserialize( BigQuerySourceSplitSerializer.VERSION, serialized); - Assert.assertEquals(split, split1); + assertThat(split).isEqualTo(split1); } @Test(expected = IllegalArgumentException.class) @@ -46,6 +47,6 @@ public void testWrongSerializerVersion() throws IOException { BigQuerySourceSplitSerializer.INSTANCE.deserialize(1000, serialized); // should never reach here - assert (true); + assertThat(true).isFalse(); } } diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java index 39ec5e59..e71a4b82 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java @@ -16,9 +16,10 @@ package com.google.cloud.flink.bigquery.source.split; -import org.assertj.core.api.Assertions; import org.junit.Test; +import static com.google.common.truth.Truth.assertThat; + /** */ public class BigQuerySourceSplitStateTest { @@ -27,10 +28,10 @@ public void testSplitStateTransformation() { String streamName = "somestream"; BigQuerySourceSplit originalSplit = new BigQuerySourceSplit(streamName, 10L); - Assertions.assertThat(originalSplit.splitId()).isEqualTo(streamName); + assertThat(originalSplit.splitId()).isEqualTo(streamName); BigQuerySourceSplitState splitState = new BigQuerySourceSplitState(originalSplit); - Assertions.assertThat(splitState.toBigQuerySourceSplit()).isEqualTo(originalSplit); + assertThat(splitState.toBigQuerySourceSplit()).isEqualTo(originalSplit); } @Test @@ -40,17 +41,17 @@ public void testSplitsEquals() { BigQuerySourceSplit split1 = new BigQuerySourceSplit(streamName1, 10L); String streamName2 = "somestream"; BigQuerySourceSplit split2 = new BigQuerySourceSplit(streamName2, 10L); - Assertions.assertThat(split1).isEqualTo(split2); + assertThat(split1).isEqualTo(split2); BigQuerySourceSplitState splitState1 = new BigQuerySourceSplitState(split1); BigQuerySourceSplitState splitState2 = new BigQuerySourceSplitState(split2); - Assertions.assertThat(splitState1).isEqualTo(splitState2); + assertThat(splitState1).isEqualTo(splitState2); BigQuerySourceSplit split3 = new BigQuerySourceSplit(streamName2, 11L); - Assertions.assertThat(split1).isNotEqualTo(split3); + assertThat(split1).isNotEqualTo(split3); BigQuerySourceSplitState splitState3 = new BigQuerySourceSplitState(split3); - Assertions.assertThat(splitState1).isNotEqualTo(splitState3); + assertThat(splitState1).isNotEqualTo(splitState3); } @Test @@ -63,12 +64,11 @@ public void testSplitStateMutation() { splitState.updateOffset(); BigQuerySourceSplit otherSplit = new BigQuerySourceSplit(streamName, 11L); - Assertions.assertThat(splitState.toBigQuerySourceSplit()).isEqualTo(otherSplit); - Assertions.assertThat(splitState.toBigQuerySourceSplit().hashCode()) - .isEqualTo(otherSplit.hashCode()); + assertThat(splitState.toBigQuerySourceSplit()).isEqualTo(otherSplit); + assertThat(splitState.toBigQuerySourceSplit().hashCode()).isEqualTo(otherSplit.hashCode()); // should be different since they started from different splits - Assertions.assertThat(splitState).isNotEqualTo(new BigQuerySourceSplitState(otherSplit)); - Assertions.assertThat(splitState.hashCode()) + assertThat(splitState).isNotEqualTo(new BigQuerySourceSplitState(otherSplit)); + assertThat(splitState.hashCode()) .isNotEqualTo(new BigQuerySourceSplitState(otherSplit).hashCode()); } } diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReaderTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReaderTest.java index 72aadf34..98a04190 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReaderTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/reader/BigQuerySourceSplitReaderTest.java @@ -27,12 +27,13 @@ import com.google.cloud.flink.bigquery.source.reader.BigQuerySourceReaderContext; import com.google.cloud.flink.bigquery.source.split.BigQuerySourceSplit; import org.apache.avro.generic.GenericRecord; -import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; import java.io.IOException; +import static com.google.common.truth.Truth.assertThat; + /** */ public class BigQuerySourceSplitReaderTest { @@ -60,28 +61,28 @@ public void testSplitReaderSmall() throws IOException { // this should fetch us some data RecordsWithSplitIds records = reader.fetch(); // there is one finished split and is named stream1 - Assert.assertTrue(records.finishedSplits().size() == 1); + assertThat(records.finishedSplits()).hasSize(1); String firstSplit = records.nextSplit(); - Assert.assertNotNull(firstSplit); - Assert.assertTrue(firstSplit.equals(splitName)); + assertThat(firstSplit).isNotNull(); + assertThat(firstSplit).isEqualTo(splitName); int i = 0; while (records.nextRecordFromSplit() != null) { i++; } // there were 10 generic records read - Assert.assertTrue(i == 10); + assertThat(i).isEqualTo(10); // there are no more splits - Assert.assertNull(records.nextSplit()); + assertThat(records.nextSplit()).isNull(); // now there should be another split to process records = reader.fetch(); - Assert.assertTrue(!records.finishedSplits().isEmpty()); + assertThat(records.finishedSplits()).isNotEmpty(); // after processing no more splits can be retrieved records = reader.fetch(); - Assert.assertTrue(records.finishedSplits().isEmpty()); + assertThat(records.finishedSplits()).isEmpty(); } @Test @@ -109,27 +110,27 @@ public void testSplitReaderMultipleFetch() throws IOException { // this should fetch us some data RecordsWithSplitIds records = reader.fetch(); // there shouldn't be a finished split - Assert.assertTrue(records.finishedSplits().isEmpty()); + assertThat(records.finishedSplits()).isEmpty(); String firstPartialSplit = records.nextSplit(); - Assert.assertNotNull(firstPartialSplit); - Assert.assertTrue(firstPartialSplit.equals(splitName)); + assertThat(firstPartialSplit).isNotNull(); + assertThat(firstPartialSplit).isEqualTo(splitName); int i = 0; while (records.nextRecordFromSplit() != null) { i++; } // there were less than 10000 generic records read, the max per fetch - Assert.assertTrue(i <= 10000); + assertThat(i).isLessThan(10001); // there are no more splits - Assert.assertNull(records.nextSplit()); + assertThat(records.nextSplit()).isNull(); // now there should be more data in the split and now should be able to finalize it records = reader.fetch(); - Assert.assertTrue(!records.finishedSplits().isEmpty()); + assertThat(records.finishedSplits()).isNotEmpty(); // after processing no more splits can be retrieved records = reader.fetch(); - Assert.assertTrue(records.finishedSplits().isEmpty()); + assertThat(records.finishedSplits()).isEmpty(); } } diff --git a/pom.xml b/pom.xml index 90bc2911..00f2d1ec 100644 --- a/pom.xml +++ b/pom.xml @@ -51,7 +51,7 @@ under the License. 17.0 5.9.3 - 3.24.2 + 1.1.4 4.11.0 false @@ -114,8 +114,8 @@ under the License. - org.assertj - assertj-core + com.google.truth.extensions + truth-java8-extension test @@ -314,9 +314,9 @@ under the License. - org.assertj - assertj-core - ${assertj.version} + com.google.truth.extensions + truth-java8-extension + ${google-truth.version} test diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml index 5d96995b..5138e27c 100644 --- a/tools/maven/suppressions.xml +++ b/tools/maven/suppressions.xml @@ -23,4 +23,6 @@ under the License. "http://www.puppycrawl.com/dtds/suppressions_1_1.dtd"> + + From 517de82f5bb936807d1312d303bfaa0f238333ab Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 8 Aug 2023 20:57:56 -0700 Subject: [PATCH 22/34] added google-truth to tests --- .../flink/bigquery/source/BigQuerySourceITCase.java | 9 +++++---- tools/maven/suppressions.xml | 1 + 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceITCase.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceITCase.java index 407bd7c9..2db01ec8 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceITCase.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceITCase.java @@ -36,7 +36,6 @@ import com.google.cloud.flink.bigquery.fakes.StorageClientFaker; import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; import com.google.cloud.flink.bigquery.source.reader.deserializer.AvroToRowDataDeserializationSchema; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; @@ -46,6 +45,8 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import static com.google.common.truth.Truth.assertThat; + /** */ @TestInstance(Lifecycle.PER_CLASS) public class BigQuerySourceITCase { @@ -107,7 +108,7 @@ public void testReadCount() throws Exception { .executeAndCollect()); // we only create 2 streams as response - Assertions.assertThat(results).hasSize(TOTAL_ROW_COUNT_PER_STREAM * PARALLELISM); + assertThat(results).hasSize(TOTAL_ROW_COUNT_PER_STREAM * PARALLELISM); } @Test @@ -121,7 +122,7 @@ public void testLimit() throws Exception { env.fromSource(bqSource, WatermarkStrategy.noWatermarks(), "BigQuery-Source") .executeAndCollect(TOTAL_ROW_COUNT_PER_STREAM); - Assertions.assertThat(results).hasSize(limitSize * PARALLELISM); + assertThat(results).hasSize(limitSize * PARALLELISM); } @Test @@ -141,7 +142,7 @@ public void testRecovery() throws Exception { .map(new FailingMapper(failed)) .executeAndCollect()); - Assertions.assertThat(results).hasSize(TOTAL_ROW_COUNT_PER_STREAM * PARALLELISM); + assertThat(results).hasSize(TOTAL_ROW_COUNT_PER_STREAM * PARALLELISM); } private static class FailingMapper diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml index 5138e27c..9b243ef5 100644 --- a/tools/maven/suppressions.xml +++ b/tools/maven/suppressions.xml @@ -24,5 +24,6 @@ under the License. + From 11fce0dba59d0fea38d6f517773d4a0aebcc480c Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 8 Aug 2023 21:13:29 -0700 Subject: [PATCH 23/34] added google-truth to tests --- .../cloud/flink/bigquery/source/BigQuerySourceTest.java | 9 ++++----- .../bigquery/table/BigQueryDynamicTableFactoryTest.java | 7 ++++--- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceTest.java index 19e919e6..23f5e03a 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/BigQuerySourceTest.java @@ -23,11 +23,12 @@ import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; -import org.assertj.core.api.Assertions; import org.junit.Test; import java.io.IOException; +import static com.google.common.truth.Truth.assertThat; + /** */ public class BigQuerySourceTest { @@ -39,8 +40,7 @@ public void testReadAvros() throws IOException { BigQuerySource source = BigQuerySource.readAvros(readOptions); TypeInformation expected = new GenericRecordAvroTypeInfo(StorageClientFaker.SIMPLE_AVRO_SCHEMA); - Assertions.assertThat(source.getDeserializationSchema().getProducedType()) - .isEqualTo(expected); + assertThat(source.getDeserializationSchema().getProducedType()).isEqualTo(expected); } @Test @@ -55,7 +55,6 @@ public void testReadAvrosFromQuery() throws IOException { new GenericRecordAvroTypeInfo( new Schema.Parser() .parse(StorageClientFaker.SIMPLE_AVRO_SCHEMA_FORQUERY_STRING)); - Assertions.assertThat(source.getDeserializationSchema().getProducedType()) - .isEqualTo(expected); + assertThat(source.getDeserializationSchema().getProducedType()).isEqualTo(expected); } } diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableFactoryTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableFactoryTest.java index e0ad3afd..16c812c3 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableFactoryTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableFactoryTest.java @@ -29,7 +29,6 @@ import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; import com.google.cloud.flink.bigquery.table.config.BigQueryConnectorOptions; import org.assertj.core.api.Assertions; -import org.junit.Assert; import org.junit.Test; import java.io.IOException; @@ -39,6 +38,8 @@ import java.util.HashMap; import java.util.Map; +import static com.google.common.truth.Truth.assertThat; + /** Tests for the {@link BigQueryDynamicTableSource} factory class. */ public class BigQueryDynamicTableFactoryTest { @@ -61,7 +62,7 @@ public void testBigQuerySourceCommonProperties() throws IOException { BigQueryDynamicTableSource expectedSource = new BigQueryDynamicTableSource( getConnectorOptions(), SCHEMA.toPhysicalRowDataType()); - Assert.assertEquals("The sources are not equals.", actualSource, expectedSource); + assertThat(actualSource).isEqualTo(expectedSource); } @Test @@ -90,7 +91,7 @@ public void testBigQueryReadProperties() throws IOException { BigQueryDynamicTableSource expected = new BigQueryDynamicTableSource(readOptions, SCHEMA.toPhysicalRowDataType()); - Assertions.assertThat(actual).isEqualTo(expected); + assertThat(actual).isEqualTo(expected); } @Test From 6b28a0c4a31491642d3c70808245006cf779fd79 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 31 Oct 2023 17:18:21 -0700 Subject: [PATCH 24/34] removing guava dependency from file --- .../bigquery/fakes/StorageClientFaker.java | 34 ++++++++++++++----- 1 file changed, 25 insertions(+), 9 deletions(-) diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java index 3eef94dd..ae3b906c 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java @@ -19,8 +19,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.function.SerializableFunction; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobStatistics; import com.google.api.services.bigquery.model.JobStatistics2; @@ -53,6 +51,9 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Optional; @@ -84,7 +85,7 @@ public QueryDataClient getQueryDataClient(CredentialsOptions readOptions) { @Override public List retrieveTablePartitions( String project, String dataset, String table) { - return Lists.newArrayList("2023062811"); + return Arrays.asList("2023062811"); } @Override @@ -216,7 +217,7 @@ public void close() {} public static final TableSchema SIMPLE_BQ_TABLE_SCHEMA = new TableSchema() .setFields( - Lists.newArrayList( + Arrays.asList( new TableFieldSchema() .setName("name") .setType("STRING") @@ -281,9 +282,24 @@ public static List createResponse( double progressAtResponseEnd) { // BigQuery delivers the data in 1024 elements chunks, so we partition the generated list // into multiple ones with that size max. - List> responsesData = Lists.partition(genericRecords, 1024); - - return responsesData.stream() + return IntStream.range(0, genericRecords.size()) + .collect( + () -> new HashMap>(), + (map, idx) -> + map.computeIfAbsent(idx, key -> new ArrayList<>()) + .add(genericRecords.get(idx)), + (map1, map2) -> + map2.entrySet() + .forEach( + entry -> + map1.merge( + entry.getKey(), + entry.getValue(), + (list1, list2) -> { + list1.addAll(list2); + return list1; + }))) + .values().stream() // for each data response chunk we generate a read response object .map( genRecords -> { @@ -305,8 +321,8 @@ public static List createResponse( AvroRows.newBuilder() .setSerializedBinaryRows( ByteString.copyFrom( - outputStream.toByteArray())) - .setRowCount(genRecords.size())) + outputStream + .toByteArray()))) .setAvroSchema( AvroSchema.newBuilder() .setSchema(schema.toString()) From 87780c5f02cf58759bfad2b22868b1bbe0b30260 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Wed, 1 Nov 2023 17:14:29 -0700 Subject: [PATCH 25/34] adding serializable autovalue annotation to avoid storing an Optional field since the dynamic table source and the source impl needs to store the read options as property and it becomes not serializable after changing the read options to use an optional property. --- .../flink/bigquery/common/config/BigQueryConnectOptions.java | 2 ++ .../cloud/flink/bigquery/source/config/BigQueryReadOptions.java | 2 ++ 2 files changed, 4 insertions(+) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/config/BigQueryConnectOptions.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/config/BigQueryConnectOptions.java index 8144ebda..acb91238 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/config/BigQueryConnectOptions.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/config/BigQueryConnectOptions.java @@ -20,6 +20,7 @@ import org.apache.flink.util.function.SerializableSupplier; import com.google.auto.value.AutoValue; +import com.google.auto.value.extension.serializable.SerializableAutoValue; import com.google.cloud.flink.bigquery.services.BigQueryServices; import javax.annotation.Nullable; @@ -30,6 +31,7 @@ /** BigQuery client connection configuration. */ @AutoValue +@SerializableAutoValue @PublicEvolving public abstract class BigQueryConnectOptions implements Serializable { diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java index d83c950d..d325b220 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/source/config/BigQueryReadOptions.java @@ -22,6 +22,7 @@ import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; import com.google.auto.value.AutoValue; +import com.google.auto.value.extension.serializable.SerializableAutoValue; import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; import com.google.cloud.flink.bigquery.common.config.CredentialsOptions; import org.threeten.bp.Instant; @@ -37,6 +38,7 @@ /** The options available to read data from BigQuery. */ @AutoValue +@SerializableAutoValue @PublicEvolving public abstract class BigQueryReadOptions implements Serializable { From 3f4d1be3c08dabcdf673ce33d250629b1bf522e4 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Wed, 1 Nov 2023 17:27:53 -0700 Subject: [PATCH 26/34] addressing comments from review --- .../cloud/flink/bigquery/services/BigQueryServicesImpl.java | 2 ++ .../cloud/flink/bigquery/table/BigQueryDynamicTableSource.java | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesImpl.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesImpl.java index ab0448f7..a3cc5b51 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesImpl.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesImpl.java @@ -234,6 +234,8 @@ public Optional> retrievePartitionColumnName QueryJobConfiguration queryConfig = QueryJobConfiguration.newBuilder(query).build(); + // TODO: change this method to use getTable method, see comment: + // https://github.com/GoogleCloudDataproc/flink-bigquery-connector/pull/46#discussion_r1371229725 TableResult results = bigQuery.query(queryConfig); return StreamSupport.stream(results.iterateAll().spliterator(), false) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java index 40096300..1073e458 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java @@ -227,7 +227,7 @@ public void applyPartitions(List> remainingPartitions) { + "=" + entry.getValue() + ")") - .collect(Collectors.joining(" AND "))) + .collect(Collectors.joining(" OR "))) .build(); } } From c8eb7890cab814713fdb175c6e618ba698bae4e5 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Thu, 2 Nov 2023 15:21:02 -0700 Subject: [PATCH 27/34] addressed comments from review --- .../common/utils/SchemaTransform.java | 97 +++++----- .../services/BigQueryServicesImpl.java | 7 +- .../table/BigQueryDynamicTableSource.java | 96 +++++----- .../restrictions/BigQueryRestriction.java | 176 ++++++++---------- 4 files changed, 184 insertions(+), 192 deletions(-) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java index 744e55f2..6eb7b86c 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/common/utils/SchemaTransform.java @@ -16,20 +16,20 @@ package com.google.cloud.flink.bigquery.common.utils; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableCollection; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMultimap; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.Field; import com.google.cloud.bigquery.FieldList; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -49,29 +49,33 @@ public class SchemaTransform { *

Some BigQuery types are duplicated here since slightly different Avro records are produced * when exporting data in Avro format and when reading data directly using the read API. */ - static final ImmutableMultimap BIG_QUERY_TO_AVRO_TYPES = - ImmutableMultimap.builder() - .put("STRING", Schema.Type.STRING) - .put("GEOGRAPHY", Schema.Type.STRING) - .put("BYTES", Schema.Type.BYTES) - .put("INTEGER", Schema.Type.LONG) - .put("INT64", Schema.Type.LONG) - .put("FLOAT", Schema.Type.DOUBLE) - .put("FLOAT64", Schema.Type.DOUBLE) - .put("NUMERIC", Schema.Type.BYTES) - .put("BIGNUMERIC", Schema.Type.BYTES) - .put("BOOLEAN", Schema.Type.BOOLEAN) - .put("BOOL", Schema.Type.BOOLEAN) - .put("TIMESTAMP", Schema.Type.LONG) - .put("RECORD", Schema.Type.RECORD) - .put("STRUCT", Schema.Type.RECORD) - .put("DATE", Schema.Type.STRING) - .put("DATE", Schema.Type.INT) - .put("DATETIME", Schema.Type.STRING) - .put("TIME", Schema.Type.STRING) - .put("TIME", Schema.Type.LONG) - .put("JSON", Schema.Type.STRING) - .build(); + static final Map> BIG_QUERY_TO_AVRO_TYPES = + initializeBigQueryToAvroTypesMapping(); + + private static Map> initializeBigQueryToAvroTypesMapping() { + Map> mapping = new HashMap<>(); + + mapping.put("STRING", Arrays.asList(Schema.Type.STRING)); + mapping.put("GEOGRAPHY", Arrays.asList(Schema.Type.STRING)); + mapping.put("BYTES", Arrays.asList(Schema.Type.BYTES)); + mapping.put("INTEGER", Arrays.asList(Schema.Type.LONG)); + mapping.put("INT64", Arrays.asList(Schema.Type.LONG)); + mapping.put("FLOAT", Arrays.asList(Schema.Type.DOUBLE)); + mapping.put("FLOAT64", Arrays.asList(Schema.Type.DOUBLE)); + mapping.put("NUMERIC", Arrays.asList(Schema.Type.BYTES)); + mapping.put("BIGNUMERIC", Arrays.asList(Schema.Type.BYTES)); + mapping.put("BOOLEAN", Arrays.asList(Schema.Type.BOOLEAN)); + mapping.put("BOOL", Arrays.asList(Schema.Type.BOOLEAN)); + mapping.put("TIMESTAMP", Arrays.asList(Schema.Type.LONG)); + mapping.put("RECORD", Arrays.asList(Schema.Type.RECORD)); + mapping.put("STRUCT", Arrays.asList(Schema.Type.RECORD)); + mapping.put("DATE", Arrays.asList(Schema.Type.STRING, Schema.Type.INT)); + mapping.put("DATETIME", Arrays.asList(Schema.Type.STRING)); + mapping.put("TIME", Arrays.asList(Schema.Type.STRING, Schema.Type.LONG)); + mapping.put("JSON", Arrays.asList(Schema.Type.STRING)); + + return mapping; + } public static Schema toGenericAvroSchema( String schemaName, List fieldSchemas, String namespace) { @@ -124,8 +128,7 @@ private static boolean hasNamespaceCollision(List fieldSchemas "nullness" // Avro library not annotated }) private static Schema.Field convertField(TableFieldSchema bigQueryField, String namespace) { - ImmutableCollection avroTypes = - BIG_QUERY_TO_AVRO_TYPES.get(bigQueryField.getType()); + List avroTypes = BIG_QUERY_TO_AVRO_TYPES.get(bigQueryField.getType()); if (avroTypes.isEmpty()) { throw new IllegalArgumentException( "Unable to map BigQuery field type " @@ -197,29 +200,21 @@ static List fieldListToListOfTableFieldSchema(FieldList fieldL .map( fList -> fList.stream() - .map( - field -> - new TableFieldSchema() - .setName(field.getName()) - .setDescription( - field.getDescription()) - .setDefaultValueExpression( - field - .getDefaultValueExpression()) - .setCollation(field.getCollation()) - .setMode( - Optional.ofNullable( - field - .getMode()) - .map(m -> m.name()) - .orElse(null)) - .setType(field.getType().name()) - .setFields( - fieldListToListOfTableFieldSchema( - field - .getSubFields()))) + .map(field -> fieldToTableFieldSchema(field)) .collect(Collectors.toList())) - .orElse(Lists.newArrayList()); + .orElse(new ArrayList<>()); + } + + static TableFieldSchema fieldToTableFieldSchema(Field field) { + + return new TableFieldSchema() + .setName(field.getName()) + .setDescription(field.getDescription()) + .setDefaultValueExpression(field.getDefaultValueExpression()) + .setCollation(field.getCollation()) + .setMode(Optional.ofNullable(field.getMode()).map(m -> m.name()).orElse(null)) + .setType(field.getType().name()) + .setFields(fieldListToListOfTableFieldSchema(field.getSubFields())); } /** diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesImpl.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesImpl.java index a3cc5b51..2f542d34 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesImpl.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesImpl.java @@ -20,8 +20,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - import com.google.api.gax.core.FixedCredentialsProvider; import com.google.api.gax.rpc.FixedHeaderProvider; import com.google.api.gax.rpc.HeaderProvider; @@ -53,6 +51,7 @@ import org.threeten.bp.Duration; import java.io.IOException; +import java.util.Arrays; import java.util.Iterator; import java.util.List; import java.util.Optional; @@ -179,7 +178,7 @@ public QueryDataClientImpl(CredentialsOptions options) { public List retrieveTablePartitions(String project, String dataset, String table) { try { String query = - Lists.newArrayList( + Arrays.asList( "SELECT", " partition_id", "FROM", @@ -217,7 +216,7 @@ public Optional> retrievePartitionColumnName String project, String dataset, String table) { try { String query = - Lists.newArrayList( + Arrays.asList( "SELECT", " column_name, data_type", "FROM", diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java index 1073e458..953ff696 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java @@ -33,8 +33,7 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - +import com.google.cloud.bigquery.StandardSQLTypeName; import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; import com.google.cloud.flink.bigquery.services.BigQueryServices; import com.google.cloud.flink.bigquery.services.BigQueryServicesFactory; @@ -44,6 +43,7 @@ import com.google.cloud.flink.bigquery.table.restrictions.BigQueryPartition; import com.google.cloud.flink.bigquery.table.restrictions.BigQueryRestriction; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -126,9 +126,10 @@ public Result applyFilters(List filters) { Map>> translatedFilters = filters.stream() .map( - exp -> + expression -> Tuple2.>of( - exp, BigQueryRestriction.convert(exp))) + expression, + BigQueryRestriction.convert(expression))) .map( transExp -> Tuple3.of( @@ -139,12 +140,12 @@ public Result applyFilters(List filters) { Collectors.groupingBy( (Tuple3 t) -> t.f0)); String rowRestriction = - translatedFilters.getOrDefault(true, Lists.newArrayList()).stream() + translatedFilters.getOrDefault(true, new ArrayList<>()).stream() .map(t -> t.f1) .collect(Collectors.joining(" AND ")); this.readOptions = this.readOptions.toBuilder().setRowRestriction(rowRestriction).build(); return Result.of( - translatedFilters.getOrDefault(true, Lists.newArrayList()).stream() + translatedFilters.getOrDefault(true, new ArrayList<>()).stream() .map(t -> t.f2) .collect(Collectors.toList()), filters); @@ -184,27 +185,14 @@ public Optional>> listPartitions() { connectOptions.getDataset(), connectOptions.getTable()) .map( - tuple -> { - // we retrieve the existing partition ids and transform them into valid - // values given the column data type - return BigQueryPartition.partitionValuesFromIdAndDataType( - dataClient.retrieveTablePartitions( - connectOptions.getProjectId(), - connectOptions.getDataset(), - connectOptions.getTable()), - tuple.f1) - .stream() - // for each of those valid partition values we create an map - // with the column name and the value - .map( - pValue -> { - Map partitionColAndValue = - new HashMap<>(); - partitionColAndValue.put(tuple.f0, pValue); - return partitionColAndValue; - }) - .collect(Collectors.toList()); - }); + columnNameAndType -> + transformPartitionIds( + connectOptions.getProjectId(), + connectOptions.getDataset(), + connectOptions.getTable(), + columnNameAndType.f0, + columnNameAndType.f1, + dataClient)); } @Override @@ -213,21 +201,45 @@ public void applyPartitions(List> remainingPartitions) { this.readOptions .toBuilder() .setRowRestriction( - // lets set the row restriction concating previously set restriction - // (coming from the table definition) with the partition restriction - // sent by Flink planner. - this.readOptions.getRowRestriction() - + " AND " - + remainingPartitions.stream() - .flatMap(map -> map.entrySet().stream()) - .map( - entry -> - "(" - + entry.getKey() - + "=" - + entry.getValue() - + ")") - .collect(Collectors.joining(" OR "))) + rebuildRestrictionsApplyingPartitions( + this.readOptions.getRowRestriction(), remainingPartitions)) .build(); } + + private static List> transformPartitionIds( + String projectId, + String dataset, + String table, + String columnName, + StandardSQLTypeName dataType, + BigQueryServices.QueryDataClient dataClient) { + + // we retrieve the existing partition ids and transform them into valid + // values given the column data type + return BigQueryPartition.partitionValuesFromIdAndDataType( + dataClient.retrieveTablePartitions(projectId, dataset, table), dataType) + .stream() + // for each of those valid partition values we create an map + // with the column name and the value + .map( + pValue -> { + Map partitionColAndValue = new HashMap<>(); + partitionColAndValue.put(columnName, pValue); + return partitionColAndValue; + }) + .collect(Collectors.toList()); + } + + private static String rebuildRestrictionsApplyingPartitions( + String currentRestriction, List> remainingPartitions) { + // lets set the row restriction concating previously set restriction + // (coming from the table definition) with the partition restriction + // sent by Flink planner. + return currentRestriction + + " AND " + + remainingPartitions.stream() + .flatMap(map -> map.entrySet().stream()) + .map(entry -> String.format("(%s=%s)", entry.getKey(), entry.getValue())) + .collect(Collectors.joining(" OR ")); + } } diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryRestriction.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryRestriction.java index 49860a12..78fd6f7b 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryRestriction.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryRestriction.java @@ -25,21 +25,21 @@ import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.functions.FunctionDefinition; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap; +import com.google.cloud.flink.bigquery.common.exceptions.BigQueryConnectorException; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.function.BiFunction; import java.util.regex.Matcher; import java.util.regex.Pattern; /** - * Utility class to transform a flink {@link ResolvedExpression} in a string restriction that can be + * Utility class to transform a Flink {@link ResolvedExpression} in a string restriction that can be * send to BigQuery as a row restriction. Heavily based in the Iceberg expression translation * implementation. */ @@ -51,7 +51,7 @@ private BigQueryRestriction() {} private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%"); /** Represents the possible BQ expressions supported for the correspondent flink ones. */ - public enum Operation { + enum Operation { EQ, NOT_EQ, GT, @@ -66,30 +66,34 @@ public enum Operation { STARTS_WITH } - private static final Map FILTERS = - ImmutableMap.builder() - .put(BuiltInFunctionDefinitions.EQUALS, Operation.EQ) - .put(BuiltInFunctionDefinitions.NOT_EQUALS, Operation.NOT_EQ) - .put(BuiltInFunctionDefinitions.GREATER_THAN, Operation.GT) - .put(BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, Operation.GT_EQ) - .put(BuiltInFunctionDefinitions.LESS_THAN, Operation.LT) - .put(BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, Operation.LT_EQ) - .put(BuiltInFunctionDefinitions.IS_NULL, Operation.IS_NULL) - .put(BuiltInFunctionDefinitions.IS_NOT_NULL, Operation.NOT_NULL) - .put(BuiltInFunctionDefinitions.AND, Operation.AND) - .put(BuiltInFunctionDefinitions.OR, Operation.OR) - .put(BuiltInFunctionDefinitions.NOT, Operation.NOT) - .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH) - .build(); + private static final Map FILTERS = initializeOperationMapper(); + + private static Map initializeOperationMapper() { + Map mapping = new HashMap<>(); + + mapping.put(BuiltInFunctionDefinitions.EQUALS, Operation.EQ); + mapping.put(BuiltInFunctionDefinitions.NOT_EQUALS, Operation.NOT_EQ); + mapping.put(BuiltInFunctionDefinitions.GREATER_THAN, Operation.GT); + mapping.put(BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, Operation.GT_EQ); + mapping.put(BuiltInFunctionDefinitions.LESS_THAN, Operation.LT); + mapping.put(BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, Operation.LT_EQ); + mapping.put(BuiltInFunctionDefinitions.IS_NULL, Operation.IS_NULL); + mapping.put(BuiltInFunctionDefinitions.IS_NOT_NULL, Operation.NOT_NULL); + mapping.put(BuiltInFunctionDefinitions.AND, Operation.AND); + mapping.put(BuiltInFunctionDefinitions.OR, Operation.OR); + mapping.put(BuiltInFunctionDefinitions.NOT, Operation.NOT); + mapping.put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH); + return mapping; + } /** * Convert a flink expression into a BigQuery row restriction. * - *

the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the + *

the BETWEEN, NOT_BETWEEN, IN expression will be converted by Flink automatically. the * BETWEEN will be converted to (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ * OR GT_EQ), the IN will be converted to OR, so we do not add the conversion here * - * @param flinkExpression the flink expression + * @param flinkExpression the Flink expression * @return An {@link Optional} potentially containing the resolved row restriction for BigQuery. */ public static Optional convert(Expression flinkExpression) { @@ -99,67 +103,55 @@ public static Optional convert(Expression flinkExpression) { CallExpression call = (CallExpression) flinkExpression; Operation op = FILTERS.get(call.getFunctionDefinition()); - if (op != null) { - switch (op) { - case IS_NULL: - return onlyChildAs(call, FieldReferenceExpression.class) - .map(FieldReferenceExpression::getName) - .map(field -> field + " IS NULL"); - - case NOT_NULL: - return onlyChildAs(call, FieldReferenceExpression.class) - .map(FieldReferenceExpression::getName) - .map(field -> "NOT " + field + " IS NULL"); - - case LT: - return convertFieldAndLiteral( - (left, right) -> "(" + left + " < " + right + ")", - (left, right) -> "(" + left + " > " + right + ")", - call); - - case LT_EQ: - return convertFieldAndLiteral( - (left, right) -> "(" + left + " <= " + right + ")", - (left, right) -> "(" + left + " >= " + right + ")", - call); - - case GT: - return convertFieldAndLiteral( - (left, right) -> "(" + left + " > " + right + ")", - (left, right) -> "(" + left + " < " + right + ")", - call); - - case GT_EQ: - return convertFieldAndLiteral( - (left, right) -> "(" + left + " >= " + right + ")", - (left, right) -> "(" + left + " <= " + right + ")", - call); - - case EQ: - return convertFieldAndLiteral((ref, lit) -> ref + " = " + lit, call); - - case NOT_EQ: - return convertFieldAndLiteral((ref, lit) -> ref + " <> " + lit, call); - - case NOT: - return onlyChildAs(call, CallExpression.class) - .flatMap(BigQueryRestriction::convert) - .map(field -> "NOT " + field); - - case AND: - return convertLogicExpression( - (left, right) -> "(" + left + " AND " + right + ")", call); - - case OR: - return convertLogicExpression( - (left, right) -> "(" + left + " OR " + right + ")", call); - - case STARTS_WITH: - return convertLike(call); - } - } + switch (op) { + case IS_NULL: + return onlyChildAs(call, FieldReferenceExpression.class) + .map(FieldReferenceExpression::getName) + .map(field -> field + " IS NULL"); - return Optional.empty(); + case NOT_NULL: + return onlyChildAs(call, FieldReferenceExpression.class) + .map(FieldReferenceExpression::getName) + .map(field -> "NOT " + field + " IS NULL"); + + case LT: + return convertOperationPartsWithItsSymbol("<", call); + + case LT_EQ: + return convertOperationPartsWithItsSymbol("<=", call); + + case GT: + return convertOperationPartsWithItsSymbol(">", call); + + case GT_EQ: + return convertOperationPartsWithItsSymbol(">=", call); + + case EQ: + return convertOperationPartsWithItsSymbol("=", call); + + case NOT_EQ: + return convertOperationPartsWithItsSymbol("<>", call); + + case NOT: + return onlyChildAs(call, CallExpression.class) + .flatMap(BigQueryRestriction::convert) + .map(field -> "NOT " + field); + + case AND: + return convertLogicExpressionWithOperandsSymbol("AND", call); + + case OR: + return convertLogicExpressionWithOperandsSymbol("OR", call); + + case STARTS_WITH: + return convertLike(call); + + default: + throw new BigQueryConnectorException( + String.format( + "The provided Flink expression is not supported %s.", + call.getFunctionName())); + } } private static Optional onlyChildAs( @@ -209,8 +201,8 @@ private static Optional convertLike(CallExpression call) { return Optional.empty(); } - private static Optional convertLogicExpression( - BiFunction function, CallExpression call) { + private static Optional convertLogicExpressionWithOperandsSymbol( + String operandsSymbol, CallExpression call) { List args = call.getResolvedChildren(); if (args == null || args.size() != 2) { return Optional.empty(); @@ -219,7 +211,8 @@ private static Optional convertLogicExpression( Optional left = convert(args.get(0)); Optional right = convert(args.get(1)); if (left.isPresent() && right.isPresent()) { - return Optional.of(function.apply(left.get(), right.get())); + return Optional.of( + String.format("(%s %s %s)", left.get(), operandsSymbol, right.get())); } return Optional.empty(); @@ -251,15 +244,8 @@ private static Optional convertLiteral(ValueLiteralExpression expression }); } - private static Optional convertFieldAndLiteral( - BiFunction expr, CallExpression call) { - return convertFieldAndLiteral(expr, expr, call); - } - - private static Optional convertFieldAndLiteral( - BiFunction convertLR, - BiFunction convertRL, - CallExpression call) { + private static Optional convertOperationPartsWithItsSymbol( + String operationSymbol, CallExpression call) { List args = call.getResolvedChildren(); if (args.size() != 2) { return Optional.empty(); @@ -272,14 +258,14 @@ private static Optional convertFieldAndLiteral( String name = ((FieldReferenceExpression) left).getName(); Optional lit = convertLiteral((ValueLiteralExpression) right); if (lit.isPresent()) { - return Optional.of(convertLR.apply(name, lit.get())); + return Optional.of(String.format("(%s %s %s)", name, operationSymbol, lit.get())); } } else if (left instanceof ValueLiteralExpression && right instanceof FieldReferenceExpression) { Optional lit = convertLiteral((ValueLiteralExpression) left); String name = ((FieldReferenceExpression) right).getName(); if (lit.isPresent()) { - return Optional.of(convertRL.apply(name, lit.get())); + return Optional.of(String.format("(%s %s %s)", lit.get(), operationSymbol, name)); } } From cce65673e5f5a5587a2aae188efa58a1647ae851 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Fri, 3 Nov 2023 11:24:39 -0700 Subject: [PATCH 28/34] partition list in 1024 chunks --- .../google/cloud/flink/bigquery/fakes/StorageClientFaker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java index ae3b906c..f06ac18a 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java @@ -286,7 +286,7 @@ public static List createResponse( .collect( () -> new HashMap>(), (map, idx) -> - map.computeIfAbsent(idx, key -> new ArrayList<>()) + map.computeIfAbsent(idx / 1024, key -> new ArrayList<>()) .add(genericRecords.get(idx)), (map1, map2) -> map2.entrySet() From 52600730fb2862e57f342144ef5a09d07fd45b42 Mon Sep 17 00:00:00 2001 From: Jayant Jain Date: Thu, 23 Nov 2023 21:16:59 +0530 Subject: [PATCH 29/34] Address review comments --- .../table/BigQueryDynamicTableSource.java | 6 ++--- .../restrictions/BigQueryRestriction.java | 22 ++++++++++--------- 2 files changed, 15 insertions(+), 13 deletions(-) diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java index 953ff696..c2f535bf 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSource.java @@ -235,11 +235,11 @@ private static String rebuildRestrictionsApplyingPartitions( // lets set the row restriction concating previously set restriction // (coming from the table definition) with the partition restriction // sent by Flink planner. - return currentRestriction - + " AND " - + remainingPartitions.stream() + String partitionRestrictions = + remainingPartitions.stream() .flatMap(map -> map.entrySet().stream()) .map(entry -> String.format("(%s=%s)", entry.getKey(), entry.getValue())) .collect(Collectors.joining(" OR ")); + return currentRestriction + " AND (" + partitionRestrictions + ")"; } } diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryRestriction.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryRestriction.java index 78fd6f7b..af7c9072 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryRestriction.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryRestriction.java @@ -254,19 +254,21 @@ private static Optional convertOperationPartsWithItsSymbol( Expression left = args.get(0); Expression right = args.get(1); + Optional leftOption = Optional.empty(); + Optional rightOption = Optional.empty(); if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { - String name = ((FieldReferenceExpression) left).getName(); - Optional lit = convertLiteral((ValueLiteralExpression) right); - if (lit.isPresent()) { - return Optional.of(String.format("(%s %s %s)", name, operationSymbol, lit.get())); - } + leftOption = Optional.of(((FieldReferenceExpression) left).getName()); + rightOption = convertLiteral((ValueLiteralExpression) right); } else if (left instanceof ValueLiteralExpression && right instanceof FieldReferenceExpression) { - Optional lit = convertLiteral((ValueLiteralExpression) left); - String name = ((FieldReferenceExpression) right).getName(); - if (lit.isPresent()) { - return Optional.of(String.format("(%s %s %s)", lit.get(), operationSymbol, name)); - } + leftOption = convertLiteral((ValueLiteralExpression) left); + rightOption = Optional.of(((FieldReferenceExpression) right).getName()); + } + + if (leftOption.isPresent() && rightOption.isPresent()) { + return Optional.of( + String.format( + "(%s %s %s)", leftOption.get(), operationSymbol, rightOption.get())); } return Optional.empty(); From 936076b4bb28ec31d65fcfa1e81a5d4520fd2e86 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Thu, 29 Jun 2023 11:15:26 -0700 Subject: [PATCH 30/34] adding the example and shaded distro jar, fixes a NPE when the provided table is expired and no longer accessible. --- flink-connector-bigquery-examples/pom.xml | 131 ++++++++++++++++ .../gcp/bigquery/BigQueryExample.java | 140 ++++++++++++++++++ .../services/BigQueryServicesImpl.java | 19 ++- flink-sql-connector-bigquery/pom.xml | 80 ++++++++++ .../src/main/resources/META-INF/NOTICE | 5 + pom.xml | 2 + 6 files changed, 371 insertions(+), 6 deletions(-) create mode 100644 flink-connector-bigquery-examples/pom.xml create mode 100644 flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java create mode 100644 flink-sql-connector-bigquery/pom.xml create mode 100644 flink-sql-connector-bigquery/src/main/resources/META-INF/NOTICE diff --git a/flink-connector-bigquery-examples/pom.xml b/flink-connector-bigquery-examples/pom.xml new file mode 100644 index 00000000..cc183d30 --- /dev/null +++ b/flink-connector-bigquery-examples/pom.xml @@ -0,0 +1,131 @@ + + + + 4.0.0 + + + org.apache.flink + flink-connector-bigquery-parent + 1.1-SNAPSHOT + + + flink-connector-bigquery-examples + Flink : Connectors : Google BigQuery Examples + jar + + + true + + + + + org.apache.flink + flink-connector-bigquery + ${project.version} + + + org.apache.flink + flink-streaming-java + ${flink.version} + + + + + BigQueryExample + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + false + + + src/test/resources + + + + + org.apache.flink.examples.gcp.bigquery.BigQueryExample + + + log4j2-bqexample.properties + src/main/resources/log4j2-bqexample.properties + + + + + org.apache.flink:flink-connector-bigquery + org.apache.flink:flink-avro + org.apache.flink:flink-metrics-dropwizard + com.google.cloud:google-cloud-bigquerystorage + com.google.*:* + commons-codec:commons-codec + dev.failsafe:* + org.apache.avro:* + org.apache.httpcomponents:* + org.codehaus.mojo:animal-sniffer-annotations + org.conscrypt:* + com.fasterxml.jackson.*:* + org.threeten:* + org.checkerframework:* + io.dropwizard.metrics:* + io.grpc:* + io.opencensus:* + io.perfmark:* + + + + + com.google + org.apache.flink.examples.gcp.bigquery.shaded.com.google + + + + + + org.apache.flink:flink-connector-bigquery-examples* + + org/apache/flink/examples/gcp/bigquery/** + + + + + + + + + + \ No newline at end of file diff --git a/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java b/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java new file mode 100644 index 00000000..37874080 --- /dev/null +++ b/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.examples.gcp.bigquery; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Collector; + +import com.google.cloud.flink.bigquery.common.config.BigQueryConnectOptions; +import com.google.cloud.flink.bigquery.source.BigQuerySource; +import com.google.cloud.flink.bigquery.source.config.BigQueryReadOptions; +import org.apache.avro.generic.GenericRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A simple BigQuery table read example with Flink's DataStream API. + * + *

The Flink pipeline will try to read the specified BigQuery table, potentially limiting the + * element count to the specified row restriction and limit count, returning {@link GenericRecord} + * representing the rows, to finally prints out some aggregated values given the provided payload's + * field. + * + *

Note on row restriction: In case of including a restriction with a temporal reference, + * something like {@code "TIMESTAMP_TRUNC(ingestion_timestamp, HOUR) = '2023-06-20 19:00:00'"}, and + * launching the job from Flink's Rest API is known the single quotes are not supported and will + * make the pipeline fail. As a workaround for that case using \u0027 as a replacement will make it + * work, example {@code "TIMESTAMP_TRUNC(ingestion_timestamp, HOUR) = \u00272023-06-20 + * 19:00:00\u0027"}. + */ +public class BigQueryExample { + + private static final Logger LOG = LoggerFactory.getLogger(BigQueryExample.class); + + public static void main(String[] args) throws Exception { + // parse input arguments + final ParameterTool parameterTool = ParameterTool.fromArgs(args); + + if (parameterTool.getNumberOfParameters() < 4) { + LOG.error( + "Missing parameters!\n" + + "Usage: flink run BigQuery.jar" + + " --gcp-project --bq-dataset " + + " --bq-table

--agg-prop " + + " --restriction " + + " --limit "); + return; + } + + String projectName = parameterTool.getRequired("gcp-project"); + String datasetName = parameterTool.getRequired("bq-dataset"); + String tableName = parameterTool.getRequired("bq-table"); + String rowRestriction = parameterTool.get("restriction", "").replace("\\u0027", "'"); + Integer recordLimit = parameterTool.getInt("limit", -1); + String recordPropertyToAggregate = parameterTool.getRequired("agg-prop"); + + runFlinkJob( + projectName, + datasetName, + tableName, + recordPropertyToAggregate, + rowRestriction, + recordLimit); + } + + private static void runFlinkJob( + String projectName, + String datasetName, + String tableName, + String recordPropertyToAggregate, + String rowRestriction, + Integer limit) + throws Exception { + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(60000L); + + /** + * we will be reading avro generic records from BigQuery, and in this case we are assuming + * the GOOGLE_APPLICATION_CREDENTIALS env variable will be present in the execution runtime. + * In case of needing authenticate differently, the credentials builder (part of the + * BigQueryConnectOptions) should enable capturing the credentials from various sources. + */ + BigQuerySource bqSource = + BigQuerySource.readAvros( + BigQueryReadOptions.builder() + .setBigQueryConnectOptions( + BigQueryConnectOptions.builder() + .setProjectId(projectName) + .setDataset(datasetName) + .setTable(tableName) + .build()) + .setRowRestriction(rowRestriction) + .build(), + limit); + + env.fromSource(bqSource, WatermarkStrategy.noWatermarks(), "BigQuerySource") + .flatMap(new FlatMapper(recordPropertyToAggregate)) + .keyBy(t -> t.f0) + .max("f1") + .print(); + + env.execute("Flink BigQuery Example"); + } + + static class FlatMapper implements FlatMapFunction> { + + private final String recordPropertyToAggregate; + + public FlatMapper(String recordPropertyToAggregate) { + this.recordPropertyToAggregate = recordPropertyToAggregate; + } + + @Override + public void flatMap(GenericRecord record, Collector> out) + throws Exception { + out.collect( + Tuple2.of( + (String) record.get(recordPropertyToAggregate).toString(), 1)); + } + } +} diff --git a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesImpl.java b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesImpl.java index 2f542d34..4b33da2f 100644 --- a/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesImpl.java +++ b/flink-connector-bigquery/src/main/java/com/google/cloud/flink/bigquery/services/BigQueryServicesImpl.java @@ -205,7 +205,8 @@ public List retrieveTablePartitions(String project, String dataset, Stri } catch (Exception ex) { throw new RuntimeException( String.format( - "Problems while trying to retrieve table partitions (table: %s.%s.%s).", + "Problems while trying to retrieve table partitions" + + " (table: %s.%s.%s).", project, dataset, table), ex); } @@ -252,7 +253,8 @@ public Optional> retrievePartitionColumnName } catch (Exception ex) { throw new RuntimeException( String.format( - "Problems while trying to retrieve table partition's column name (table: %s.%s.%s).", + "Problems while trying to retrieve table partition's" + + " column name (table: %s.%s.%s).", project, dataset, table), ex); } @@ -260,10 +262,15 @@ public Optional> retrievePartitionColumnName @Override public TableSchema getTableSchema(String project, String dataset, String table) { - return SchemaTransform.bigQuerySchemaToTableSchema( - bigQuery.getTable(TableId.of(project, dataset, table)) - .getDefinition() - .getSchema()); + return Optional.ofNullable(bigQuery.getTable(TableId.of(project, dataset, table))) + .map(t -> t.getDefinition().getSchema()) + .map(schema -> SchemaTransform.bigQuerySchemaToTableSchema(schema)) + .orElseThrow( + () -> + new IllegalArgumentException( + String.format( + "The provided table %s.%s.%s does not exists.", + project, dataset, table))); } @Override diff --git a/flink-sql-connector-bigquery/pom.xml b/flink-sql-connector-bigquery/pom.xml new file mode 100644 index 00000000..48f8af4f --- /dev/null +++ b/flink-sql-connector-bigquery/pom.xml @@ -0,0 +1,80 @@ + + + + + 4.0.0 + + + flink-connector-bigquery-parent + org.apache.flink + 1.1-SNAPSHOT + + + flink-sql-connector-bigquery + Flink : Connectors : SQL : Google BigQuery + + jar + + + true + + + + + org.apache.flink + flink-connector-bigquery + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + false + + + *:* + + + + + com.google + org.apache.flink.connector.bigquery.shaded.com.google + + + + + + + + + + diff --git a/flink-sql-connector-bigquery/src/main/resources/META-INF/NOTICE b/flink-sql-connector-bigquery/src/main/resources/META-INF/NOTICE new file mode 100644 index 00000000..27b8ca73 --- /dev/null +++ b/flink-sql-connector-bigquery/src/main/resources/META-INF/NOTICE @@ -0,0 +1,5 @@ +flink-sql-connector-bigquery +Copyright 2023 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). diff --git a/pom.xml b/pom.xml index e6018eca..ae25d4cc 100644 --- a/pom.xml +++ b/pom.xml @@ -66,6 +66,8 @@ under the License. flink-connector-bigquery + flink-sql-connector-bigquery + flink-connector-bigquery-examples From 1460af6ce07f9cab82f57fb55b91a0e031dce154 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Thu, 29 Jun 2023 22:15:41 -0700 Subject: [PATCH 31/34] added covertura analysis through open clover and improved few tests --- README.md | 2 +- cloudbuild/cloudbuild.yaml | 16 +- cloudbuild/presubmit.sh | 13 +- flink-connector-bigquery/pom.xml | 48 ----- .../bigquery/fakes/StorageClientFaker.java | 17 +- .../split/BigQuerySourceSplitStateTest.java | 2 + .../BigQueryDynamicTableSourceITCase.java | 7 +- .../restrictions/BigQueryPartitionTest.java | 176 ++++++++++++++++++ pom.xml | 49 ++++- tools/maven/clover.xml | 45 +++++ 10 files changed, 292 insertions(+), 83 deletions(-) create mode 100644 flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryPartitionTest.java create mode 100644 tools/maven/clover.xml diff --git a/README.md b/README.md index 59136720..6cbbeaa5 100644 --- a/README.md +++ b/README.md @@ -22,7 +22,7 @@ Prerequisites: ``` git clone https://github.com/GoogleCloudDataproc/flink-bigquery-connector -cd flink-connector-bigquery +cd flink-bigquery-connector mvn clean package -DskipTests ``` diff --git a/cloudbuild/cloudbuild.yaml b/cloudbuild/cloudbuild.yaml index 63db5a5e..d62996a8 100644 --- a/cloudbuild/cloudbuild.yaml +++ b/cloudbuild/cloudbuild.yaml @@ -13,27 +13,15 @@ steps: env: - 'CODECOV_TOKEN=${_CODECOV_TOKEN}' -# 3. Run unit tests +# 3. Run unit & integration tests - name: 'gcr.io/$PROJECT_ID/dataproc-flink-bigquery-connector-presubmit' id: 'unit-tests' waitFor: ['init'] entrypoint: 'bash' - args: ['/workspace/cloudbuild/presubmit.sh', 'unittest'] + args: ['/workspace/cloudbuild/presubmit.sh', 'tests'] env: - 'CODECOV_TOKEN=${_CODECOV_TOKEN}' -# 4. Run integration tests concurrently with unit tests -# Commeneted out until integration tests are ported -# - name: 'gcr.io/$PROJECT_ID/dataproc-flink-bigquery-connector-presubmit' -# id: 'integration-tests' -# waitFor: ['unit-tests'] -# entrypoint: 'bash' -# args: ['/workspace/cloudbuild/presubmit.sh', 'integrationtest'] -# env: -# - 'GOOGLE_CLOUD_PROJECT=${_GOOGLE_CLOUD_PROJECT}' -# - 'TEMPORARY_GCS_BUCKET=${_TEMPORARY_GCS_BUCKET}' -# - 'CODECOV_TOKEN=${_CODECOV_TOKEN}' - # Tests take around 20 mins in general. timeout: 1800s diff --git a/cloudbuild/presubmit.sh b/cloudbuild/presubmit.sh index 8ed18096..e651b3ee 100644 --- a/cloudbuild/presubmit.sh +++ b/cloudbuild/presubmit.sh @@ -30,18 +30,13 @@ cd /workspace case $STEP in # Download maven and all the dependencies init) - $MVN install -DskipTests + $MVN clean install -DskipTests exit ;; - # Run unit tests - unittest) - $MVN test jacoco:report jacoco:report-aggregate - ;; - - # Run integration tests - integrationtest) - $MVN failsafe:integration-test failsafe:verify jacoco:report jacoco:report-aggregate + # Run unit & integration tests + tests) + $MVN mvn clean clover:setup verify clover:aggregate clover:clover -Pclover -pl flink-connector-bigquery ;; *) diff --git a/flink-connector-bigquery/pom.xml b/flink-connector-bigquery/pom.xml index c75d8643..cfcd2d82 100644 --- a/flink-connector-bigquery/pom.xml +++ b/flink-connector-bigquery/pom.xml @@ -162,58 +162,10 @@ under the License. org.apache.maven.plugins maven-jar-plugin - - - - test-jar - - - org.apache.maven.plugins maven-surefire-plugin - - - default-test - test - - test - - - ${argLine} -XX:+UseG1GC -Xms256m -Xmx1024m - - - - integration-tests - integration-test - - test - - - -XX:+UseG1GC -Xms256m -Xmx2048m - - - - - - org.jacoco - jacoco-maven-plugin - - - prepare-agent - - prepare-agent - - - - report - install - - report - - - diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java index f06ac18a..af722ced 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/fakes/StorageClientFaker.java @@ -51,6 +51,8 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.Serializable; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -82,16 +84,18 @@ public StorageReadClient getStorageClient(CredentialsOptions readOptions) @Override public QueryDataClient getQueryDataClient(CredentialsOptions readOptions) { return new QueryDataClient() { + @Override public List retrieveTablePartitions( String project, String dataset, String table) { - return Arrays.asList("2023062811"); + return Arrays.asList( + LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyyMMddHH"))); } @Override public Optional> retrievePartitionColumnName( String project, String dataset, String table) { - return Optional.of(Tuple2.of("number", StandardSQLTypeName.INT64)); + return Optional.of(Tuple2.of("ts", StandardSQLTypeName.TIMESTAMP)); } @Override @@ -191,8 +195,9 @@ public void close() {} public static final String SIMPLE_AVRO_SCHEMA_FIELDS_STRING = " \"fields\": [\n" - + " {\"name\": \"name\", \"type\": \"string\"},\n" - + " {\"name\": \"number\", \"type\": \"long\"}\n" + + " {\"name\": \"name\", \"type\": \"string\"},\n" + + " {\"name\": \"number\", \"type\": \"long\"},\n" + + " {\"name\" : \"ts\", \"type\" : {\"type\" : \"long\",\"logicalType\" : \"timestamp-micros\"}}\n" + " ]\n"; public static final String SIMPLE_AVRO_SCHEMA_STRING = "{\"namespace\": \"project.dataset\",\n" @@ -225,6 +230,10 @@ public void close() {} new TableFieldSchema() .setName("number") .setType("INTEGER") + .setMode("REQUIRED"), + new TableFieldSchema() + .setName("ts") + .setType("TIMESTAMP") .setMode("REQUIRED"))); /** Represents the parameters needed for the Avro data generation. */ diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java index e71a4b82..74ca783b 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/source/split/BigQuerySourceSplitStateTest.java @@ -32,6 +32,8 @@ public void testSplitStateTransformation() { BigQuerySourceSplitState splitState = new BigQuerySourceSplitState(originalSplit); assertThat(splitState.toBigQuerySourceSplit()).isEqualTo(originalSplit); + assertThat(splitState) + .isEqualTo(new BigQuerySourceSplitState(splitState.toBigQuerySourceSplit())); } @Test diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSourceITCase.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSourceITCase.java index 69c1ad38..f3832f39 100644 --- a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSourceITCase.java +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/BigQueryDynamicTableSourceITCase.java @@ -211,7 +211,12 @@ public void testProject() { @Test public void testRestriction() { - String sqlFilter = "id = 0 AND NOT optString IS NULL"; + String sqlFilter = + "id = 0" + + " AND NOT optString IS NULL" + + " AND optString LIKE 's%'" + + " AND optDouble > -1" + + " AND optDouble <= 1.0 "; tEnv.executeSql(createTestDDl(null)); Iterator collected = diff --git a/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryPartitionTest.java b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryPartitionTest.java new file mode 100644 index 00000000..23697c02 --- /dev/null +++ b/flink-connector-bigquery/src/test/java/com/google/cloud/flink/bigquery/table/restrictions/BigQueryPartitionTest.java @@ -0,0 +1,176 @@ +/* + * Copyright (C) 2023 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.table.restrictions; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import com.google.cloud.bigquery.StandardSQLTypeName; +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import java.util.List; + +/** */ +public class BigQueryPartitionTest { + + @Test + public void testPartitionHour() { + List partitionIds = Lists.newArrayList("2023062822", "2023062823"); + // ISO formatted dates as single quote string literals at the beginning of the hour. + List expectedValues = + Lists.newArrayList("'2023-06-28 22:00:00'", "'2023-06-28 23:00:00'"); + List values = + BigQueryPartition.partitionValuesFromIdAndDataType( + partitionIds, StandardSQLTypeName.TIMESTAMP); + + Assertions.assertThat(values).isEqualTo(expectedValues); + } + + @Test + public void testPartitionDay() { + List partitionIds = Lists.newArrayList("20230628", "20230628"); + // ISO formatted dates as single quote string literals. + List expectedValues = Lists.newArrayList("'2023-06-28'", "'2023-06-28'"); + List values = + BigQueryPartition.partitionValuesFromIdAndDataType( + partitionIds, StandardSQLTypeName.DATETIME); + + Assertions.assertThat(values).isEqualTo(expectedValues); + } + + @Test + public void testPartitionMonth() { + List partitionIds = Lists.newArrayList("202306", "202307"); + // ISO formatted dates as single quote string literals + List expectedValues = Lists.newArrayList("'2023-06'", "'2023-07'"); + List values = + BigQueryPartition.partitionValuesFromIdAndDataType( + partitionIds, StandardSQLTypeName.DATE); + + Assertions.assertThat(values).isEqualTo(expectedValues); + } + + @Test + public void testPartitionYear() { + List partitionIds = Lists.newArrayList("2023", "2022"); + // ISO formatted dates as single quote string literals + List expectedValues = Lists.newArrayList("'2023'", "'2022'"); + List values = + BigQueryPartition.partitionValuesFromIdAndDataType( + partitionIds, StandardSQLTypeName.TIMESTAMP); + + Assertions.assertThat(values).isEqualTo(expectedValues); + } + + @Test + public void testPartitionInteger() { + List partitionIds = Lists.newArrayList("2023", "2022"); + // ISO formatted dates as single quote string literals + List expectedValues = Lists.newArrayList("2023", "2022"); + List values = + BigQueryPartition.partitionValuesFromIdAndDataType( + partitionIds, StandardSQLTypeName.INT64); + + Assertions.assertThat(values).isEqualTo(expectedValues); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongTemporalPartition() { + List partitionIds = Lists.newArrayList("202308101112"); + BigQueryPartition.partitionValuesFromIdAndDataType( + partitionIds, StandardSQLTypeName.TIMESTAMP); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongArrayPartition() { + List partitionIds = Lists.newArrayList("2023", "2022"); + BigQueryPartition.partitionValuesFromIdAndDataType(partitionIds, StandardSQLTypeName.ARRAY); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongStructPartition() { + List partitionIds = Lists.newArrayList("2023", "2022"); + BigQueryPartition.partitionValuesFromIdAndDataType( + partitionIds, StandardSQLTypeName.STRUCT); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongJsonPartition() { + List partitionIds = Lists.newArrayList("2023", "2022"); + BigQueryPartition.partitionValuesFromIdAndDataType(partitionIds, StandardSQLTypeName.JSON); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongGeoPartition() { + List partitionIds = Lists.newArrayList("2023", "2022"); + BigQueryPartition.partitionValuesFromIdAndDataType( + partitionIds, StandardSQLTypeName.GEOGRAPHY); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongBigNumPartition() { + List partitionIds = Lists.newArrayList("2023", "2022"); + BigQueryPartition.partitionValuesFromIdAndDataType( + partitionIds, StandardSQLTypeName.BIGNUMERIC); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongBoolPartition() { + List partitionIds = Lists.newArrayList("2023", "2022"); + BigQueryPartition.partitionValuesFromIdAndDataType(partitionIds, StandardSQLTypeName.BOOL); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongBytesPartition() { + List partitionIds = Lists.newArrayList("2023", "2022"); + BigQueryPartition.partitionValuesFromIdAndDataType(partitionIds, StandardSQLTypeName.BYTES); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongFloatPartition() { + List partitionIds = Lists.newArrayList("2023", "2022"); + BigQueryPartition.partitionValuesFromIdAndDataType( + partitionIds, StandardSQLTypeName.FLOAT64); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongStringPartition() { + List partitionIds = Lists.newArrayList("2023", "2022"); + BigQueryPartition.partitionValuesFromIdAndDataType( + partitionIds, StandardSQLTypeName.STRING); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongTimePartition() { + List partitionIds = Lists.newArrayList("2023", "2022"); + BigQueryPartition.partitionValuesFromIdAndDataType(partitionIds, StandardSQLTypeName.TIME); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongIntervalPartition() { + List partitionIds = Lists.newArrayList("2023", "2022"); + BigQueryPartition.partitionValuesFromIdAndDataType( + partitionIds, StandardSQLTypeName.INTERVAL); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongNumeriPartition() { + List partitionIds = Lists.newArrayList("2023", "2022"); + BigQueryPartition.partitionValuesFromIdAndDataType( + partitionIds, StandardSQLTypeName.NUMERIC); + } +} diff --git a/pom.xml b/pom.xml index ae25d4cc..01840830 100644 --- a/pom.xml +++ b/pom.xml @@ -56,12 +56,13 @@ under the License. false 3.0.0-1.16 - 0.8.10 + 4.4.1 1.7.36 2.17.2 flink-connector-bigquery-parent + target/test-report @@ -402,11 +403,47 @@ under the License. org.commonjava.maven.plugins directory-maven-plugin - - org.jacoco - jacoco-maven-plugin - ${jacoco.version} - + + + clover + + + + org.openclover + clover-maven-plugin + ${clover.version} + + + ${maven.multiModuleProjectDirectory}/tools/maven/clover.xml + 55% + + **/com/google/cloud/flink/bigquery/common/config/* + **/com/google/cloud/flink/bigquery/common/utils/GoogleCredentialsSupplier.* + **/com/google/cloud/flink/bigquery/services/**Impl.* + **/com/google/cloud/flink/bigquery/source/config/* + + + + + org.apache.maven.plugins + maven-surefire-plugin + + ${surefire.and.failsafe.report.dir} + + + + + + + + org.openclover + clover-maven-plugin + ${clover.version} + + + + + diff --git a/tools/maven/clover.xml b/tools/maven/clover.xml new file mode 100644 index 00000000..8149f176 --- /dev/null +++ b/tools/maven/clover.xml @@ -0,0 +1,45 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file From a8499c3925d85d0ba56dcc3f10546e7c3da7ae43 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 11 Jul 2023 16:17:01 -0700 Subject: [PATCH 32/34] removed repeated mvn reference from command --- cloudbuild/presubmit.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cloudbuild/presubmit.sh b/cloudbuild/presubmit.sh index e651b3ee..71e7162d 100644 --- a/cloudbuild/presubmit.sh +++ b/cloudbuild/presubmit.sh @@ -36,7 +36,7 @@ case $STEP in # Run unit & integration tests tests) - $MVN mvn clean clover:setup verify clover:aggregate clover:clover -Pclover -pl flink-connector-bigquery + $MVN clean clover:setup verify clover:aggregate clover:clover -Pclover -pl flink-connector-bigquery ;; *) From c9f73ee25423b272ad7cd1068c61cbc907680adf Mon Sep 17 00:00:00 2001 From: Jayant Jain Date: Mon, 27 Nov 2023 14:04:07 +0530 Subject: [PATCH 33/34] Address review comments --- .../gcp/bigquery/BigQueryExample.java | 36 +++++++++++-------- 1 file changed, 22 insertions(+), 14 deletions(-) diff --git a/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java b/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java index 37874080..93a27888 100644 --- a/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java +++ b/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java @@ -32,19 +32,27 @@ import org.slf4j.LoggerFactory; /** - * A simple BigQuery table read example with Flink's DataStream API. + * A simple Flink application using DataStream API and BigQuery connector. * - *

The Flink pipeline will try to read the specified BigQuery table, potentially limiting the - * element count to the specified row restriction and limit count, returning {@link GenericRecord} - * representing the rows, to finally prints out some aggregated values given the provided payload's - * field. + *

The Flink pipeline will try to read the specified BigQuery table, limiting the element count + * to the specified row restriction and limit, returning {@link GenericRecord} representing the + * rows, and finally print out some aggregated values given the provided payload's field. The + * sequence of operations in this pipeline is: source > flatMap > keyBy > max > print. * - *

Note on row restriction: In case of including a restriction with a temporal reference, - * something like {@code "TIMESTAMP_TRUNC(ingestion_timestamp, HOUR) = '2023-06-20 19:00:00'"}, and - * launching the job from Flink's Rest API is known the single quotes are not supported and will - * make the pipeline fail. As a workaround for that case using \u0027 as a replacement will make it - * work, example {@code "TIMESTAMP_TRUNC(ingestion_timestamp, HOUR) = \u00272023-06-20 - * 19:00:00\u0027"}. + *

Flink command line format to execute this application:
+ * flink run {additional runtime params} {path to this jar}/BigQueryExample.jar
+ * --gcp-project {required; project ID which contains the BigQuery table}
+ * --bq-dataset {required; name of BigQuery dataset containing the desired table}
+ * --bq-table {required; name of BigQuery table to read}
+ * --agg-prop {required; record property to aggregate in Flink job}
+ * --restriction {optional; SQL-like filter applied at the BigQuery table before reading}
+ * --limit {optional; maximum records to read from BigQuery table} + * + *

Note on row restriction: In case a restriction relies on temporal reference, something like + * {@code "TIMESTAMP_TRUNC(ingestion_timestamp, HOUR) = '2023-06-20 19:00:00'"}, and if launching + * the job from Flink's Rest API, a known issue is that single quotes are not supported and will + * cause the pipeline to fail. As a workaround, using \u0027 instead of the quotes will work. For + * example {@code "TIMESTAMP_TRUNC(ingestion_timestamp, HOUR) = \u00272023-06-20 19:00:00\u0027"}. */ public class BigQueryExample { @@ -59,8 +67,8 @@ public static void main(String[] args) throws Exception { "Missing parameters!\n" + "Usage: flink run BigQuery.jar" + " --gcp-project --bq-dataset " - + " --bq-table

--agg-prop " - + " --restriction " + + " --bq-table
--agg-prop " + + " --restriction " + " --limit "); return; } @@ -114,7 +122,7 @@ private static void runFlinkJob( env.fromSource(bqSource, WatermarkStrategy.noWatermarks(), "BigQuerySource") .flatMap(new FlatMapper(recordPropertyToAggregate)) - .keyBy(t -> t.f0) + .keyBy(mappedTuple -> mappedTuple.f0) .max("f1") .print(); From 27c34d042619d3b11f52802f00a30b3953d2cddf Mon Sep 17 00:00:00 2001 From: Jayant Jain Date: Mon, 27 Nov 2023 18:13:31 +0530 Subject: [PATCH 34/34] Make checkpoint interval configurable in example jar --- .../gcp/bigquery/BigQueryExample.java | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java b/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java index 93a27888..df6b6f4e 100644 --- a/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java +++ b/flink-connector-bigquery-examples/src/main/java/org/apache/flink/examples/gcp/bigquery/BigQueryExample.java @@ -46,7 +46,8 @@ * --bq-table {required; name of BigQuery table to read}
* --agg-prop {required; record property to aggregate in Flink job}
* --restriction {optional; SQL-like filter applied at the BigQuery table before reading}
- * --limit {optional; maximum records to read from BigQuery table} + * --limit {optional; maximum records to read from BigQuery table}
+ * --checkpoint-interval {optional; time interval between state checkpoints in milliseconds} * *

Note on row restriction: In case a restriction relies on temporal reference, something like * {@code "TIMESTAMP_TRUNC(ingestion_timestamp, HOUR) = '2023-06-20 19:00:00'"}, and if launching @@ -66,10 +67,13 @@ public static void main(String[] args) throws Exception { LOG.error( "Missing parameters!\n" + "Usage: flink run BigQuery.jar" - + " --gcp-project --bq-dataset " - + " --bq-table

--agg-prop " - + " --restriction " - + " --limit "); + + " --gcp-project " + + " --bq-dataset " + + " --bq-table
" + + " --agg-prop " + + " --restriction " + + " --limit " + + " --checkpoint-interval "); return; } @@ -79,6 +83,7 @@ public static void main(String[] args) throws Exception { String rowRestriction = parameterTool.get("restriction", "").replace("\\u0027", "'"); Integer recordLimit = parameterTool.getInt("limit", -1); String recordPropertyToAggregate = parameterTool.getRequired("agg-prop"); + Long checkpointInterval = parameterTool.getLong("checkpoint-interval", 60000L); runFlinkJob( projectName, @@ -86,7 +91,8 @@ public static void main(String[] args) throws Exception { tableName, recordPropertyToAggregate, rowRestriction, - recordLimit); + recordLimit, + checkpointInterval); } private static void runFlinkJob( @@ -95,11 +101,12 @@ private static void runFlinkJob( String tableName, String recordPropertyToAggregate, String rowRestriction, - Integer limit) + Integer limit, + Long checkpointInterval) throws Exception { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.enableCheckpointing(60000L); + env.enableCheckpointing(checkpointInterval); /** * we will be reading avro generic records from BigQuery, and in this case we are assuming