[
https://issues.apache.org/jira/browse/BEAM-11648?focusedWorklogId=561223&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-561223
]
ASF GitHub Bot logged work on BEAM-11648:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 05/Mar/21 00:00
Start Date: 05/Mar/21 00:00
Worklog Time Spent: 10m
Work Description: yirutang commented on a change in pull request #14136:
URL: https://github.com/apache/beam/pull/14136#discussion_r587918403
##########
File path:
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoTest.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.beam.sdk.io.gcp.bigquery;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.protobuf.DescriptorProtos.DescriptorProto;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto.Label;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto.Type;
+import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.DynamicMessage;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Functions;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.BaseEncoding;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+@SuppressWarnings({
+ "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+/** Unit tests for {@link
org.apache.beam.sdk.io.gcp.bigquery.TableRowToStorageApiProto}. */
+public class TableRowToStorageApiProtoTest {
+ private static final TableSchema BASE_TABLE_SCHEMA =
+ new TableSchema()
+ .setFields(
+ ImmutableList.<TableFieldSchema>builder()
+ .add(new
TableFieldSchema().setType("STRING").setName("stringValue"))
Review comment:
I see.
##########
File path:
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java
##########
@@ -0,0 +1,297 @@
+/*
+ * 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.beam.sdk.io.gcp.bigquery;
+
+import static java.util.stream.Collectors.toList;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.DescriptorProtos.DescriptorProto;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto.Label;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto.Type;
+import com.google.protobuf.DescriptorProtos.FileDescriptorProto;
+import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.Descriptors.DescriptorValidationException;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FileDescriptor;
+import com.google.protobuf.DynamicMessage;
+import com.google.protobuf.Message;
+import java.util.AbstractMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.function.Function;
+import javax.annotation.Nullable;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.BaseEncoding;
+
+/**
+ * Utility methods for converting JSON {@link TableRow} objects to dynamic
protocol message, for use
+ * with the Storage write API.
+ */
+public class TableRowToStorageApiProto {
Review comment:
High leve, why do we need BeamRow->proto and TableRow->proto? Would
BeamRow->proto be enough?
##########
File path:
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoTest.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.beam.sdk.io.gcp.bigquery;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.protobuf.DescriptorProtos.DescriptorProto;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto.Label;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto.Type;
+import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.DynamicMessage;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Functions;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.BaseEncoding;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+@SuppressWarnings({
+ "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+/** Unit tests for {@link
org.apache.beam.sdk.io.gcp.bigquery.TableRowToStorageApiProto}. */
+public class TableRowToStorageApiProtoTest {
+ private static final TableSchema BASE_TABLE_SCHEMA =
+ new TableSchema()
+ .setFields(
+ ImmutableList.<TableFieldSchema>builder()
+ .add(new
TableFieldSchema().setType("STRING").setName("stringValue"))
+ .add(new
TableFieldSchema().setType("BYTES").setName("bytesValue"))
+ .add(new
TableFieldSchema().setType("INT64").setName("int64Value"))
+ .add(new
TableFieldSchema().setType("INTEGER").setName("intValue"))
+ .add(new
TableFieldSchema().setType("FLOAT64").setName("float64Value"))
+ .add(new
TableFieldSchema().setType("FLOAT").setName("floatValue"))
+ .add(new
TableFieldSchema().setType("BOOL").setName("boolValue"))
+ .add(new
TableFieldSchema().setType("BOOLEAN").setName("booleanValue"))
+ .add(new
TableFieldSchema().setType("TIMESTAMP").setName("timestampValue"))
+ .add(new
TableFieldSchema().setType("TIME").setName("timeValue"))
+ .add(new
TableFieldSchema().setType("DATETIME").setName("datetimeValue"))
+ .add(new
TableFieldSchema().setType("DATE").setName("dateValue"))
+ .add(new
TableFieldSchema().setType("NUMERIC").setName("numericValue"))
+ .build());
+
+ private static final DescriptorProto BASE_TABLE_SCHEMA_PROTO =
+ DescriptorProto.newBuilder()
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("stringvalue")
+ .setNumber(1)
+ .setType(Type.TYPE_STRING)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("bytesvalue")
+ .setNumber(2)
+ .setType(Type.TYPE_BYTES)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("int64value")
+ .setNumber(3)
+ .setType(Type.TYPE_INT64)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("intvalue")
+ .setNumber(4)
+ .setType(Type.TYPE_INT64)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("float64value")
+ .setNumber(5)
+ .setType(Type.TYPE_DOUBLE)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("floatvalue")
+ .setNumber(6)
+ .setType(Type.TYPE_DOUBLE)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("boolvalue")
+ .setNumber(7)
+ .setType(Type.TYPE_BOOL)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("booleanvalue")
+ .setNumber(8)
+ .setType(Type.TYPE_BOOL)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("timestampvalue")
+ .setNumber(9)
+ .setType(Type.TYPE_STRING)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("timevalue")
+ .setNumber(10)
+ .setType(Type.TYPE_STRING)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("datetimevalue")
+ .setNumber(11)
+ .setType(Type.TYPE_STRING)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("datevalue")
+ .setNumber(12)
+ .setType(Type.TYPE_STRING)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("numericvalue")
+ .setNumber(13)
+ .setType(Type.TYPE_STRING)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .build();
+
+ private static final TableSchema NESTED_TABLE_SCHEMA =
Review comment:
It is worth to verify the case of array's element type is nested, as you
did in the other test.
##########
File path:
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoTest.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.beam.sdk.io.gcp.bigquery;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.protobuf.DescriptorProtos.DescriptorProto;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto.Label;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto.Type;
+import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.DynamicMessage;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Functions;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.BaseEncoding;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+@SuppressWarnings({
+ "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+/** Unit tests for {@link
org.apache.beam.sdk.io.gcp.bigquery.TableRowToStorageApiProto}. */
+public class TableRowToStorageApiProtoTest {
+ private static final TableSchema BASE_TABLE_SCHEMA =
+ new TableSchema()
+ .setFields(
+ ImmutableList.<TableFieldSchema>builder()
+ .add(new
TableFieldSchema().setType("STRING").setName("stringValue"))
+ .add(new
TableFieldSchema().setType("BYTES").setName("bytesValue"))
+ .add(new
TableFieldSchema().setType("INT64").setName("int64Value"))
+ .add(new
TableFieldSchema().setType("INTEGER").setName("intValue"))
+ .add(new
TableFieldSchema().setType("FLOAT64").setName("float64Value"))
+ .add(new
TableFieldSchema().setType("FLOAT").setName("floatValue"))
+ .add(new
TableFieldSchema().setType("BOOL").setName("boolValue"))
+ .add(new
TableFieldSchema().setType("BOOLEAN").setName("booleanValue"))
+ .add(new
TableFieldSchema().setType("TIMESTAMP").setName("timestampValue"))
+ .add(new
TableFieldSchema().setType("TIME").setName("timeValue"))
+ .add(new
TableFieldSchema().setType("DATETIME").setName("datetimeValue"))
+ .add(new
TableFieldSchema().setType("DATE").setName("dateValue"))
+ .add(new
TableFieldSchema().setType("NUMERIC").setName("numericValue"))
+ .build());
+
+ private static final DescriptorProto BASE_TABLE_SCHEMA_PROTO =
+ DescriptorProto.newBuilder()
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("stringvalue")
+ .setNumber(1)
+ .setType(Type.TYPE_STRING)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("bytesvalue")
+ .setNumber(2)
+ .setType(Type.TYPE_BYTES)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("int64value")
+ .setNumber(3)
+ .setType(Type.TYPE_INT64)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("intvalue")
+ .setNumber(4)
+ .setType(Type.TYPE_INT64)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("float64value")
+ .setNumber(5)
+ .setType(Type.TYPE_DOUBLE)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("floatvalue")
+ .setNumber(6)
+ .setType(Type.TYPE_DOUBLE)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("boolvalue")
+ .setNumber(7)
+ .setType(Type.TYPE_BOOL)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("booleanvalue")
+ .setNumber(8)
+ .setType(Type.TYPE_BOOL)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("timestampvalue")
+ .setNumber(9)
+ .setType(Type.TYPE_STRING)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("timevalue")
+ .setNumber(10)
+ .setType(Type.TYPE_STRING)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("datetimevalue")
+ .setNumber(11)
+ .setType(Type.TYPE_STRING)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("datevalue")
+ .setNumber(12)
+ .setType(Type.TYPE_STRING)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .addField(
+ FieldDescriptorProto.newBuilder()
+ .setName("numericvalue")
+ .setNumber(13)
+ .setType(Type.TYPE_STRING)
+ .setLabel(Label.LABEL_OPTIONAL)
+ .build())
+ .build();
+
+ private static final TableSchema NESTED_TABLE_SCHEMA =
+ new TableSchema()
+ .setFields(
+ ImmutableList.<TableFieldSchema>builder()
+ .add(
+ new TableFieldSchema()
+ .setType("STRUCT")
+ .setName("nestedValue1")
+ .setFields(BASE_TABLE_SCHEMA.getFields()))
+ .add(
+ new TableFieldSchema()
+ .setType("RECORD")
+ .setName("nestedValue2")
+ .setFields(BASE_TABLE_SCHEMA.getFields()))
+ .build());
+
+ // For now, test that no exceptions are thrown.
+ @Test
+ public void testDescriptorFromTableSchema() {
+ DescriptorProto descriptor =
+
TableRowToStorageApiProto.descriptorSchemaFromTableSchema(BASE_TABLE_SCHEMA);
+ Map<String, Type> types =
+ descriptor.getFieldList().stream()
+ .collect(
+ Collectors.toMap(FieldDescriptorProto::getName,
FieldDescriptorProto::getType));
+ Map<String, Type> expectedTypes =
+ BASE_TABLE_SCHEMA_PROTO.getFieldList().stream()
+ .collect(
+ Collectors.toMap(FieldDescriptorProto::getName,
FieldDescriptorProto::getType));
+ assertEquals(expectedTypes, types);
+ }
+
+ @Test
+ public void testNestedFromTableSchema() {
+ DescriptorProto descriptor =
+
TableRowToStorageApiProto.descriptorSchemaFromTableSchema(NESTED_TABLE_SCHEMA);
+ Map<String, Type> expectedBaseTypes =
+ BASE_TABLE_SCHEMA_PROTO.getFieldList().stream()
+ .collect(
+ Collectors.toMap(FieldDescriptorProto::getName,
FieldDescriptorProto::getType));
+
+ Map<String, Type> types =
+ descriptor.getFieldList().stream()
+ .collect(
+ Collectors.toMap(FieldDescriptorProto::getName,
FieldDescriptorProto::getType));
+ Map<String, String> typeNames =
+ descriptor.getFieldList().stream()
+ .collect(
+ Collectors.toMap(FieldDescriptorProto::getName,
FieldDescriptorProto::getTypeName));
+ assertEquals(2, types.size());
+
+ Map<String, DescriptorProto> nestedTypes =
+ descriptor.getNestedTypeList().stream()
+ .collect(Collectors.toMap(DescriptorProto::getName,
Functions.identity()));
+ assertEquals(2, nestedTypes.size());
+ assertEquals(Type.TYPE_MESSAGE, types.get("nestedvalue1"));
+ String nestedTypeName1 = typeNames.get("nestedvalue1");
+ Map<String, Type> nestedTypes1 =
+ nestedTypes.get(nestedTypeName1).getFieldList().stream()
+ .collect(
+ Collectors.toMap(FieldDescriptorProto::getName,
FieldDescriptorProto::getType));
+ assertEquals(expectedBaseTypes, nestedTypes1);
+
+ assertEquals(Type.TYPE_MESSAGE, types.get("nestedvalue2"));
+ String nestedTypeName2 = typeNames.get("nestedvalue2");
+ Map<String, Type> nestedTypes2 =
+ nestedTypes.get(nestedTypeName2).getFieldList().stream()
+ .collect(
+ Collectors.toMap(FieldDescriptorProto::getName,
FieldDescriptorProto::getType));
+ assertEquals(expectedBaseTypes, nestedTypes2);
+ }
+
+ @Test
+ public void testRepeatedDescriptorFromTableSchema() {
+
TableRowToStorageApiProto.descriptorSchemaFromTableSchema(BASE_TABLE_SCHEMA);
+ }
+
+ private static final TableRow BASE_TABLE_ROW =
+ new TableRow()
+ .set("stringValue", "string")
+ .set(
+ "bytesValue",
BaseEncoding.base64().encode("string".getBytes(StandardCharsets.UTF_8)))
+ .set("int64Value", "42")
+ .set("intValue", "43")
+ .set("float64Value", "2.8168")
+ .set("floatValue", "2.817")
+ .set("boolValue", "true")
+ .set("booleanValue", "true")
+ .set("timestampValue", "43")
+ .set("timeValue", "00:52:07[.123]|[.123456] UTC")
+ .set("datetimeValue", "2019-08-16 00:52:07[.123]|[.123456] UTC")
+ .set("dateValue", "2019-08-16")
+ .set("numericValue", "23.4");
+
+ private void assertBaseRecord(DynamicMessage msg) {
Review comment:
maybe you haven't pushed the change?
##########
File path:
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BeamRowToStorageApiProto.java
##########
@@ -0,0 +1,339 @@
+/*
+ * 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.beam.sdk.io.gcp.bigquery;
+
+import com.google.protobuf.ByteString;
+import com.google.protobuf.DescriptorProtos.DescriptorProto;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto.Label;
+import com.google.protobuf.DescriptorProtos.FieldDescriptorProto.Type;
+import com.google.protobuf.DescriptorProtos.FileDescriptorProto;
+import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.Descriptors.DescriptorValidationException;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FileDescriptor;
+import com.google.protobuf.DynamicMessage;
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.Schema.LogicalType;
+import org.apache.beam.sdk.schemas.Schema.TypeName;
+import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType;
+import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes;
+import org.apache.beam.sdk.values.Row;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Functions;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.Bytes;
+import org.joda.time.Instant;
+import org.joda.time.ReadableInstant;
+
+/**
+ * Utility methods for converting Beam {@link Row} objects to dynamic protocol
message, for use with
+ * the Storage write API.
+ */
+public class BeamRowToStorageApiProto {
+ // Number of digits after the decimal point supported by the NUMERIC data
type.
+ private static final int NUMERIC_SCALE = 9;
+ // Maximum and minimum allowed values for the NUMERIC data type.
+ private static final BigDecimal MAX_NUMERIC_VALUE =
+ new BigDecimal("99999999999999999999999999999.999999999");
+ private static final BigDecimal MIN_NUMERIC_VALUE =
+ new BigDecimal("-99999999999999999999999999999.999999999");
+
+ // TODO(reuvenlax): Support BIGNUMERIC and GEOGRAPHY types.
+ static final Map<TypeName, Type> PRIMITIVE_TYPES =
+ ImmutableMap.<TypeName, Type>builder()
+ .put(TypeName.INT16, Type.TYPE_INT32)
+ .put(TypeName.BYTE, Type.TYPE_INT32)
+ .put(TypeName.INT32, Type.TYPE_INT32)
+ .put(TypeName.INT64, Type.TYPE_INT64)
+ .put(TypeName.FLOAT, Type.TYPE_FLOAT)
+ .put(TypeName.DOUBLE, Type.TYPE_DOUBLE)
+ .put(TypeName.STRING, Type.TYPE_STRING)
+ .put(TypeName.BOOLEAN, Type.TYPE_BOOL)
+ .put(TypeName.DATETIME, Type.TYPE_INT64)
+ .put(TypeName.BYTES, Type.TYPE_BYTES)
+ .put(TypeName.DECIMAL, Type.TYPE_BYTES)
+ .build();
+
+ // A map of supported logical types to the protobuf field type.
+ static final Map<String, Type> LOGICAL_TYPES =
+ ImmutableMap.<String, Type>builder()
+ .put(SqlTypes.DATE.getIdentifier(), Type.TYPE_INT32)
+ .put(SqlTypes.TIME.getIdentifier(), Type.TYPE_INT64)
+ .put(SqlTypes.DATETIME.getIdentifier(), Type.TYPE_INT64)
+ .put(SqlTypes.TIMESTAMP.getIdentifier(), Type.TYPE_INT64)
+ .put(EnumerationType.IDENTIFIER, Type.TYPE_STRING)
+ .build();
+
+ static final Map<TypeName, Function<Object, Object>> PRIMITIVE_ENCODERS =
+ ImmutableMap.<TypeName, Function<Object, Object>>builder()
+ .put(TypeName.INT16, o -> Integer.valueOf((Short) o))
+ .put(TypeName.BYTE, o -> Integer.valueOf((Byte) o))
+ .put(TypeName.INT32, Functions.identity())
+ .put(TypeName.INT64, Functions.identity())
+ .put(TypeName.FLOAT, Function.identity())
+ .put(TypeName.DOUBLE, Function.identity())
+ .put(TypeName.STRING, Function.identity())
+ .put(TypeName.BOOLEAN, Function.identity())
+ .put(TypeName.DATETIME, o -> ((ReadableInstant) o).getMillis() *
1000)
Review comment:
nvm, just realize it is DATETIME from different types.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 561223)
Time Spent: 24h 40m (was: 24.5h)
> Implement new BigQuery sink (Vortex)
> ------------------------------------
>
> Key: BEAM-11648
> URL: https://issues.apache.org/jira/browse/BEAM-11648
> Project: Beam
> Issue Type: New Feature
> Components: extensions-java-gcp
> Reporter: Reuven Lax
> Priority: P2
> Time Spent: 24h 40m
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)