diff --git a/build.gradle b/build.gradle index e854c35a..e44b05c3 100644 --- a/build.gradle +++ b/build.gradle @@ -22,13 +22,15 @@ plugins { } group 'com.gotocompany' -version '0.9.2' +version '0.10.0' repositories { mavenLocal() mavenCentral() } +configurations.configureEach { exclude group: 'com.google.guava', module: 'listenablefuture' } + dependencies { implementation group: 'com.google.protobuf', name: 'protobuf-java', version: '3.25.0' implementation group: 'com.datadoghq', name: 'java-dogstatsd-client', version: '2.13.0' @@ -44,6 +46,10 @@ dependencies { implementation(group: 'com.google.cloud', name: 'google-cloud-bigtable', version: '2.24.1') { exclude group: "io.grpc" } + implementation (group: 'com.aliyun.odps', name: 'odps-sdk-core', version: '0.51.0-public.rc1') { + exclude group: "com.google" + exclude group: "io.grpc" + } implementation 'io.grpc:grpc-all:1.55.1' implementation group: 'org.slf4j', name: 'jul-to-slf4j', version: '1.7.35' implementation group: 'redis.clients', name: 'jedis', version: '3.10.0' @@ -53,7 +59,9 @@ dependencies { implementation group: 'com.jayway.jsonpath', name: 'json-path', version: '2.8.0' implementation group: 'com.squareup.okhttp3', name: 'okhttp', version: '4.9.1' implementation group: 'joda-time', name: 'joda-time', version: '2.10.2' + implementation('com.google.guava:guava:32.0.1-jre') { force = true } testImplementation group: 'junit', name: 'junit', version: '4.13.1' + testImplementation group: 'org.assertj', name: 'assertj-core', version: '3.26.3' testImplementation 'org.hamcrest:hamcrest-all:1.3' testImplementation 'org.mockito:mockito-core:4.5.1' testImplementation 'com.github.tomakehurst:wiremock:2.16.0' @@ -201,7 +209,10 @@ jacocoTestCoverageVerification { '**/serializer/**', '**/cortexpb/**', '**/Clock**', - '**/GoGoProtos**',]) + '**/GoGoProtos**', + '**/MaxComputeClient**', + '**/MaxComputeSinkFactory**', + '']) }) } violationRules { diff --git a/docs/reference/configuration/maxcompute.md b/docs/reference/configuration/maxcompute.md new file mode 100644 index 00000000..c267eb92 --- /dev/null +++ b/docs/reference/configuration/maxcompute.md @@ -0,0 +1,244 @@ +# MaxCompute Sink + +A MaxCompute sink requires these configurations to be passed on alongside with generic ones + +## SINK_MAXCOMPUTE_ODPS_URL + +Contains the URL of the MaxCompute endpoint. Further documentation on MaxCompute [ODPS URL](https://www.alibabacloud.com/help/en/maxcompute/user-guide/endpoints). +* Example value: `http://service.ap-southeast-5.maxcompute.aliyun.com/api` +* Type: `required` + +## SINK_MAXCOMPUTE_ACCESS_ID + +Contains the access id of the MaxCompute project. Further documentation on MaxCompute [Access ID](https://www.alibabacloud.com/help/en/tablestore/support/obtain-an-accesskey-pair). +* Example value: `access-id` +* Type: `required` + +## SINK_MAXCOMPUTE_ACCESS_KEY + +Contains the access key of the MaxCompute project. Further documentation on MaxCompute [Access Key](https://www.alibabacloud.com/help/en/tablestore/support/obtain-an-accesskey-pair). +* Example value: `access-key` +* Type: `required` + +## SINK_MAXCOMPUTE_PROJECT_ID + +Contains the identifier of a MaxCompute project. Further documentation on MaxCompute [Project ID](https://www.alibabacloud.com/help/en/maxcompute/product-overview/project). +* Example value: `project-id` +* Type: `required` + +## SINK_MAXCOMPUTE_ADD_METADATA_ENABLED + +Configuration for enabling metadata in top of the record. This config will be used for adding metadata information to the record. Metadata information will be added to the record in the form of key-value pair. +* Example value: `false` +* Type: `required` +* Default value: `true` + +## SINK_MAXCOMPUTE_METADATA_NAMESPACE + +Configuration for wrapping the metadata fields under a specific namespace. This will result in the metadata fields contained in a struct. +Empty string will result in the metadata fields being added directly to the root level. +* Example value: `__kafka_metadata` +* Type: `optional` + +## SINK_MAXCOMPUTE_METADATA_COLUMNS_TYPES + +Configuration for defining the metadata columns and their types. This config will be used for defining the metadata columns and their types. The format of this config is `column1=type1,column2=type2`. +Supported types are `string`, `integer`, `long`, `timestamp`, `float`, `double`, `boolean`. + +* Example value: `topic=string,partition=integer,offset=integer,timestamp=timestamp` +* Type: `optional` + +## SINK_MAXCOMPUTE_SCHEMA + +Contains the schema of the MaxCompute table. Schema is a dataset grouping of table columns. Further documentation on MaxCompute [Schema](https://www.alibabacloud.com/help/en/maxcompute/user-guide/schemas). +* Example value: `your_dataset_name` +* Type: `required` +* Default value: `default` + +## SINK_MAXCOMPUTE_TABLE_PARTITIONING_ENABLE + +Configuration for enabling partitioning in the MaxCompute table. This config will be used for enabling partitioning in the MaxCompute table. +* Example value: `true` +* Type: `required` +* Default value: `false` + +## SINK_MAXCOMPUTE_TABLE_PARTITION_KEY + +Contains the partition key of the MaxCompute table. Partition key is referring to the payload field that will be used as partition key in the MaxCompute table. +Supported MaxCompute type for partition key is `string`, `tinyint`, `smallint`, `int`, `bigint`, `timestamp_ntz`. +* Example value: `column1` +* Type: `optional` +* Default value: `default` + +## SINK_MAXCOMPUTE_TABLE_PARTITION_BY_TIMESTAMP_TIME_UNIT + +Contains the time unit for partitioning by timestamp. This config will be used for setting the time unit for partitioning by timestamp. +Supported time units are `YEAR`, `MONTH`, `DAY`, `HOUR`. Configuration is case-sensitive. + +* Example value: `DAYS` +* Type: `required` +* Default value: `DAYS` + +## SINK_MAXCOMPUTE_TABLE_PARTITION_COLUMN_NAME + +Contains the partition column name of the MaxCompute table. This could be the same as the partition key or different. This will reflect the column name in the MaxCompute table. +Here the SINK_MAXCOMPUTE_TABLE_PARTITION_COLUMN_NAME is differentiated with SINK_MAXCOMPUTE_TABLE_PARTITION_KEY to allow the user to have a different column name in the MaxCompute table. +This is used for timestamp auto-partitioning feature where the partition column coexists with the original column. + +* Example value: `column1` +* Type: `optional` + +## SINK_MAXCOMPUTE_TABLE_NAME + +Contains the name of the MaxCompute table. Further documentation on MaxCompute [Table Name](https://www.alibabacloud.com/help/en/maxcompute/user-guide/tables). +* Example value: `table_name` +* Type: `required` + +## SINK_MAXCOMPUTE_TABLE_LIFECYCLE_DAYS + +Contains the lifecycle of the MaxCompute table. This config will be used for setting the lifecycle of the MaxCompute table. +Not setting this config will result in table with lifecycle. Lifecycle is applied at partition level. Further documentation on MaxCompute [Table Lifecycle](https://www.alibabacloud.com/help/en/maxcompute/product-overview/lifecycle). +* Example value: `30` +* Type: `optional` + +## SINK_MAXCOMPUTE_RECORD_PACK_FLUSH_TIMEOUT_MS + +Contains the timeout for flushing the record pack in milliseconds. This config will be used for setting the timeout for flushing the record pack. Negative value indicates no timeout. +* Example value: `1000` +* Type: `required` +* Default value: `-1` + +## SINK_MAXCOMPUTE_STREAMING_INSERT_COMPRESSION_ENABLED + +Configuration for enabling compression in the streaming insert operation. This config will be used for enabling compression in the streaming insert operation. +* Example value: `false` +* Type: `required` +* Default value: `true` + +## SINK_MAXCOMPUTE_STREAMING_INSERT_COMPRESSION_ALGORITHM + +Configuration for defining the compression algorithm in the streaming insert operation. This config will be used for defining the compression algorithm in the streaming insert operation. +Supported values are ODPS_RAW, ODPS_ZLIB, ODPS_LZ4_FRAME, ODPS_ARROW_LZ4_FRAME, ODPS_ARROW_ZSTD +* Example value: `ODPS_ZLIB` +* Type: `required` +* Default value: `ODPS_LZ4_FRAME` + +## SINK_MAXCOMPUTE_STREAMING_INSERT_COMPRESSION_LEVEL + +Configuration for defining the compression level in the streaming insert operation. This config will be used for defining the compression level in the streaming insert operation. +Further documentation on MaxCompute [Compression](https://www.alibabacloud.com/help/en/maxcompute/user-guide/sdk-interfaces#section-cg2-7mb-849). +* Example value: `1` +* Type: `required` +* Default value: `1` + +## SINK_MAXCOMPUTE_STREAMING_INSERT_COMPRESSION_STRATEGY + +Configuration for defining the compression strategy in the streaming insert operation. This config will be used for defining the compression strategy in the streaming insert operation. +Further documentation on MaxCompute [Compression](https://www.alibabacloud.com/help/en/maxcompute/user-guide/sdk-interfaces#section-cg2-7mb-849). + +* Example value: `1` +* Type: `required` +* Default value: `0` + +## SINK_MAXCOMPUTE_STREAMING_INSERT_MAXIMUM_SESSION_COUNT + +Contains the maximum session cached count for the streaming insert operation. This config will be used for setting the maximum session cache capacity for the streaming insert operation. +Least recently used session will be removed if the cache is full. + +* Example value: `7` +* Type: `required` +* Default value: `2` + +## SINK_MAXCOMPUTE_ZONE_ID + +Contains ZoneID used for parsing the timestamp in the record. This config will be used for parsing the timestamp in the record. + +* Example value: `Asia/Bangkok` +* Type: `required` +* Default value: `Asia/Bangkok` + +## SINK_MAXCOMPUTE_MAX_DDL_RETRY_COUNT + +Contains the maximum retry count for DDL operations. This config will be used for setting the maximum retry count for DDL operations (create and update table schema). + +* Example value: `3` +* Type: `required` +* Default value: `3` + +## SINK_MAXCOMPUTE_DDL_RETRY_BACKOFF_MILLIS + +Contains the backoff time in milliseconds for DDL operations. This config will be used for setting the backoff time in milliseconds for DDL operations (create and update table schema). + +* Example value: `10000` +* Type: `required` +* Default value: `1000` + +## SINK_MAXCOMPUTE_ODPS_GLOBAL_SETTINGS + +Contains the global settings for the MaxCompute sink. This config will be used for setting the global settings for the MaxCompute sink. The format of this config is `key1=value1,key2=value2`. + +* Example value: `odps.schema.evolution.enable=true,odps.namespace.schema=true,odps.sql.type.system.odps2=true` +* Type: `optional` +* Default value: `odps.schema.evolution.enable=true,odps.namespace.schema=true` + +## SINK_MAXCOMPUTE_TABLE_VALIDATOR_NAME_REGEX + +Contains the regex pattern for the table name validation. This config will be used for validating the table name. The table name should match the regex pattern. +Check the official documentation for the [table name](https://www.alibabacloud.com/help/en/maxcompute/product-overview/limits-4#:~:text=A%20table%20can%20contain%20a%20maximum%20of%2060%2C000%20partitions.&text=A%20table%20can%20contain%20a%20maximum%20of%20six%20levels%20of%20partitions.&text=A%20SELECT%20statement%20can%20return%20a%20maximum%20of%2010%2C000%20rows.&text=A%20MULTI%2DINSERT%20statement%20allows,tables%20at%20the%20same%20time.) for more information. + +* Example value: `^[a-zA-Z_][a-zA-Z0-9_]*$` +* Type: `required` +* Default value: `^[A-Za-z][A-Za-z0-9_]{0,127}$` + +## SINK_MAXCOMPUTE_TABLE_VALIDATOR_MAX_COLUMNS_PER_TABLE + +Contains the maximum number of columns allowed in the table. This config will be used for setting the maximum number of columns allowed in the table. +Check the official documentation for the [table name](https://www.alibabacloud.com/help/en/maxcompute/product-overview/limits-4#:~:text=A%20table%20can%20contain%20a%20maximum%20of%2060%2C000%20partitions.&text=A%20table%20can%20contain%20a%20maximum%20of%20six%20levels%20of%20partitions.&text=A%20SELECT%20statement%20can%20return%20a%20maximum%20of%2010%2C000%20rows.&text=A%20MULTI%2DINSERT%20statement%20allows,tables%20at%20the%20same%20time.) for more information. + +* Example value: `1000` +* Type: `required` +* Default value: `1200` + +## SINK_MAXCOMPUTE_TABLE_VALIDATOR_MAX_PARTITION_KEYS_PER_TABLE + +Contains the maximum number of partition keys allowed in the table. This config will be used for setting the maximum number of partition keys allowed in the table. +Check the official documentation for the [table name](https://www.alibabacloud.com/help/en/maxcompute/product-overview/limits-4#:~:text=A%20table%20can%20contain%20a%20maximum%20of%2060%2C000%20partitions.&text=A%20table%20can%20contain%20a%20maximum%20of%20six%20levels%20of%20partitions.&text=A%20SELECT%20statement%20can%20return%20a%20maximum%20of%2010%2C000%20rows.&text=A%20MULTI%2DINSERT%20statement%20allows,tables%20at%20the%20same%20time.) for more information. + +* Example value: `6` +* Type: `required` +* Default value: `6` + +## SINK_MAXCOMPUTE_VALID_MIN_TIMESTAMP + +Contains the minimum valid timestamp. Records with timestamp field less than this value will be considered as invalid message. +Timestamp should be in the format of `yyyy-MM-ddTHH:mm:ss`. + +* Example value: `0` +* Type: `required` +* Default value: `1970-01-01T00:00:00` + +## SINK_MAXCOMPUTE_VALID_MAX_TIMESTAMP + +Contains the maximum valid timestamp. Records with timestamp field more than this value will be considered as invalid message. +Timestamp should be in the format of `yyyy-MM-ddTHH:mm:ss`. + +* Example value: `0` +* Type: `required` +* Default value: `9999-12-31T23:59:59` + +## SINK_MAXCOMPUTE_MAX_PAST_EVENT_TIME_DIFFERENCE_YEAR + +Contains the maximum past event time difference in years. Records with event time difference more than this value will be considered as invalid message. + +* Example value: `1` +* Type: `required` +* Default value: `5` + +## SINK_MAXCOMPUTE_MAX_FUTURE_EVENT_TIME_DIFFERENCE_YEAR + +Contains the maximum future event time difference in years. Records with event time difference more than this value will be considered as invalid message. + +* Example value: `1` +* Type: `required` +* Default value: `1` + diff --git a/docs/sinks/maxcompute.md b/docs/sinks/maxcompute.md new file mode 100644 index 00000000..0c91efd1 --- /dev/null +++ b/docs/sinks/maxcompute.md @@ -0,0 +1,56 @@ +# MaxCompute sink + +### Datatype Protobuf + +MaxCompute sink has several responsibilities, including : + +1. Creation of MaxCompute table if it does not exist. +2. Updating the MaxCompute table schema based on the latest protobuf schema. +3. Translating protobuf messages into MaxCompute compatible records and inserting them into MaxCompute tables. + +## MaxCompute Table Schema Update + +### JSON (ToDo) + +### Protobuf + +MaxCompute Sink update the MaxCompute table schema on separate table update operation. MaxCompute +utilise [Stencil](https://github.com/goto/stencil) to parse protobuf messages generate schema and update MaxCompute +tables with the latest schema. +The stencil client periodically reload the descriptor cache. Table schema update happened after the descriptor caches +uploaded. + +#### Supported Protobuf - MaxCompute Table Type Mapping + +| Protobuf Type | MaxCompute Type | +|------------------------------------------------------------------------------------|-------------------------------| +| bytes | BINARY | +| string | STRING | +| enum | STRING | +| float | FLOAT | +| double | DOUBLE | +| bool | BOOLEAN | +| int64, uint64, int32, uint32, fixed64, fixed32, sfixed64, sfixed32, sint64, sint32 | BIGINT | +| message | STRUCT | +| .google.protobuf.Timestamp | TIMESTAMP_NTZ | +| .google.protobuf.Struct | STRING (Json Serialised) | +| .google.protobuf.Duration | STRUCT | +| map | ARRAY> | + +## Partitioning + +MaxCompute Sink supports creation of table with partition configuration. Currently, MaxCompute Sink supports primitive field(STRING, TINYINT, SMALLINT, BIGINT) +and timestamp field based partitioning. Timestamp based partitioning strategy introduces a pseudo-partition column with the value of the timestamp field truncated to the nearest start of day. + +## Clustering + +MaxCompute Sink currently does not support clustering. + +## Metadata + +For data quality checking purposes, sometimes some metadata need to be added on the record. +if `SINK_MAXCOMPUTE_ADD_METADATA_ENABLED` is true then the metadata will be added. +`SINK_MAXCOMPUTE_METADATA_NAMESPACE` is used for another namespace to add columns +if namespace is empty, the metadata columns will be added in the root level. +`SINK_MAXCOMPUTE_METADATA_COLUMNS_TYPES` is set with kafka metadata column and their type, +An example of metadata columns that can be added for kafka records. \ No newline at end of file diff --git a/src/main/java/com/gotocompany/depot/config/MaxComputeSinkConfig.java b/src/main/java/com/gotocompany/depot/config/MaxComputeSinkConfig.java new file mode 100644 index 00000000..9f31307f --- /dev/null +++ b/src/main/java/com/gotocompany/depot/config/MaxComputeSinkConfig.java @@ -0,0 +1,143 @@ +package com.gotocompany.depot.config; + +import com.aliyun.odps.tunnel.io.CompressOption; +import com.gotocompany.depot.common.TupleString; +import com.gotocompany.depot.config.converter.ConfToListConverter; +import com.gotocompany.depot.config.converter.LocalDateTimeConverter; +import com.gotocompany.depot.config.converter.MaxComputeOdpsGlobalSettingsConverter; +import com.gotocompany.depot.config.converter.ZoneIdConverter; +import org.aeonbits.owner.Config; + +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.List; +import java.util.Map; + +public interface MaxComputeSinkConfig extends Config { + + @Key("SINK_MAXCOMPUTE_ODPS_URL") + String getMaxComputeOdpsUrl(); + + @Key("SINK_MAXCOMPUTE_TUNNEL_URL") + String getMaxComputeTunnelUrl(); + + @Key("SINK_MAXCOMPUTE_ACCESS_ID") + String getMaxComputeAccessId(); + + @Key("SINK_MAXCOMPUTE_ACCESS_KEY") + String getMaxComputeAccessKey(); + + @Key("SINK_MAXCOMPUTE_PROJECT_ID") + String getMaxComputeProjectId(); + + @Key("SINK_MAXCOMPUTE_METADATA_NAMESPACE") + @DefaultValue("") + String getMaxcomputeMetadataNamespace(); + + @Key("SINK_MAXCOMPUTE_ADD_METADATA_ENABLED") + @DefaultValue("true") + boolean shouldAddMetadata(); + + @DefaultValue("") + @Key("SINK_MAXCOMPUTE_METADATA_COLUMNS_TYPES") + @ConverterClass(ConfToListConverter.class) + @Separator(ConfToListConverter.ELEMENT_SEPARATOR) + List getMetadataColumnsTypes(); + + @Key("SINK_MAXCOMPUTE_SCHEMA") + @DefaultValue("default") + String getMaxComputeSchema(); + + @Key("SINK_MAXCOMPUTE_TABLE_PARTITIONING_ENABLE") + @DefaultValue("false") + Boolean isTablePartitioningEnabled(); + + @Key("SINK_MAXCOMPUTE_TABLE_PARTITION_KEY") + String getTablePartitionKey(); + + @Key("SINK_MAXCOMPUTE_TABLE_PARTITION_COLUMN_NAME") + String getTablePartitionColumnName(); + + @Key("SINK_MAXCOMPUTE_TABLE_PARTITION_BY_TIMESTAMP_TIME_UNIT") + @DefaultValue("DAY") + String getTablePartitionByTimestampTimeUnit(); + + @Key("SINK_MAXCOMPUTE_TABLE_NAME") + String getMaxComputeTableName(); + + @Key("SINK_MAXCOMPUTE_TABLE_LIFECYCLE_DAYS") + Long getMaxComputeTableLifecycleDays(); + + @Key("SINK_MAXCOMPUTE_RECORD_PACK_FLUSH_TIMEOUT_MS") + @DefaultValue("-1") + Long getMaxComputeRecordPackFlushTimeoutMs(); + + @Key("SINK_MAXCOMPUTE_STREAMING_INSERT_COMPRESSION_ENABLED") + @DefaultValue("false") + boolean isStreamingInsertCompressEnabled(); + + @Key("SINK_MAXCOMPUTE_STREAMING_INSERT_COMPRESSION_ALGORITHM") + @DefaultValue("ODPS_LZ4_FRAME") + CompressOption.CompressAlgorithm getMaxComputeCompressionAlgorithm(); + + @Key("SINK_MAXCOMPUTE_STREAMING_INSERT_COMPRESSION_LEVEL") + @DefaultValue("1") + int getMaxComputeCompressionLevel(); + + @Key("SINK_MAXCOMPUTE_STREAMING_INSERT_COMPRESSION_STRATEGY") + @DefaultValue("0") + int getMaxComputeCompressionStrategy(); + + @Key("SINK_MAXCOMPUTE_STREAMING_INSERT_MAXIMUM_SESSION_COUNT") + @DefaultValue("2") + int getStreamingInsertMaximumSessionCount(); + + @Key("SINK_MAXCOMPUTE_ZONE_ID") + @ConverterClass(ZoneIdConverter.class) + @DefaultValue("Asia/Bangkok") + ZoneId getZoneId(); + + @Key("SINK_MAXCOMPUTE_MAX_DDL_RETRY_COUNT") + @DefaultValue("10") + int getMaxDdlRetryCount(); + + @Key("SINK_MAXCOMPUTE_DDL_RETRY_BACKOFF_MILLIS") + @DefaultValue("1000") + long getDdlRetryBackoffMillis(); + + @Key("SINK_MAXCOMPUTE_ODPS_GLOBAL_SETTINGS") + @ConverterClass(MaxComputeOdpsGlobalSettingsConverter.class) + @DefaultValue("odps.schema.evolution.enable=true,odps.namespace.schema=true") + Map getOdpsGlobalSettings(); + + @Key("SINK_MAXCOMPUTE_TABLE_VALIDATOR_NAME_REGEX") + @DefaultValue("^[A-Za-z][A-Za-z0-9_]{0,127}$") + String getTableValidatorNameRegex(); + + @Key("SINK_MAXCOMPUTE_TABLE_VALIDATOR_MAX_COLUMNS_PER_TABLE") + @DefaultValue("1200") + int getTableValidatorMaxColumnsPerTable(); + + @Key("SINK_MAXCOMPUTE_TABLE_VALIDATOR_MAX_PARTITION_KEYS_PER_TABLE") + @DefaultValue("6") + int getTableValidatorMaxPartitionKeysPerTable(); + + @Key("SINK_MAXCOMPUTE_VALID_MIN_TIMESTAMP") + @ConverterClass(LocalDateTimeConverter.class) + @DefaultValue("1970-01-01T00:00:00") + LocalDateTime getValidMinTimestamp(); + + @Key("SINK_MAXCOMPUTE_VALID_MAX_TIMESTAMP") + @ConverterClass(LocalDateTimeConverter.class) + @DefaultValue("9999-12-31T23:59:59") + LocalDateTime getValidMaxTimestamp(); + + @Key("SINK_MAXCOMPUTE_MAX_PAST_EVENT_TIME_DIFFERENCE_YEAR") + @DefaultValue("5") + int getMaxPastYearEventTimeDifference(); + + @Key("SINK_MAXCOMPUTE_MAX_FUTURE_EVENT_TIME_DIFFERENCE_YEAR") + @DefaultValue("1") + int getMaxFutureYearEventTimeDifference(); + +} diff --git a/src/main/java/com/gotocompany/depot/config/SinkConfig.java b/src/main/java/com/gotocompany/depot/config/SinkConfig.java index 8380326e..1b4f4c9e 100644 --- a/src/main/java/com/gotocompany/depot/config/SinkConfig.java +++ b/src/main/java/com/gotocompany/depot/config/SinkConfig.java @@ -6,6 +6,7 @@ import com.gotocompany.depot.config.converter.SinkConnectorSchemaDataTypeConverter; import com.gotocompany.depot.config.converter.SinkConnectorSchemaMessageModeConverter; import com.gotocompany.depot.config.enums.SinkConnectorSchemaDataType; +import com.gotocompany.depot.message.ProtoUnknownFieldValidationType; import com.gotocompany.depot.message.SinkConnectorSchemaMessageMode; import com.gotocompany.depot.common.TupleString; import com.gotocompany.stencil.cache.SchemaRefreshStrategy; @@ -99,4 +100,8 @@ public interface SinkConfig extends Config { @DefaultValue("true") boolean getSinkDefaultFieldValueEnable(); + @Key("SINK_CONNECTOR_SCHEMA_PROTO_UNKNOWN_FIELDS_VALIDATION") + @DefaultValue("MESSAGE") + ProtoUnknownFieldValidationType getSinkConnectorSchemaProtoUnknownFieldsValidation(); + } diff --git a/src/main/java/com/gotocompany/depot/config/converter/LocalDateTimeConverter.java b/src/main/java/com/gotocompany/depot/config/converter/LocalDateTimeConverter.java new file mode 100644 index 00000000..8bbf797a --- /dev/null +++ b/src/main/java/com/gotocompany/depot/config/converter/LocalDateTimeConverter.java @@ -0,0 +1,16 @@ +package com.gotocompany.depot.config.converter; + +import org.aeonbits.owner.Converter; + +import java.lang.reflect.Method; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; + +public class LocalDateTimeConverter implements Converter { + + @Override + public LocalDateTime convert(Method method, String s) { + return LocalDateTime.parse(s, DateTimeFormatter.ISO_DATE_TIME); + } + +} diff --git a/src/main/java/com/gotocompany/depot/config/converter/MaxComputeOdpsGlobalSettingsConverter.java b/src/main/java/com/gotocompany/depot/config/converter/MaxComputeOdpsGlobalSettingsConverter.java new file mode 100644 index 00000000..df195127 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/config/converter/MaxComputeOdpsGlobalSettingsConverter.java @@ -0,0 +1,33 @@ +package com.gotocompany.depot.config.converter; + +import org.aeonbits.owner.Converter; +import org.apache.commons.lang3.StringUtils; + +import java.lang.reflect.Method; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +public class MaxComputeOdpsGlobalSettingsConverter implements Converter> { + + private static final String CONFIG_SEPARATOR = ","; + private static final String KEY_VALUE_SEPARATOR = "="; + + @Override + public Map convert(Method method, String s) { + Map settings = new HashMap<>(); + if (Objects.isNull(s) || StringUtils.isEmpty(s.trim())) { + return settings; + } + String[] pairs = s.split(CONFIG_SEPARATOR); + for (String pair : pairs) { + String[] keyValue = pair.split(KEY_VALUE_SEPARATOR); + if (keyValue.length != 2) { + throw new IllegalArgumentException("Invalid key-value pair: " + pair); + } + settings.put(keyValue[0].trim(), keyValue[1].trim()); + } + return settings; + } + +} diff --git a/src/main/java/com/gotocompany/depot/config/converter/ZoneIdConverter.java b/src/main/java/com/gotocompany/depot/config/converter/ZoneIdConverter.java new file mode 100644 index 00000000..052f7c2d --- /dev/null +++ b/src/main/java/com/gotocompany/depot/config/converter/ZoneIdConverter.java @@ -0,0 +1,20 @@ +package com.gotocompany.depot.config.converter; + +import com.gotocompany.depot.exception.ConfigurationException; +import org.aeonbits.owner.Converter; + +import java.lang.reflect.Method; +import java.time.ZoneId; + +public class ZoneIdConverter implements Converter { + + @Override + public ZoneId convert(Method method, String s) { + try { + return ZoneId.of(s); + } catch (Exception e) { + throw new ConfigurationException("Invalid ZoneId: " + s, e); + } + } + +} diff --git a/src/main/java/com/gotocompany/depot/exception/InvalidMessageException.java b/src/main/java/com/gotocompany/depot/exception/InvalidMessageException.java new file mode 100644 index 00000000..b85b10c6 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/exception/InvalidMessageException.java @@ -0,0 +1,7 @@ +package com.gotocompany.depot.exception; + +public class InvalidMessageException extends RuntimeException { + public InvalidMessageException(String message) { + super(message); + } +} diff --git a/src/main/java/com/gotocompany/depot/exception/NonRetryableException.java b/src/main/java/com/gotocompany/depot/exception/NonRetryableException.java new file mode 100644 index 00000000..58947ddb --- /dev/null +++ b/src/main/java/com/gotocompany/depot/exception/NonRetryableException.java @@ -0,0 +1,9 @@ +package com.gotocompany.depot.exception; + +public class NonRetryableException extends RuntimeException { + + public NonRetryableException(String message, Throwable cause) { + super(message, cause); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/MaxComputeSchemaHelper.java b/src/main/java/com/gotocompany/depot/maxcompute/MaxComputeSchemaHelper.java new file mode 100644 index 00000000..0178f98c --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/MaxComputeSchemaHelper.java @@ -0,0 +1,75 @@ +package com.gotocompany.depot.maxcompute; + +import com.aliyun.odps.Column; +import com.aliyun.odps.TableSchema; +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.converter.ProtobufConverterOrchestrator; +import com.gotocompany.depot.maxcompute.model.MaxComputeSchema; +import com.gotocompany.depot.maxcompute.schema.partition.PartitioningStrategy; +import com.gotocompany.depot.maxcompute.util.MetadataUtil; +import lombok.RequiredArgsConstructor; +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +@RequiredArgsConstructor +public class MaxComputeSchemaHelper { + + private final ProtobufConverterOrchestrator protobufConverterOrchestrator; + private final MaxComputeSinkConfig maxComputeSinkConfig; + private final PartitioningStrategy partitioningStrategy; + + public MaxComputeSchema build(Descriptors.Descriptor descriptor) { + List metadataColumns = buildMetadataColumns(); + TableSchema.Builder tableSchemaBuilder = com.aliyun.odps.TableSchema.builder() + .withColumns(metadataColumns) + .withColumns(buildDataColumns(descriptor)); + Column partitionColumn = maxComputeSinkConfig.isTablePartitioningEnabled() ? buildPartitionColumn() : null; + if (Objects.nonNull(partitionColumn)) { + tableSchemaBuilder.withPartitionColumn(partitionColumn); + } + return new MaxComputeSchema( + tableSchemaBuilder.build(), + metadataColumns.stream().collect(Collectors.toMap(Column::getName, Column::getTypeInfo)) + ); + + } + + private List buildDataColumns(Descriptors.Descriptor descriptor) { + return descriptor.getFields() + .stream() + .filter(fieldDescriptor -> { + if (!maxComputeSinkConfig.isTablePartitioningEnabled() || !fieldDescriptor.getName().equals(maxComputeSinkConfig.getTablePartitionKey())) { + return true; + } + return !partitioningStrategy.shouldReplaceOriginalColumn(); + }) + .map(fieldDescriptor -> Column.newBuilder(fieldDescriptor.getName(), + protobufConverterOrchestrator.convert(fieldDescriptor)).build()) + .collect(Collectors.toList()); + } + + private Column buildPartitionColumn() { + return partitioningStrategy.getPartitionColumn(); + } + + private List buildMetadataColumns() { + if (!maxComputeSinkConfig.shouldAddMetadata()) { + return new ArrayList<>(); + } + if (StringUtils.isBlank(maxComputeSinkConfig.getMaxcomputeMetadataNamespace())) { + return maxComputeSinkConfig.getMetadataColumnsTypes() + .stream() + .map(tuple -> Column.newBuilder(tuple.getFirst(), MetadataUtil.getMetadataTypeInfo(tuple.getSecond())).build()) + .collect(Collectors.toList()); + } + return Collections.singletonList(Column.newBuilder(maxComputeSinkConfig.getMaxcomputeMetadataNamespace(), + MetadataUtil.getMetadataTypeInfo(maxComputeSinkConfig.getMetadataColumnsTypes())).build()); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/MaxComputeSink.java b/src/main/java/com/gotocompany/depot/maxcompute/MaxComputeSink.java new file mode 100644 index 00000000..399ea606 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/MaxComputeSink.java @@ -0,0 +1,63 @@ +package com.gotocompany.depot.maxcompute; + +import com.aliyun.odps.tunnel.TunnelException; +import com.gotocompany.depot.Sink; +import com.gotocompany.depot.SinkResponse; +import com.gotocompany.depot.error.ErrorInfo; +import com.gotocompany.depot.error.ErrorType; +import com.gotocompany.depot.exception.SinkException; +import com.gotocompany.depot.maxcompute.client.MaxComputeClient; +import com.gotocompany.depot.maxcompute.converter.record.MessageRecordConverter; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.maxcompute.model.RecordWrappers; +import com.gotocompany.depot.message.Message; +import com.gotocompany.depot.metrics.Instrumentation; +import com.gotocompany.depot.metrics.MaxComputeMetrics; +import lombok.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.List; + +@RequiredArgsConstructor +@Slf4j +public class MaxComputeSink implements Sink { + + private final MaxComputeClient maxComputeClient; + private final MessageRecordConverter messageRecordConverter; + private final Instrumentation instrumentation; + private final MaxComputeMetrics maxComputeMetrics; + + @Override + public SinkResponse pushToSink(List messages) throws SinkException { + SinkResponse sinkResponse = new SinkResponse(); + RecordWrappers recordWrappers = messageRecordConverter.convert(messages); + recordWrappers.getInvalidRecords() + .forEach(invalidRecord -> sinkResponse.getErrors().put(invalidRecord.getIndex(), invalidRecord.getErrorInfo())); + try { + maxComputeClient.insert(recordWrappers.getValidRecords()); + } catch (IOException | TunnelException e) { + log.error("Error while inserting records to MaxCompute: ", e); + mapInsertionError(recordWrappers.getValidRecords(), sinkResponse, new ErrorInfo(e, ErrorType.SINK_RETRYABLE_ERROR)); + instrumentation.incrementCounter(maxComputeMetrics.getMaxComputeOperationTotalMetric(), + String.format(MaxComputeMetrics.MAXCOMPUTE_ERROR_TAG, e.getClass().getSimpleName())); + } catch (Exception e) { + log.error("Error while inserting records to MaxCompute: ", e); + mapInsertionError(recordWrappers.getValidRecords(), sinkResponse, new ErrorInfo(e, ErrorType.DEFAULT_ERROR)); + instrumentation.incrementCounter(maxComputeMetrics.getMaxComputeOperationTotalMetric(), + String.format(MaxComputeMetrics.MAXCOMPUTE_ERROR_TAG, e.getClass().getSimpleName())); + } + return sinkResponse; + } + + @Override + public void close() throws IOException { + } + + private void mapInsertionError(List recordWrapperList, + SinkResponse sinkResponse, + ErrorInfo errorInfo) { + recordWrapperList.forEach(recordWrapper -> sinkResponse.getErrors().put(recordWrapper.getIndex(), errorInfo)); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/MaxComputeSinkFactory.java b/src/main/java/com/gotocompany/depot/maxcompute/MaxComputeSinkFactory.java new file mode 100644 index 00000000..a0bfce19 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/MaxComputeSinkFactory.java @@ -0,0 +1,82 @@ +package com.gotocompany.depot.maxcompute; + +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.Sink; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.config.SinkConfig; +import com.gotocompany.depot.maxcompute.client.MaxComputeClient; +import com.gotocompany.depot.maxcompute.converter.ProtobufConverterOrchestrator; +import com.gotocompany.depot.maxcompute.converter.record.ProtoMessageRecordConverter; +import com.gotocompany.depot.maxcompute.record.RecordDecorator; +import com.gotocompany.depot.maxcompute.record.RecordDecoratorFactory; +import com.gotocompany.depot.maxcompute.schema.MaxComputeSchemaCache; +import com.gotocompany.depot.maxcompute.schema.MaxComputeSchemaCacheFactory; +import com.gotocompany.depot.maxcompute.schema.partition.PartitioningStrategy; +import com.gotocompany.depot.maxcompute.schema.partition.PartitioningStrategyFactory; +import com.gotocompany.depot.message.MessageParser; +import com.gotocompany.depot.message.MessageParserFactory; +import com.gotocompany.depot.message.SinkConnectorSchemaMessageMode; +import com.gotocompany.depot.metrics.Instrumentation; +import com.gotocompany.depot.metrics.MaxComputeMetrics; +import com.gotocompany.depot.metrics.StatsDReporter; +import com.gotocompany.stencil.client.StencilClient; +import org.aeonbits.owner.ConfigFactory; + +import java.util.Map; + +public class MaxComputeSinkFactory { + + private final MaxComputeSinkConfig maxComputeSinkConfig; + private final SinkConfig sinkConfig; + private final StatsDReporter statsDReporter; + private final StencilClient stencilClient; + private final ProtobufConverterOrchestrator protobufConverterOrchestrator; + private final MaxComputeMetrics maxComputeMetrics; + private final MaxComputeClient maxComputeClient; + + private MaxComputeSchemaCache maxComputeSchemaCache; + private PartitioningStrategy partitioningStrategy; + private MessageParser messageParser; + + public MaxComputeSinkFactory(StatsDReporter statsDReporter, + StencilClient stencilClient, + Map env) { + this.statsDReporter = statsDReporter; + this.maxComputeSinkConfig = ConfigFactory.create(MaxComputeSinkConfig.class, env); + this.sinkConfig = ConfigFactory.create(SinkConfig.class, env); + this.stencilClient = stencilClient; + this.protobufConverterOrchestrator = new ProtobufConverterOrchestrator(maxComputeSinkConfig); + this.maxComputeMetrics = new MaxComputeMetrics(sinkConfig); + this.maxComputeClient = new MaxComputeClient(maxComputeSinkConfig, new Instrumentation(statsDReporter, MaxComputeClient.class), maxComputeMetrics); + } + + public void init() { + Descriptors.Descriptor descriptor = stencilClient.get(getProtoSchemaClassName(sinkConfig)); + this.partitioningStrategy = PartitioningStrategyFactory.createPartitioningStrategy(protobufConverterOrchestrator, maxComputeSinkConfig, descriptor); + this.maxComputeSchemaCache = MaxComputeSchemaCacheFactory.createMaxComputeSchemaCache(protobufConverterOrchestrator, + maxComputeSinkConfig, partitioningStrategy, sinkConfig, maxComputeClient); + this.messageParser = MessageParserFactory.getParser(sinkConfig, statsDReporter, maxComputeSchemaCache); + maxComputeSchemaCache.setMessageParser(messageParser); + maxComputeSchemaCache.updateSchema(); + } + + public Sink create() { + RecordDecorator recordDecorator = RecordDecoratorFactory.createRecordDecorator( + protobufConverterOrchestrator, + maxComputeSchemaCache, + messageParser, + partitioningStrategy, + maxComputeSinkConfig, + sinkConfig + ); + ProtoMessageRecordConverter protoMessageRecordConverter = new ProtoMessageRecordConverter(recordDecorator, maxComputeSchemaCache); + return new MaxComputeSink(maxComputeClient, protoMessageRecordConverter, + new Instrumentation(statsDReporter, MaxComputeSink.class), maxComputeMetrics); + } + + private static String getProtoSchemaClassName(SinkConfig sinkConfig) { + return SinkConnectorSchemaMessageMode.LOG_MESSAGE == sinkConfig.getSinkConnectorSchemaMessageMode() + ? sinkConfig.getSinkConnectorSchemaProtoMessageClass() : sinkConfig.getSinkConnectorSchemaProtoKeyClass(); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/client/MaxComputeClient.java b/src/main/java/com/gotocompany/depot/maxcompute/client/MaxComputeClient.java new file mode 100644 index 00000000..1b907c1c --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/client/MaxComputeClient.java @@ -0,0 +1,83 @@ +package com.gotocompany.depot.maxcompute.client; + +import com.aliyun.odps.Odps; +import com.aliyun.odps.OdpsException; +import com.aliyun.odps.TableSchema; +import com.aliyun.odps.account.Account; +import com.aliyun.odps.account.AliyunAccount; +import com.aliyun.odps.tunnel.TableTunnel; +import com.aliyun.odps.tunnel.TunnelException; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.client.ddl.DdlManager; +import com.gotocompany.depot.maxcompute.client.insert.InsertManager; +import com.gotocompany.depot.maxcompute.client.insert.InsertManagerFactory; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.metrics.Instrumentation; +import com.gotocompany.depot.metrics.MaxComputeMetrics; +import lombok.AllArgsConstructor; +import lombok.NoArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.List; + +@AllArgsConstructor +@NoArgsConstructor +@Slf4j +public class MaxComputeClient { + + private Odps odps; + private MaxComputeSinkConfig maxComputeSinkConfig; + private TableTunnel tableTunnel; + private InsertManager insertManager; + private DdlManager ddlManager; + private MaxComputeMetrics maxComputeMetrics; + private Instrumentation instrumentation; + + public MaxComputeClient(MaxComputeSinkConfig maxComputeSinkConfig, + Instrumentation instrumentation, MaxComputeMetrics maxComputeMetrics) { + this.maxComputeSinkConfig = maxComputeSinkConfig; + this.instrumentation = instrumentation; + this.odps = initializeOdps(); + this.tableTunnel = new TableTunnel(odps); + this.tableTunnel.setEndpoint(maxComputeSinkConfig.getMaxComputeTunnelUrl()); + this.maxComputeMetrics = maxComputeMetrics; + this.insertManager = initializeInsertManager(); + this.ddlManager = initializeDdlManager(); + } + + public TableSchema getLatestTableSchema() { + return odps.tables() + .get(maxComputeSinkConfig.getMaxComputeProjectId(), + maxComputeSinkConfig.getMaxComputeSchema(), + maxComputeSinkConfig.getMaxComputeTableName()) + .getSchema(); + } + + public void createOrUpdateTable(TableSchema tableSchema) throws OdpsException { + ddlManager.createOrUpdateTable(tableSchema); + } + + public void insert(List recordWrappers) throws TunnelException, IOException { + insertManager.insert(recordWrappers); + } + + private Odps initializeOdps() { + Account account = new AliyunAccount(maxComputeSinkConfig.getMaxComputeAccessId(), maxComputeSinkConfig.getMaxComputeAccessKey()); + Odps odpsClient = new Odps(account); + odpsClient.setDefaultProject(maxComputeSinkConfig.getMaxComputeProjectId()); + odpsClient.setEndpoint(maxComputeSinkConfig.getMaxComputeOdpsUrl()); + odpsClient.setCurrentSchema(maxComputeSinkConfig.getMaxComputeSchema()); + odpsClient.setGlobalSettings(maxComputeSinkConfig.getOdpsGlobalSettings()); + return odpsClient; + } + + private InsertManager initializeInsertManager() { + return InsertManagerFactory.createInsertManager(maxComputeSinkConfig, tableTunnel, instrumentation, maxComputeMetrics); + } + + private DdlManager initializeDdlManager() { + return new DdlManager(odps, maxComputeSinkConfig, instrumentation, maxComputeMetrics); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/client/ddl/DdlManager.java b/src/main/java/com/gotocompany/depot/maxcompute/client/ddl/DdlManager.java new file mode 100644 index 00000000..4df0daa4 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/client/ddl/DdlManager.java @@ -0,0 +1,128 @@ +package com.gotocompany.depot.maxcompute.client.ddl; + +import com.aliyun.odps.Column; +import com.aliyun.odps.Instance; +import com.aliyun.odps.Odps; +import com.aliyun.odps.OdpsException; +import com.aliyun.odps.TableSchema; +import com.aliyun.odps.task.SQLTask; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.exception.MaxComputeTableOperationException; +import com.gotocompany.depot.maxcompute.schema.SchemaDifferenceUtils; +import com.gotocompany.depot.maxcompute.schema.validator.TableValidator; +import com.gotocompany.depot.metrics.Instrumentation; +import com.gotocompany.depot.metrics.MaxComputeMetrics; +import com.gotocompany.depot.utils.RetryUtils; +import lombok.extern.slf4j.Slf4j; + +import java.time.Instant; +import java.util.Deque; +import java.util.LinkedList; +import java.util.Objects; + +@Slf4j +public class DdlManager { + + private final Odps odps; + private final MaxComputeSinkConfig maxComputeSinkConfig; + private final Instrumentation instrumentation; + private final MaxComputeMetrics maxComputeMetrics; + private final TableValidator tableValidator; + + public DdlManager(Odps odps, MaxComputeSinkConfig maxComputeSinkConfig, Instrumentation instrumentation, MaxComputeMetrics maxComputeMetrics) { + this.odps = odps; + this.maxComputeSinkConfig = maxComputeSinkConfig; + this.instrumentation = instrumentation; + this.maxComputeMetrics = maxComputeMetrics; + this.tableValidator = new TableValidator(maxComputeSinkConfig); + } + + public void createOrUpdateTable(TableSchema tableSchema) throws OdpsException { + String projectName = maxComputeSinkConfig.getMaxComputeProjectId(); + String datasetName = maxComputeSinkConfig.getMaxComputeSchema(); + String tableName = maxComputeSinkConfig.getMaxComputeTableName(); + if (!this.odps.tables().exists(tableName)) { + createTable(tableSchema, projectName, datasetName, tableName); + return; + } + updateTable(tableSchema, projectName, datasetName, tableName); + } + + private void createTable(TableSchema tableSchema, String projectName, String datasetName, String tableName) { + log.info("Creating table: {} schema:{}", tableName, tableSchema); + tableValidator.validate(tableName, maxComputeSinkConfig.getMaxComputeTableLifecycleDays(), tableSchema); + RetryUtils.executeWithRetry(() -> { + Instant start = Instant.now(); + this.odps.tables().create(projectName, datasetName, tableName, tableSchema, "", + true, maxComputeSinkConfig.getMaxComputeTableLifecycleDays(), + null, null); + instrumentation.logInfo("Successfully created maxCompute table " + tableName); + instrument(start, MaxComputeMetrics.MaxComputeAPIType.TABLE_CREATE); + }, maxComputeSinkConfig.getMaxDdlRetryCount(), maxComputeSinkConfig.getDdlRetryBackoffMillis(), + e -> e instanceof OdpsException); + } + + private void updateTable(TableSchema tableSchema, String projectName, String datasetName, String tableName) { + log.info("Updating table: {} schema:{}", tableName, tableSchema); + Instant start = Instant.now(); + TableSchema oldSchema = this.odps.tables().get(projectName, datasetName, tableName) + .getSchema(); + checkPartitionPrecondition(oldSchema); + Deque schemaDifferenceSql = new LinkedList<>(SchemaDifferenceUtils.getSchemaDifferenceSql(oldSchema, tableSchema, datasetName, tableName)); + RetryUtils.executeWithRetry(() -> { + while (!schemaDifferenceSql.isEmpty()) { + String sql = schemaDifferenceSql.peekFirst(); + Instance instance = execute(sql); + if (!instance.isSuccessful()) { + instrumentation.logError("Failed to execute SQL: " + sql); + String errorMessage = instance.getRawTaskResults().get(0).getResult().getString(); + throw new MaxComputeTableOperationException(String.format("Failed to update table schema with reason: %s", errorMessage)); + } + schemaDifferenceSql.pollFirst(); + } + instrumentation.logInfo("Successfully updated maxCompute table " + tableName); + instrument(start, MaxComputeMetrics.MaxComputeAPIType.TABLE_UPDATE); + }, maxComputeSinkConfig.getMaxDdlRetryCount(), maxComputeSinkConfig.getDdlRetryBackoffMillis(), + e -> e instanceof OdpsException); + } + + private void checkPartitionPrecondition(TableSchema oldSchema) { + if (maxComputeSinkConfig.isTablePartitioningEnabled() && oldSchema.getPartitionColumns().isEmpty()) { + throw new MaxComputeTableOperationException("Updating non-partitioned table to partitioned table is not supported"); + } + if (maxComputeSinkConfig.isTablePartitioningEnabled()) { + String currentPartitionColumnKey = oldSchema.getPartitionColumns() + .stream() + .findFirst() + .map(Column::getName) + .orElse(null); + if (!Objects.equals(maxComputeSinkConfig.getTablePartitionColumnName(), currentPartitionColumnKey)) { + throw new MaxComputeTableOperationException("Changing partition column is not supported"); + } + } + } + + private Instance execute(String sql) throws OdpsException { + log.info("Executing SQL: {}", sql); + Instance instance = SQLTask.run(odps, sql); + instance.waitForSuccess(); + return instance; + } + + private void instrument(Instant startTime, MaxComputeMetrics.MaxComputeAPIType type) { + instrumentation.incrementCounter( + maxComputeMetrics.getMaxComputeOperationTotalMetric(), + String.format(MaxComputeMetrics.MAXCOMPUTE_TABLE_TAG, maxComputeSinkConfig.getMaxComputeTableName()), + String.format(MaxComputeMetrics.MAXCOMPUTE_PROJECT_TAG, maxComputeSinkConfig.getMaxComputeProjectId()), + String.format(MaxComputeMetrics.MAXCOMPUTE_API_TAG, type) + ); + instrumentation.captureDurationSince( + maxComputeMetrics.getMaxComputeOperationLatencyMetric(), + startTime, + String.format(MaxComputeMetrics.MAXCOMPUTE_TABLE_TAG, maxComputeSinkConfig.getMaxComputeTableName()), + String.format(MaxComputeMetrics.MAXCOMPUTE_PROJECT_TAG, maxComputeSinkConfig.getMaxComputeProjectId()), + String.format(MaxComputeMetrics.MAXCOMPUTE_API_TAG, type) + ); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/client/insert/InsertManager.java b/src/main/java/com/gotocompany/depot/maxcompute/client/insert/InsertManager.java new file mode 100644 index 00000000..bdb0acd9 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/client/insert/InsertManager.java @@ -0,0 +1,47 @@ +package com.gotocompany.depot.maxcompute.client.insert; + +import com.aliyun.odps.tunnel.TableTunnel; +import com.aliyun.odps.tunnel.TunnelException; +import com.aliyun.odps.tunnel.io.CompressOption; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.metrics.Instrumentation; +import com.gotocompany.depot.metrics.MaxComputeMetrics; +import lombok.Getter; +import lombok.RequiredArgsConstructor; + +import java.io.IOException; +import java.time.Instant; +import java.util.List; + +@RequiredArgsConstructor +@Getter +public abstract class InsertManager { + + private final MaxComputeSinkConfig maxComputeSinkConfig; + private final Instrumentation instrumentation; + private final MaxComputeMetrics maxComputeMetrics; + + public abstract void insert(List recordWrappers) throws TunnelException, IOException; + + protected TableTunnel.StreamRecordPack newRecordPack(TableTunnel.StreamUploadSession streamUploadSession) throws IOException, TunnelException { + if (!maxComputeSinkConfig.isStreamingInsertCompressEnabled()) { + return streamUploadSession.newRecordPack(); + } + return streamUploadSession.newRecordPack(new CompressOption(maxComputeSinkConfig.getMaxComputeCompressionAlgorithm(), + maxComputeSinkConfig.getMaxComputeCompressionLevel(), + maxComputeSinkConfig.getMaxComputeCompressionStrategy())); + } + + protected void instrument(Instant start, TableTunnel.FlushResult flushResult) { + instrumentation.incrementCounter(maxComputeMetrics.getMaxComputeOperationTotalMetric(), + String.format(MaxComputeMetrics.MAXCOMPUTE_API_TAG, MaxComputeMetrics.MaxComputeAPIType.TABLE_INSERT)); + instrumentation.captureDurationSince(maxComputeMetrics.getMaxComputeOperationLatencyMetric(), start, + String.format(MaxComputeMetrics.MAXCOMPUTE_API_TAG, MaxComputeMetrics.MaxComputeAPIType.TABLE_INSERT)); + instrumentation.captureCount(maxComputeMetrics.getMaxComputeFlushRecordMetric(), flushResult.getRecordCount(), + String.format(MaxComputeMetrics.MAXCOMPUTE_COMPRESSION_TAG, maxComputeSinkConfig.isStreamingInsertCompressEnabled(), maxComputeSinkConfig.getMaxComputeCompressionAlgorithm())); + instrumentation.captureCount(maxComputeMetrics.getMaxComputeFlushSizeMetric(), flushResult.getFlushSize(), + String.format(MaxComputeMetrics.MAXCOMPUTE_COMPRESSION_TAG, maxComputeSinkConfig.isStreamingInsertCompressEnabled(), maxComputeSinkConfig.getMaxComputeCompressionAlgorithm())); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/client/insert/InsertManagerFactory.java b/src/main/java/com/gotocompany/depot/maxcompute/client/insert/InsertManagerFactory.java new file mode 100644 index 00000000..7ef49570 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/client/insert/InsertManagerFactory.java @@ -0,0 +1,24 @@ +package com.gotocompany.depot.maxcompute.client.insert; + +import com.aliyun.odps.tunnel.TableTunnel; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.client.insert.session.StreamingSessionManager; +import com.gotocompany.depot.metrics.Instrumentation; +import com.gotocompany.depot.metrics.MaxComputeMetrics; + +public class InsertManagerFactory { + + public static InsertManager createInsertManager(MaxComputeSinkConfig maxComputeSinkConfig, + TableTunnel tableTunnel, + Instrumentation instrumentation, + MaxComputeMetrics maxComputeMetrics) { + StreamingSessionManager streamingSessionManager = maxComputeSinkConfig.isTablePartitioningEnabled() + ? StreamingSessionManager.createPartitioned(tableTunnel, maxComputeSinkConfig) : StreamingSessionManager.createNonPartitioned(tableTunnel, maxComputeSinkConfig); + if (maxComputeSinkConfig.isTablePartitioningEnabled()) { + return new PartitionedInsertManager(maxComputeSinkConfig, instrumentation, maxComputeMetrics, streamingSessionManager); + } else { + return new NonPartitionedInsertManager(maxComputeSinkConfig, instrumentation, maxComputeMetrics, streamingSessionManager); + } + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/client/insert/NonPartitionedInsertManager.java b/src/main/java/com/gotocompany/depot/maxcompute/client/insert/NonPartitionedInsertManager.java new file mode 100644 index 00000000..a72d26bd --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/client/insert/NonPartitionedInsertManager.java @@ -0,0 +1,51 @@ +package com.gotocompany.depot.maxcompute.client.insert; + +import com.aliyun.odps.tunnel.TableTunnel; +import com.aliyun.odps.tunnel.TunnelException; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.client.insert.session.StreamingSessionManager; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.metrics.Instrumentation; +import com.gotocompany.depot.metrics.MaxComputeMetrics; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.time.Instant; +import java.util.List; + +@Slf4j +public class NonPartitionedInsertManager extends InsertManager { + + private static final String NON_PARTITIONED = "non-partitioned"; + private final StreamingSessionManager streamingSessionManager; + private final TableTunnel.FlushOption flushOption; + + public NonPartitionedInsertManager(MaxComputeSinkConfig maxComputeSinkConfig, + Instrumentation instrumentation, + MaxComputeMetrics maxComputeMetrics, + StreamingSessionManager streamingSessionManager) { + super(maxComputeSinkConfig, instrumentation, maxComputeMetrics); + this.streamingSessionManager = streamingSessionManager; + this.flushOption = new TableTunnel.FlushOption() + .timeout(super.getMaxComputeSinkConfig().getMaxComputeRecordPackFlushTimeoutMs()); + } + + @Override + public void insert(List recordWrappers) throws TunnelException, IOException { + TableTunnel.StreamUploadSession streamUploadSession = streamingSessionManager.getSession(NON_PARTITIONED); + TableTunnel.StreamRecordPack recordPack = newRecordPack(streamUploadSession); + for (RecordWrapper recordWrapper : recordWrappers) { + try { + recordPack.append(recordWrapper.getRecord()); + } catch (IOException e) { + log.error("Schema Mismatch, rebuilding the session", e); + streamingSessionManager.refreshSession(NON_PARTITIONED); + throw e; + } + } + Instant start = Instant.now(); + TableTunnel.FlushResult flushResult = recordPack.flush(flushOption); + instrument(start, flushResult); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/client/insert/PartitionedInsertManager.java b/src/main/java/com/gotocompany/depot/maxcompute/client/insert/PartitionedInsertManager.java new file mode 100644 index 00000000..928d428c --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/client/insert/PartitionedInsertManager.java @@ -0,0 +1,56 @@ +package com.gotocompany.depot.maxcompute.client.insert; + +import com.aliyun.odps.tunnel.TableTunnel; +import com.aliyun.odps.tunnel.TunnelException; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.client.insert.session.StreamingSessionManager; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.metrics.Instrumentation; +import com.gotocompany.depot.metrics.MaxComputeMetrics; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.time.Instant; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +@Slf4j +public class PartitionedInsertManager extends InsertManager { + + private final StreamingSessionManager streamingSessionManager; + private final TableTunnel.FlushOption flushOption; + + public PartitionedInsertManager(MaxComputeSinkConfig maxComputeSinkConfig, + Instrumentation instrumentation, + MaxComputeMetrics maxComputeMetrics, + StreamingSessionManager streamingSessionManager) { + super(maxComputeSinkConfig, instrumentation, maxComputeMetrics); + this.streamingSessionManager = streamingSessionManager; + this.flushOption = new TableTunnel.FlushOption() + .timeout(super.getMaxComputeSinkConfig().getMaxComputeRecordPackFlushTimeoutMs()); + } + + @Override + public void insert(List recordWrappers) throws TunnelException, IOException { + Map> partitionSpecRecordWrapperMap = recordWrappers.stream() + .collect(Collectors.groupingBy(record -> record.getPartitionSpec().toString())); + for (Map.Entry> entry : partitionSpecRecordWrapperMap.entrySet()) { + TableTunnel.StreamUploadSession streamUploadSession = streamingSessionManager.getSession(entry.getKey()); + TableTunnel.StreamRecordPack recordPack = newRecordPack(streamUploadSession); + for (RecordWrapper recordWrapper : entry.getValue()) { + try { + recordPack.append(recordWrapper.getRecord()); + } catch (IOException e) { + log.error("Schema Mismatch, clearing the session", e); + streamingSessionManager.refreshSession(recordWrapper.getPartitionSpec().toString()); + throw e; + } + } + Instant start = Instant.now(); + TableTunnel.FlushResult flushResult = recordPack.flush(flushOption); + instrument(start, flushResult); + } + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/client/insert/session/StreamingSessionManager.java b/src/main/java/com/gotocompany/depot/maxcompute/client/insert/session/StreamingSessionManager.java new file mode 100644 index 00000000..8ef413ca --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/client/insert/session/StreamingSessionManager.java @@ -0,0 +1,61 @@ +package com.gotocompany.depot.maxcompute.client.insert.session; + +import com.aliyun.odps.tunnel.TableTunnel; + +import com.aliyun.odps.tunnel.TunnelException; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.gotocompany.depot.config.MaxComputeSinkConfig; + +public final class StreamingSessionManager { + + private final LoadingCache sessionCache; + + private StreamingSessionManager(LoadingCache loadingCache) { + sessionCache = loadingCache; + } + + public static StreamingSessionManager createNonPartitioned(TableTunnel tableTunnel, MaxComputeSinkConfig maxComputeSinkConfig) { + CacheLoader cacheLoader = new CacheLoader() { + @Override + public TableTunnel.StreamUploadSession load(String sessionId) throws TunnelException { + return tableTunnel.buildStreamUploadSession( + maxComputeSinkConfig.getMaxComputeProjectId(), + maxComputeSinkConfig.getMaxComputeTableName()) + .allowSchemaMismatch(false) + .build(); + } + }; + return new StreamingSessionManager(CacheBuilder.newBuilder() + .maximumSize(maxComputeSinkConfig.getStreamingInsertMaximumSessionCount()) + .build(cacheLoader)); + } + + public static StreamingSessionManager createPartitioned(TableTunnel tableTunnel, MaxComputeSinkConfig maxComputeSinkConfig) { + CacheLoader cacheLoader = new CacheLoader() { + @Override + public TableTunnel.StreamUploadSession load(String partitionSpecKey) throws TunnelException { + return tableTunnel.buildStreamUploadSession( + maxComputeSinkConfig.getMaxComputeProjectId(), + maxComputeSinkConfig.getMaxComputeTableName()) + .setCreatePartition(true) + .setPartitionSpec(partitionSpecKey) + .allowSchemaMismatch(false) + .build(); + } + }; + return new StreamingSessionManager(CacheBuilder.newBuilder() + .maximumSize(maxComputeSinkConfig.getStreamingInsertMaximumSessionCount()) + .build(cacheLoader)); + } + + public TableTunnel.StreamUploadSession getSession(String sessionId) { + return sessionCache.getUnchecked(sessionId); + } + + public void refreshSession(String sessionId) { + sessionCache.refresh(sessionId); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/ProtobufConverterOrchestrator.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/ProtobufConverterOrchestrator.java new file mode 100644 index 00000000..23afacd9 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/ProtobufConverterOrchestrator.java @@ -0,0 +1,78 @@ +package com.gotocompany.depot.maxcompute.converter; + +import com.aliyun.odps.type.TypeInfo; +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.converter.payload.DurationProtobufPayloadConverter; +import com.gotocompany.depot.maxcompute.converter.payload.MessageProtobufPayloadConverter; +import com.gotocompany.depot.maxcompute.converter.payload.ProtobufPayloadConverter; +import com.gotocompany.depot.maxcompute.converter.payload.PrimitiveProtobufPayloadConverter; +import com.gotocompany.depot.maxcompute.converter.payload.StructProtobufPayloadConverter; +import com.gotocompany.depot.maxcompute.converter.payload.TimestampProtobufPayloadConverter; +import com.gotocompany.depot.maxcompute.converter.type.DurationProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.converter.type.MessageProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.converter.type.PrimitiveProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.converter.type.StructProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.converter.type.TimestampProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.converter.type.ProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.model.ProtoPayload; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class ProtobufConverterOrchestrator { + + private final List protobufTypeInfoConverters; + private final List protobufPayloadConverters; + private final Map typeInfoCache; + + public ProtobufConverterOrchestrator(MaxComputeSinkConfig maxComputeSinkConfig) { + protobufTypeInfoConverters = new ArrayList<>(); + protobufPayloadConverters = new ArrayList<>(); + typeInfoCache = new ConcurrentHashMap<>(); + initializeConverters(maxComputeSinkConfig); + } + + public TypeInfo convert(Descriptors.FieldDescriptor fieldDescriptor) { + return typeInfoCache.computeIfAbsent(fieldDescriptor.getFullName(), key -> protobufTypeInfoConverters.stream() + .filter(converter -> converter.canConvert(fieldDescriptor)) + .findFirst() + .map(converter -> converter.convert(fieldDescriptor)) + .orElseThrow(() -> new IllegalArgumentException("Unsupported type: " + fieldDescriptor.getType()))); + } + + public Object convert(Descriptors.FieldDescriptor fieldDescriptor, Object object) { + return protobufPayloadConverters.stream() + .filter(converter -> converter.canConvert(fieldDescriptor)) + .findFirst() + .map(converter -> converter.convert(new ProtoPayload(fieldDescriptor, object, true))) + .orElseThrow(() -> new IllegalArgumentException("Unsupported type: " + fieldDescriptor.getType())); + } + + public void clearCache() { + typeInfoCache.clear(); + } + + private void initializeConverters(MaxComputeSinkConfig maxComputeSinkConfig) { + PrimitiveProtobufTypeInfoConverter primitiveTypeInfoConverter = new PrimitiveProtobufTypeInfoConverter(); + DurationProtobufTypeInfoConverter durationTypeInfoConverter = new DurationProtobufTypeInfoConverter(); + StructProtobufTypeInfoConverter structTypeInfoConverter = new StructProtobufTypeInfoConverter(); + TimestampProtobufTypeInfoConverter timestampTypeInfoConverter = new TimestampProtobufTypeInfoConverter(); + MessageProtobufTypeInfoConverter messageTypeInfoConverter = new MessageProtobufTypeInfoConverter(protobufTypeInfoConverters); + + protobufTypeInfoConverters.add(primitiveTypeInfoConverter); + protobufTypeInfoConverters.add(durationTypeInfoConverter); + protobufTypeInfoConverters.add(structTypeInfoConverter); + protobufTypeInfoConverters.add(timestampTypeInfoConverter); + protobufTypeInfoConverters.add(messageTypeInfoConverter); + + protobufPayloadConverters.add(new PrimitiveProtobufPayloadConverter(primitiveTypeInfoConverter)); + protobufPayloadConverters.add(new DurationProtobufPayloadConverter(durationTypeInfoConverter)); + protobufPayloadConverters.add(new StructProtobufPayloadConverter(structTypeInfoConverter)); + protobufPayloadConverters.add(new TimestampProtobufPayloadConverter(timestampTypeInfoConverter, maxComputeSinkConfig)); + protobufPayloadConverters.add(new MessageProtobufPayloadConverter(messageTypeInfoConverter, protobufPayloadConverters)); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/DurationProtobufPayloadConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/DurationProtobufPayloadConverter.java new file mode 100644 index 00000000..35880e05 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/DurationProtobufPayloadConverter.java @@ -0,0 +1,40 @@ +package com.gotocompany.depot.maxcompute.converter.payload; + +import com.aliyun.odps.data.SimpleStruct; +import com.aliyun.odps.type.StructTypeInfo; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.gotocompany.depot.maxcompute.converter.type.DurationProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.model.ProtoPayload; +import lombok.RequiredArgsConstructor; + +import java.util.ArrayList; +import java.util.List; + +@RequiredArgsConstructor +public class DurationProtobufPayloadConverter implements ProtobufPayloadConverter { + + private static final String SECONDS = "seconds"; + private static final String NANOS = "nanos"; + + private final DurationProtobufTypeInfoConverter durationTypeInfoConverter; + + @Override + public Object convertSingular(ProtoPayload protoPayload) { + List values = getValues((Message) protoPayload.getObject()); + return new SimpleStruct((StructTypeInfo) durationTypeInfoConverter.convertSingular(protoPayload.getFieldDescriptor()), values); + } + + @Override + public boolean canConvert(Descriptors.FieldDescriptor fieldDescriptor) { + return durationTypeInfoConverter.canConvert(fieldDescriptor); + } + + private static List getValues(Message durationMessage) { + List values = new ArrayList<>(); + values.add(durationMessage.getField(durationMessage.getDescriptorForType().findFieldByName(SECONDS))); + values.add(durationMessage.getField(durationMessage.getDescriptorForType().findFieldByName(NANOS))); + return values; + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/MessageProtobufPayloadConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/MessageProtobufPayloadConverter.java new file mode 100644 index 00000000..69219a2d --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/MessageProtobufPayloadConverter.java @@ -0,0 +1,45 @@ +package com.gotocompany.depot.maxcompute.converter.payload; + +import com.aliyun.odps.data.SimpleStruct; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.gotocompany.depot.maxcompute.converter.type.MessageProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.model.ProtoPayload; +import lombok.RequiredArgsConstructor; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +@RequiredArgsConstructor +public class MessageProtobufPayloadConverter implements ProtobufPayloadConverter { + + private final MessageProtobufTypeInfoConverter messageTypeInfoConverter; + private final List protobufPayloadConverters; + + @Override + public Object convertSingular(ProtoPayload protoPayload) { + Message dynamicMessage = (Message) protoPayload.getObject(); + List values = new ArrayList<>(); + Map payloadFields = dynamicMessage.getAllFields(); + protoPayload.getFieldDescriptor().getMessageType().getFields().forEach(innerFieldDescriptor -> { + if (!payloadFields.containsKey(innerFieldDescriptor)) { + values.add(null); + return; + } + Object mappedInnerValue = protobufPayloadConverters.stream() + .filter(converter -> converter.canConvert(innerFieldDescriptor)) + .findFirst() + .map(converter -> converter.convert(new ProtoPayload(innerFieldDescriptor, payloadFields.get(innerFieldDescriptor), false))) + .orElse(null); + values.add(mappedInnerValue); + }); + return new SimpleStruct(messageTypeInfoConverter.convertSingular(protoPayload.getFieldDescriptor()), values); + } + + @Override + public boolean canConvert(Descriptors.FieldDescriptor fieldDescriptor) { + return messageTypeInfoConverter.canConvert(fieldDescriptor); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/PrimitiveProtobufPayloadConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/PrimitiveProtobufPayloadConverter.java new file mode 100644 index 00000000..3f573aad --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/PrimitiveProtobufPayloadConverter.java @@ -0,0 +1,57 @@ +package com.gotocompany.depot.maxcompute.converter.payload; + +import com.aliyun.odps.data.Binary; +import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.exception.InvalidMessageException; +import com.gotocompany.depot.maxcompute.converter.type.PrimitiveProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.model.ProtoPayload; + +import java.util.HashMap; +import java.util.Map; +import java.util.function.Function; + +public class PrimitiveProtobufPayloadConverter implements ProtobufPayloadConverter { + + private final Map> mappers; + + private final PrimitiveProtobufTypeInfoConverter primitiveTypeInfoConverter; + + public PrimitiveProtobufPayloadConverter(PrimitiveProtobufTypeInfoConverter primitiveTypeInfoConverter) { + this.primitiveTypeInfoConverter = primitiveTypeInfoConverter; + this.mappers = new HashMap<>(); + this.mappers.put(Descriptors.FieldDescriptor.Type.BYTES, object -> handleBytes((ByteString) object)); + this.mappers.put(Descriptors.FieldDescriptor.Type.ENUM, Object::toString); + this.mappers.put(Descriptors.FieldDescriptor.Type.FLOAT, object -> handleFloat((float) object)); + this.mappers.put(Descriptors.FieldDescriptor.Type.DOUBLE, object -> handleDouble((double) object)); + } + + @Override + public Object convertSingular(ProtoPayload protoPayload) { + return mappers.getOrDefault(protoPayload.getFieldDescriptor().getType(), Function.identity()).apply(protoPayload.getObject()); + } + + @Override + public boolean canConvert(Descriptors.FieldDescriptor fieldDescriptor) { + return primitiveTypeInfoConverter.canConvert(fieldDescriptor); + } + + private static double handleDouble(double value) { + if (!Double.isFinite(value)) { + throw new InvalidMessageException("Invalid float value: " + value); + } + return value; + } + + private static float handleFloat(float value) { + if (!Float.isFinite(value)) { + throw new InvalidMessageException("Invalid float value: " + value); + } + return value; + } + + private static Binary handleBytes(ByteString object) { + return new Binary(object.toByteArray()); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/ProtobufPayloadConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/ProtobufPayloadConverter.java new file mode 100644 index 00000000..cd830862 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/ProtobufPayloadConverter.java @@ -0,0 +1,42 @@ +package com.gotocompany.depot.maxcompute.converter.payload; + +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.maxcompute.model.ProtoPayload; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * Interface for converting payloads based on field descriptors. + */ +public interface ProtobufPayloadConverter { + + /** + * Converts a proto payload to a format that can be used by the MaxCompute SDK. + * @param protoPayload the proto payload to convert, containing field descriptor, the actual object and level + * @return + */ + default Object convert(ProtoPayload protoPayload) { + if (!protoPayload.getFieldDescriptor().isRepeated()) { + return convertSingular(protoPayload); + } + return ((List) protoPayload.getObject()).stream() + .map(o -> convertSingular(new ProtoPayload(protoPayload.getFieldDescriptor(), o, protoPayload.isRootLevel()))) + .collect(Collectors.toList()); + } + + /** + * Converts a singular proto payload to a format that can be used by the MaxCompute SDK. + * @param protoPayload the proto payload to convert, containing field descriptor, the actual object and level + * @return + */ + Object convertSingular(ProtoPayload protoPayload); + + /** + * Checks if the converter can convert the given field descriptor. + * + * @param fieldDescriptor the field descriptor + * @return true if the converter can convert the field descriptor, false otherwise + */ + boolean canConvert(Descriptors.FieldDescriptor fieldDescriptor); +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/StructProtobufPayloadConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/StructProtobufPayloadConverter.java new file mode 100644 index 00000000..3709ac86 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/StructProtobufPayloadConverter.java @@ -0,0 +1,33 @@ +package com.gotocompany.depot.maxcompute.converter.payload; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.Message; +import com.google.protobuf.util.JsonFormat; +import com.gotocompany.depot.maxcompute.converter.type.StructProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.model.ProtoPayload; +import lombok.RequiredArgsConstructor; + +@RequiredArgsConstructor +public class StructProtobufPayloadConverter implements ProtobufPayloadConverter { + + private final StructProtobufTypeInfoConverter structTypeInfoConverter; + private final JsonFormat.Printer printer = JsonFormat.printer() + .preservingProtoFieldNames() + .omittingInsignificantWhitespace(); + + @Override + public Object convertSingular(ProtoPayload protoPayload) { + try { + return printer.print((Message) protoPayload.getObject()); + } catch (InvalidProtocolBufferException e) { + return ""; + } + } + + @Override + public boolean canConvert(Descriptors.FieldDescriptor fieldDescriptor) { + return structTypeInfoConverter.canConvert(fieldDescriptor); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/TimestampProtobufPayloadConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/TimestampProtobufPayloadConverter.java new file mode 100644 index 00000000..14899095 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/payload/TimestampProtobufPayloadConverter.java @@ -0,0 +1,79 @@ +package com.gotocompany.depot.maxcompute.converter.payload; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.exception.InvalidMessageException; +import com.gotocompany.depot.maxcompute.converter.type.TimestampProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.model.ProtoPayload; + +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.temporal.TemporalAmount; + +public class TimestampProtobufPayloadConverter implements ProtobufPayloadConverter { + + private static final String SECONDS = "seconds"; + private static final String NANOS = "nanos"; + private static final long DAYS_IN_YEAR = 365; + + private final TimestampProtobufTypeInfoConverter timestampTypeInfoConverter; + private final MaxComputeSinkConfig maxComputeSinkConfig; + private final TemporalAmount maxPastEventTimeDifference; + private final TemporalAmount maxFutureEventTimeDifference; + + public TimestampProtobufPayloadConverter(TimestampProtobufTypeInfoConverter timestampTypeInfoConverter, MaxComputeSinkConfig maxComputeSinkConfig) { + this.timestampTypeInfoConverter = timestampTypeInfoConverter; + this.maxComputeSinkConfig = maxComputeSinkConfig; + this.maxPastEventTimeDifference = Duration.ofDays(maxComputeSinkConfig.getMaxPastYearEventTimeDifference() * DAYS_IN_YEAR); + this.maxFutureEventTimeDifference = Duration.ofDays(maxComputeSinkConfig.getMaxFutureYearEventTimeDifference() * DAYS_IN_YEAR); + } + + @Override + public Object convertSingular(ProtoPayload protoPayload) { + Message message = (Message) protoPayload.getObject(); + long seconds = (long) message.getField(message.getDescriptorForType().findFieldByName(SECONDS)); + int nanos = (int) message.getField(message.getDescriptorForType().findFieldByName(NANOS)); + Instant instant = Instant.now(); + ZoneOffset zoneOffset = maxComputeSinkConfig.getZoneId().getRules().getOffset(instant); + LocalDateTime localDateTime = LocalDateTime.ofEpochSecond(seconds, nanos, zoneOffset); + validateTimestampRange(localDateTime); + validateTimestampPartitionKey(protoPayload.getFieldDescriptor().getName(), localDateTime, protoPayload.isRootLevel()); + return localDateTime; + } + + @Override + public boolean canConvert(Descriptors.FieldDescriptor fieldDescriptor) { + return timestampTypeInfoConverter.canConvert(fieldDescriptor); + } + + private void validateTimestampRange(LocalDateTime localDateTime) { + if (localDateTime.isBefore(maxComputeSinkConfig.getValidMinTimestamp()) || localDateTime.isAfter(maxComputeSinkConfig.getValidMaxTimestamp())) { + throw new InvalidMessageException(String.format("Timestamp %s is out of allowed range range min: %s max: %s", + localDateTime, maxComputeSinkConfig.getValidMinTimestamp(), maxComputeSinkConfig.getValidMaxTimestamp())); + } + } + + private void validateTimestampPartitionKey(String fieldName, LocalDateTime eventTime, boolean isRootLevel) { + if (!maxComputeSinkConfig.isTablePartitioningEnabled()) { + return; + } + if (!isRootLevel) { + return; + } + if (fieldName.equals(maxComputeSinkConfig.getTablePartitionKey())) { + Instant now = Instant.now(); + Instant eventTimeInstant = eventTime.toInstant(maxComputeSinkConfig.getZoneId().getRules().getOffset(now)); + + if (now.minus(maxPastEventTimeDifference).isAfter(eventTimeInstant)) { + throw new InvalidMessageException(String.format("Timestamp is in the past, you can only stream data within %d year(s) in the past", maxComputeSinkConfig.getMaxPastYearEventTimeDifference())); + } + if (now.plus(maxFutureEventTimeDifference).isBefore(eventTimeInstant)) { + throw new InvalidMessageException(String.format("Timestamp is in the future, you can only stream data within %d year(s) in the future", maxComputeSinkConfig.getMaxFutureYearEventTimeDifference())); + } + } + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/record/MessageRecordConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/record/MessageRecordConverter.java new file mode 100644 index 00000000..0d4d7a88 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/record/MessageRecordConverter.java @@ -0,0 +1,10 @@ +package com.gotocompany.depot.maxcompute.converter.record; + +import com.gotocompany.depot.maxcompute.model.RecordWrappers; +import com.gotocompany.depot.message.Message; + +import java.util.List; + +public interface MessageRecordConverter { + RecordWrappers convert(List messages); +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/record/ProtoMessageRecordConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/record/ProtoMessageRecordConverter.java new file mode 100644 index 00000000..cb070698 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/record/ProtoMessageRecordConverter.java @@ -0,0 +1,57 @@ +package com.gotocompany.depot.maxcompute.converter.record; + +import com.aliyun.odps.data.ArrayRecord; +import com.aliyun.odps.data.Record; +import com.gotocompany.depot.error.ErrorInfo; +import com.gotocompany.depot.error.ErrorType; +import com.gotocompany.depot.exception.InvalidMessageException; +import com.gotocompany.depot.exception.UnknownFieldsException; +import com.gotocompany.depot.maxcompute.model.MaxComputeSchema; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.maxcompute.model.RecordWrappers; +import com.gotocompany.depot.maxcompute.record.RecordDecorator; +import com.gotocompany.depot.maxcompute.schema.MaxComputeSchemaCache; +import com.gotocompany.depot.message.Message; +import lombok.RequiredArgsConstructor; + +import java.io.IOException; +import java.util.List; +import java.util.stream.IntStream; + +@RequiredArgsConstructor +public class ProtoMessageRecordConverter implements MessageRecordConverter { + + private final RecordDecorator recordDecorator; + private final MaxComputeSchemaCache maxComputeSchemaCache; + + @Override + public RecordWrappers convert(List messages) { + MaxComputeSchema maxComputeSchema = maxComputeSchemaCache.getMaxComputeSchema(); + RecordWrappers recordWrappers = new RecordWrappers(); + IntStream.range(0, messages.size()) + .forEach(index -> { + Record record = new ArrayRecord(maxComputeSchema.getColumns()); + RecordWrapper recordWrapper = new RecordWrapper(record, index, null, null); + try { + recordWrappers.addValidRecord(recordDecorator.decorate(recordWrapper, messages.get(index))); + } catch (IOException e) { + recordWrappers.addInvalidRecord( + toErrorRecordWrapper(recordWrapper, new ErrorInfo(e, ErrorType.DESERIALIZATION_ERROR)) + ); + } catch (UnknownFieldsException e) { + recordWrappers.addInvalidRecord( + toErrorRecordWrapper(recordWrapper, new ErrorInfo(e, ErrorType.UNKNOWN_FIELDS_ERROR)) + ); + } catch (InvalidMessageException e) { + recordWrappers.addInvalidRecord( + toErrorRecordWrapper(recordWrapper, new ErrorInfo(e, ErrorType.INVALID_MESSAGE_ERROR)) + ); + } + }); + return recordWrappers; + } + + private RecordWrapper toErrorRecordWrapper(RecordWrapper recordWrapper, ErrorInfo e) { + return new RecordWrapper(null, recordWrapper.getIndex(), e, recordWrapper.getPartitionSpec()); + } +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/type/DurationProtobufTypeInfoConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/type/DurationProtobufTypeInfoConverter.java new file mode 100644 index 00000000..a95b97c8 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/type/DurationProtobufTypeInfoConverter.java @@ -0,0 +1,29 @@ +package com.gotocompany.depot.maxcompute.converter.type; + +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; + +import java.util.Arrays; +import java.util.List; + +public class DurationProtobufTypeInfoConverter implements ProtobufTypeInfoConverter { + + private static final String SECONDS = "seconds"; + private static final String NANOS = "nanos"; + private static final String GOOGLE_PROTOBUF_DURATION = "google.protobuf.Duration"; + private static final List FIELD_NAMES = Arrays.asList(SECONDS, NANOS); + private static final List TYPE_INFOS = Arrays.asList(TypeInfoFactory.BIGINT, TypeInfoFactory.INT); + + @Override + public TypeInfo convertSingular(Descriptors.FieldDescriptor fieldDescriptor) { + return TypeInfoFactory.getStructTypeInfo(FIELD_NAMES, TYPE_INFOS); + } + + @Override + public boolean canConvert(Descriptors.FieldDescriptor fieldDescriptor) { + return fieldDescriptor.getType() == Descriptors.FieldDescriptor.Type.MESSAGE + && fieldDescriptor.getMessageType().getFullName().equals(GOOGLE_PROTOBUF_DURATION); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/type/MessageProtobufTypeInfoConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/type/MessageProtobufTypeInfoConverter.java new file mode 100644 index 00000000..e7bc7f5c --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/type/MessageProtobufTypeInfoConverter.java @@ -0,0 +1,39 @@ +package com.gotocompany.depot.maxcompute.converter.type; + +import com.aliyun.odps.type.StructTypeInfo; +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; + +import java.util.List; +import java.util.stream.Collectors; + +public class MessageProtobufTypeInfoConverter implements ProtobufTypeInfoConverter { + + private final List protoFieldToProtobufTypeInfoConverterList; + + public MessageProtobufTypeInfoConverter(List protoFieldToProtobufTypeInfoConverterList) { + this.protoFieldToProtobufTypeInfoConverterList = protoFieldToProtobufTypeInfoConverterList; + } + + @Override + public StructTypeInfo convertSingular(Descriptors.FieldDescriptor fieldDescriptor) { + List fieldNames = fieldDescriptor.getMessageType().getFields().stream() + .map(Descriptors.FieldDescriptor::getName) + .collect(Collectors.toList()); + List typeInfos = fieldDescriptor.getMessageType().getFields().stream() + .map(fd -> protoFieldToProtobufTypeInfoConverterList.stream() + .filter(converter -> converter.canConvert(fd)) + .findFirst() + .map(converter -> converter.convert(fd)) + .orElseThrow(() -> new IllegalArgumentException("Unsupported type: " + fd.getJavaType()))) + .collect(Collectors.toList()); + return TypeInfoFactory.getStructTypeInfo(fieldNames, typeInfos); + } + + @Override + public boolean canConvert(Descriptors.FieldDescriptor fieldDescriptor) { + return Descriptors.FieldDescriptor.Type.MESSAGE.equals(fieldDescriptor.getType()); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/type/PrimitiveProtobufTypeInfoConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/type/PrimitiveProtobufTypeInfoConverter.java new file mode 100644 index 00000000..8aa45568 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/type/PrimitiveProtobufTypeInfoConverter.java @@ -0,0 +1,44 @@ +package com.gotocompany.depot.maxcompute.converter.type; + +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.common.collect.ImmutableMap; +import com.google.protobuf.Descriptors; + +import java.util.Map; + +public class PrimitiveProtobufTypeInfoConverter implements ProtobufTypeInfoConverter { + + private static final Map PROTO_TYPE_MAP; + + static { + PROTO_TYPE_MAP = ImmutableMap.builder() + .put(Descriptors.FieldDescriptor.Type.BYTES, TypeInfoFactory.BINARY) + .put(Descriptors.FieldDescriptor.Type.STRING, TypeInfoFactory.STRING) + .put(Descriptors.FieldDescriptor.Type.ENUM, TypeInfoFactory.STRING) + .put(Descriptors.FieldDescriptor.Type.DOUBLE, TypeInfoFactory.DOUBLE) + .put(Descriptors.FieldDescriptor.Type.FLOAT, TypeInfoFactory.FLOAT) + .put(Descriptors.FieldDescriptor.Type.BOOL, TypeInfoFactory.BOOLEAN) + .put(Descriptors.FieldDescriptor.Type.INT64, TypeInfoFactory.BIGINT) + .put(Descriptors.FieldDescriptor.Type.UINT64, TypeInfoFactory.BIGINT) + .put(Descriptors.FieldDescriptor.Type.INT32, TypeInfoFactory.INT) + .put(Descriptors.FieldDescriptor.Type.UINT32, TypeInfoFactory.INT) + .put(Descriptors.FieldDescriptor.Type.FIXED64, TypeInfoFactory.BIGINT) + .put(Descriptors.FieldDescriptor.Type.FIXED32, TypeInfoFactory.INT) + .put(Descriptors.FieldDescriptor.Type.SFIXED32, TypeInfoFactory.INT) + .put(Descriptors.FieldDescriptor.Type.SFIXED64, TypeInfoFactory.BIGINT) + .put(Descriptors.FieldDescriptor.Type.SINT32, TypeInfoFactory.INT) + .put(Descriptors.FieldDescriptor.Type.SINT64, TypeInfoFactory.BIGINT) + .build(); + } + + @Override + public TypeInfo convertSingular(Descriptors.FieldDescriptor fieldDescriptor) { + return PROTO_TYPE_MAP.get(fieldDescriptor.getType()); + } + + @Override + public boolean canConvert(Descriptors.FieldDescriptor fieldDescriptor) { + return PROTO_TYPE_MAP.containsKey(fieldDescriptor.getType()); + } +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/type/ProtobufTypeInfoConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/type/ProtobufTypeInfoConverter.java new file mode 100644 index 00000000..6e1047cd --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/type/ProtobufTypeInfoConverter.java @@ -0,0 +1,16 @@ +package com.gotocompany.depot.maxcompute.converter.type; + +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; + +public interface ProtobufTypeInfoConverter { + default TypeInfo convert(Descriptors.FieldDescriptor fieldDescriptor) { + return wrap(fieldDescriptor, convertSingular(fieldDescriptor)); + } + TypeInfo convertSingular(Descriptors.FieldDescriptor fieldDescriptor); + boolean canConvert(Descriptors.FieldDescriptor fieldDescriptor); + default TypeInfo wrap(Descriptors.FieldDescriptor fieldDescriptor, TypeInfo typeInfo) { + return fieldDescriptor.isRepeated() ? TypeInfoFactory.getArrayTypeInfo(typeInfo) : typeInfo; + } +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/type/StructProtobufTypeInfoConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/type/StructProtobufTypeInfoConverter.java new file mode 100644 index 00000000..864dee93 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/type/StructProtobufTypeInfoConverter.java @@ -0,0 +1,22 @@ +package com.gotocompany.depot.maxcompute.converter.type; + +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; + +public class StructProtobufTypeInfoConverter implements ProtobufTypeInfoConverter { + + private static final String GOOGLE_PROTOBUF_STRUCT = "google.protobuf.Struct"; + + @Override + public TypeInfo convertSingular(Descriptors.FieldDescriptor fieldDescriptor) { + return TypeInfoFactory.STRING; + } + + @Override + public boolean canConvert(Descriptors.FieldDescriptor fieldDescriptor) { + return fieldDescriptor.getType() == Descriptors.FieldDescriptor.Type.MESSAGE + && fieldDescriptor.getMessageType().getFullName().equals(GOOGLE_PROTOBUF_STRUCT); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/converter/type/TimestampProtobufTypeInfoConverter.java b/src/main/java/com/gotocompany/depot/maxcompute/converter/type/TimestampProtobufTypeInfoConverter.java new file mode 100644 index 00000000..1497d4a2 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/converter/type/TimestampProtobufTypeInfoConverter.java @@ -0,0 +1,22 @@ +package com.gotocompany.depot.maxcompute.converter.type; + +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; + +public class TimestampProtobufTypeInfoConverter implements ProtobufTypeInfoConverter { + + private static final String GOOGLE_PROTOBUF_TIMESTAMP = "google.protobuf.Timestamp"; + + @Override + public TypeInfo convertSingular(Descriptors.FieldDescriptor fieldDescriptor) { + return TypeInfoFactory.TIMESTAMP_NTZ; + } + + @Override + public boolean canConvert(Descriptors.FieldDescriptor fieldDescriptor) { + return Descriptors.FieldDescriptor.Type.MESSAGE.equals(fieldDescriptor.getType()) + && fieldDescriptor.getMessageType().getFullName().equals(GOOGLE_PROTOBUF_TIMESTAMP); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/exception/MaxComputeTableOperationException.java b/src/main/java/com/gotocompany/depot/maxcompute/exception/MaxComputeTableOperationException.java new file mode 100644 index 00000000..a0aaa04a --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/exception/MaxComputeTableOperationException.java @@ -0,0 +1,13 @@ +package com.gotocompany.depot.maxcompute.exception; + +public class MaxComputeTableOperationException extends RuntimeException { + + public MaxComputeTableOperationException(String message, Exception e) { + super(message, e); + } + + public MaxComputeTableOperationException(String message) { + super(message); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/model/MaxComputeColumnDetail.java b/src/main/java/com/gotocompany/depot/maxcompute/model/MaxComputeColumnDetail.java new file mode 100644 index 00000000..66f2069f --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/model/MaxComputeColumnDetail.java @@ -0,0 +1,23 @@ +package com.gotocompany.depot.maxcompute.model; + +import com.aliyun.odps.type.TypeInfo; +import lombok.AllArgsConstructor; +import lombok.Getter; +import org.apache.commons.lang.StringUtils; + +@AllArgsConstructor +public class MaxComputeColumnDetail { + private String prefix; + private String name; + @Getter + private TypeInfo typeInfo; + private boolean isArrayElement; + + public String getDDL() { + return String.format("%s %s", getFullName(), typeInfo.toString()); + } + + public String getFullName() { + return StringUtils.isBlank(prefix) ? String.format("`%s`", name) : String.format("%s%s.`%s`", prefix, isArrayElement ? ".element" : "", name); + } +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/model/MaxComputeSchema.java b/src/main/java/com/gotocompany/depot/maxcompute/model/MaxComputeSchema.java new file mode 100644 index 00000000..7e01f0f8 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/model/MaxComputeSchema.java @@ -0,0 +1,22 @@ +package com.gotocompany.depot.maxcompute.model; + +import com.aliyun.odps.Column; +import com.aliyun.odps.TableSchema; +import com.aliyun.odps.type.TypeInfo; +import lombok.Getter; +import lombok.RequiredArgsConstructor; + +import java.util.Map; + +@RequiredArgsConstructor +@Getter +public class MaxComputeSchema { + + private final TableSchema tableSchema; + private final Map metadataColumns; + + public Column[] getColumns() { + return tableSchema.getColumns().toArray(new Column[]{}); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/model/ProtoPayload.java b/src/main/java/com/gotocompany/depot/maxcompute/model/ProtoPayload.java new file mode 100644 index 00000000..ef3dc281 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/model/ProtoPayload.java @@ -0,0 +1,13 @@ +package com.gotocompany.depot.maxcompute.model; + +import com.google.protobuf.Descriptors; +import lombok.Getter; +import lombok.RequiredArgsConstructor; + +@RequiredArgsConstructor +@Getter +public class ProtoPayload { + private final Descriptors.FieldDescriptor fieldDescriptor; + private final Object object; + private final boolean isRootLevel; +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/model/RecordWrapper.java b/src/main/java/com/gotocompany/depot/maxcompute/model/RecordWrapper.java new file mode 100644 index 00000000..8a0ce2ae --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/model/RecordWrapper.java @@ -0,0 +1,16 @@ +package com.gotocompany.depot.maxcompute.model; + +import com.aliyun.odps.PartitionSpec; +import com.aliyun.odps.data.Record; +import com.gotocompany.depot.error.ErrorInfo; +import lombok.Getter; +import lombok.RequiredArgsConstructor; + +@RequiredArgsConstructor +@Getter +public class RecordWrapper { + private final Record record; + private final long index; + private final ErrorInfo errorInfo; + private final PartitionSpec partitionSpec; +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/model/RecordWrappers.java b/src/main/java/com/gotocompany/depot/maxcompute/model/RecordWrappers.java new file mode 100644 index 00000000..8b8015a4 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/model/RecordWrappers.java @@ -0,0 +1,28 @@ +package com.gotocompany.depot.maxcompute.model; + +import lombok.AllArgsConstructor; +import lombok.Getter; + +import java.util.ArrayList; +import java.util.List; + +@AllArgsConstructor +@Getter +public class RecordWrappers { + private List validRecords; + private List invalidRecords; + + public RecordWrappers() { + this.validRecords = new ArrayList<>(); + this.invalidRecords = new ArrayList<>(); + } + + public void addValidRecord(RecordWrapper recordWrapper) { + this.validRecords.add(recordWrapper); + } + + public void addInvalidRecord(RecordWrapper recordWrapper) { + this.invalidRecords.add(recordWrapper); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/record/ProtoDataColumnRecordDecorator.java b/src/main/java/com/gotocompany/depot/maxcompute/record/ProtoDataColumnRecordDecorator.java new file mode 100644 index 00000000..a20469ee --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/record/ProtoDataColumnRecordDecorator.java @@ -0,0 +1,82 @@ +package com.gotocompany.depot.maxcompute.record; + +import com.aliyun.odps.PartitionSpec; +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.config.SinkConfig; +import com.gotocompany.depot.maxcompute.converter.ProtobufConverterOrchestrator; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.maxcompute.schema.partition.DefaultPartitioningStrategy; +import com.gotocompany.depot.maxcompute.schema.partition.PartitioningStrategy; +import com.gotocompany.depot.maxcompute.schema.partition.TimestampPartitioningStrategy; +import com.gotocompany.depot.message.Message; +import com.gotocompany.depot.message.MessageParser; +import com.gotocompany.depot.message.ParsedMessage; +import com.gotocompany.depot.message.SinkConnectorSchemaMessageMode; + +import java.io.IOException; +import java.util.Map; +import java.util.Optional; + +public class ProtoDataColumnRecordDecorator extends RecordDecorator { + + private final ProtobufConverterOrchestrator protobufConverterOrchestrator; + private final MessageParser protoMessageParser; + private final PartitioningStrategy partitioningStrategy; + private final SinkConfig sinkConfig; + + public ProtoDataColumnRecordDecorator(RecordDecorator decorator, + ProtobufConverterOrchestrator protobufConverterOrchestrator, + MessageParser messageParser, + SinkConfig sinkConfig, + PartitioningStrategy partitioningStrategy) { + super(decorator); + this.protobufConverterOrchestrator = protobufConverterOrchestrator; + this.protoMessageParser = messageParser; + this.partitioningStrategy = partitioningStrategy; + this.sinkConfig = sinkConfig; + } + + @Override + public RecordWrapper process(RecordWrapper recordWrapper, Message message) throws IOException { + String schemaClass = getSchemaClass(); + ParsedMessage parsedMessage = protoMessageParser.parse(message, sinkConfig.getSinkConnectorSchemaMessageMode(), schemaClass); + parsedMessage.validate(sinkConfig); + com.google.protobuf.Message protoMessage = (com.google.protobuf.Message) parsedMessage.getRaw(); + Map fields = protoMessage.getAllFields(); + for (Map.Entry entry : fields.entrySet()) { + if (entry.getKey().getName().equals(getPartitionFieldName()) && shouldReplaceOriginalColumn()) { + continue; + } + recordWrapper.getRecord() + .set(entry.getKey().getName(), protobufConverterOrchestrator.convert(entry.getKey(), entry.getValue())); + } + PartitionSpec partitionSpec = null; + if (partitioningStrategy != null && partitioningStrategy instanceof DefaultPartitioningStrategy) { + Descriptors.FieldDescriptor partitionFieldDescriptor = protoMessage.getDescriptorForType().findFieldByName(partitioningStrategy.getOriginalPartitionColumnName()); + Object object = protoMessage.hasField(partitionFieldDescriptor) ? protoMessage.getField(protoMessage.getDescriptorForType().findFieldByName(partitioningStrategy.getOriginalPartitionColumnName())) : null; + partitionSpec = partitioningStrategy.getPartitionSpec(object); + } + if (partitioningStrategy != null && partitioningStrategy instanceof TimestampPartitioningStrategy) { + partitionSpec = partitioningStrategy.getPartitionSpec(recordWrapper.getRecord()); + } + return new RecordWrapper(recordWrapper.getRecord(), recordWrapper.getIndex(), recordWrapper.getErrorInfo(), partitionSpec); + } + + private String getPartitionFieldName() { + return Optional.ofNullable(partitioningStrategy) + .map(PartitioningStrategy::getOriginalPartitionColumnName) + .orElse(null); + } + + private boolean shouldReplaceOriginalColumn() { + return Optional.ofNullable(partitioningStrategy) + .map(PartitioningStrategy::shouldReplaceOriginalColumn) + .orElse(false); + } + + private String getSchemaClass() { + return sinkConfig.getSinkConnectorSchemaMessageMode() == SinkConnectorSchemaMessageMode.LOG_MESSAGE + ? sinkConfig.getSinkConnectorSchemaProtoMessageClass() : sinkConfig.getSinkConnectorSchemaProtoKeyClass(); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/record/ProtoMetadataColumnRecordDecorator.java b/src/main/java/com/gotocompany/depot/maxcompute/record/ProtoMetadataColumnRecordDecorator.java new file mode 100644 index 00000000..6356f883 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/record/ProtoMetadataColumnRecordDecorator.java @@ -0,0 +1,73 @@ +package com.gotocompany.depot.maxcompute.record; + +import com.aliyun.odps.data.Record; +import com.aliyun.odps.data.SimpleStruct; +import com.aliyun.odps.type.StructTypeInfo; +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.utils.StringUtils; +import com.gotocompany.depot.common.TupleString; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.model.MaxComputeSchema; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.maxcompute.schema.MaxComputeSchemaCache; +import com.gotocompany.depot.maxcompute.util.MetadataUtil; +import com.gotocompany.depot.message.Message; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class ProtoMetadataColumnRecordDecorator extends RecordDecorator { + + private final MaxComputeSinkConfig maxComputeSinkConfig; + private final MaxComputeSchemaCache maxComputeSchemaCache; + private final Map metadataTypePairs; + + public ProtoMetadataColumnRecordDecorator(RecordDecorator recordDecorator, + MaxComputeSinkConfig maxComputeSinkConfig, + MaxComputeSchemaCache maxComputeSchemaCache) { + super(recordDecorator); + this.maxComputeSinkConfig = maxComputeSinkConfig; + this.maxComputeSchemaCache = maxComputeSchemaCache; + this.metadataTypePairs = maxComputeSinkConfig.getMetadataColumnsTypes() + .stream() + .collect(Collectors.toMap(TupleString::getFirst, TupleString::getSecond)); + } + + @Override + public RecordWrapper process(RecordWrapper recordWrapper, Message message) throws IOException { + if (StringUtils.isNotBlank(maxComputeSinkConfig.getMaxcomputeMetadataNamespace())) { + appendNamespacedMetadata(recordWrapper.getRecord(), message); + } else { + appendMetadata(recordWrapper.getRecord(), message); + } + return new RecordWrapper(recordWrapper.getRecord(), recordWrapper.getIndex(), recordWrapper.getErrorInfo(), recordWrapper.getPartitionSpec()); + } + + private void appendNamespacedMetadata(Record record, Message message) { + Map metadata = message.getMetadata(maxComputeSinkConfig.getMetadataColumnsTypes()); + MaxComputeSchema maxComputeSchema = maxComputeSchemaCache.getMaxComputeSchema(); + StructTypeInfo typeInfo = (StructTypeInfo) maxComputeSchema.getTableSchema() + .getColumn(maxComputeSinkConfig.getMaxcomputeMetadataNamespace()) + .getTypeInfo(); + List values = IntStream.range(0, typeInfo.getFieldCount()) + .mapToObj(index -> { + Object metadataValue = metadata.get(typeInfo.getFieldNames().get(index)); + return MetadataUtil.getValidMetadataValue(metadataTypePairs.get(typeInfo.getFieldNames().get(index)), metadataValue, maxComputeSinkConfig); + }).collect(Collectors.toList()); + record.set(maxComputeSinkConfig.getMaxcomputeMetadataNamespace(), new SimpleStruct(typeInfo, values)); + } + + private void appendMetadata(Record record, Message message) { + Map metadata = message.getMetadata(maxComputeSinkConfig.getMetadataColumnsTypes()); + for (Map.Entry entry : maxComputeSchemaCache.getMaxComputeSchema() + .getMetadataColumns() + .entrySet()) { + Object value = metadata.get(entry.getKey()); + record.set(entry.getKey(), MetadataUtil.getValidMetadataValue(metadataTypePairs.get(entry.getKey()), value, maxComputeSinkConfig)); + } + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/record/RecordDecorator.java b/src/main/java/com/gotocompany/depot/maxcompute/record/RecordDecorator.java new file mode 100644 index 00000000..0ec95fd9 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/record/RecordDecorator.java @@ -0,0 +1,26 @@ +package com.gotocompany.depot.maxcompute.record; + + +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.message.Message; + +import java.io.IOException; + +public abstract class RecordDecorator { + + private final RecordDecorator decorator; + + public RecordDecorator(RecordDecorator decorator) { + this.decorator = decorator; + } + + public RecordWrapper decorate(RecordWrapper recordWrapper, Message message) throws IOException { + if (decorator != null) { + return decorator.decorate(process(recordWrapper, message), message); + } + return process(recordWrapper, message); + } + + public abstract RecordWrapper process(RecordWrapper recordWrapper, Message message) throws IOException; + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/record/RecordDecoratorFactory.java b/src/main/java/com/gotocompany/depot/maxcompute/record/RecordDecoratorFactory.java new file mode 100644 index 00000000..85f8756c --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/record/RecordDecoratorFactory.java @@ -0,0 +1,26 @@ +package com.gotocompany.depot.maxcompute.record; + +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.config.SinkConfig; +import com.gotocompany.depot.maxcompute.converter.ProtobufConverterOrchestrator; +import com.gotocompany.depot.maxcompute.schema.MaxComputeSchemaCache; +import com.gotocompany.depot.maxcompute.schema.partition.PartitioningStrategy; +import com.gotocompany.depot.message.MessageParser; + +public class RecordDecoratorFactory { + + public static RecordDecorator createRecordDecorator( + ProtobufConverterOrchestrator protobufConverterOrchestrator, + MaxComputeSchemaCache maxComputeSchemaCache, + MessageParser messageParser, + PartitioningStrategy partitioningStrategy, + MaxComputeSinkConfig maxComputeSinkConfig, + SinkConfig sinkConfig) { + RecordDecorator dataColumnRecordDecorator = new ProtoDataColumnRecordDecorator(null, protobufConverterOrchestrator, messageParser, sinkConfig, partitioningStrategy); + if (!maxComputeSinkConfig.shouldAddMetadata()) { + return dataColumnRecordDecorator; + } + return new ProtoMetadataColumnRecordDecorator(dataColumnRecordDecorator, maxComputeSinkConfig, maxComputeSchemaCache); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/schema/MaxComputeSchemaCache.java b/src/main/java/com/gotocompany/depot/maxcompute/schema/MaxComputeSchemaCache.java new file mode 100644 index 00000000..8d696e6d --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/schema/MaxComputeSchemaCache.java @@ -0,0 +1,80 @@ +package com.gotocompany.depot.maxcompute.schema; + +import com.aliyun.odps.OdpsException; +import com.aliyun.odps.TableSchema; +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.config.SinkConfig; +import com.gotocompany.depot.maxcompute.client.MaxComputeClient; +import com.gotocompany.depot.maxcompute.converter.ProtobufConverterOrchestrator; +import com.gotocompany.depot.maxcompute.exception.MaxComputeTableOperationException; +import com.gotocompany.depot.maxcompute.MaxComputeSchemaHelper; +import com.gotocompany.depot.maxcompute.model.MaxComputeSchema; +import com.gotocompany.depot.message.SinkConnectorSchemaMessageMode; +import com.gotocompany.depot.message.proto.ProtoMessageParser; +import com.gotocompany.depot.stencil.DepotStencilUpdateListener; +import lombok.extern.slf4j.Slf4j; + +import java.util.Map; + +@Slf4j +public class MaxComputeSchemaCache extends DepotStencilUpdateListener { + + private final MaxComputeSchemaHelper maxComputeSchemaHelper; + private final SinkConfig sinkConfig; + private final ProtobufConverterOrchestrator protobufConverterOrchestrator; + private final MaxComputeClient maxComputeClient; + private MaxComputeSchema maxComputeSchema; + + public MaxComputeSchemaCache(MaxComputeSchemaHelper maxComputeSchemaHelper, + SinkConfig sinkConfig, + ProtobufConverterOrchestrator protobufConverterOrchestrator, + MaxComputeClient maxComputeClient) { + this.maxComputeSchemaHelper = maxComputeSchemaHelper; + this.sinkConfig = sinkConfig; + this.protobufConverterOrchestrator = protobufConverterOrchestrator; + this.maxComputeClient = maxComputeClient; + } + + public MaxComputeSchema getMaxComputeSchema() { + synchronized (this) { + if (maxComputeSchema == null) { + updateSchema(); + } + } + return maxComputeSchema; + } + + @Override + public synchronized void onSchemaUpdate(Map newDescriptor) { + Descriptors.Descriptor descriptor = newDescriptor.get(getSchemaClass()); + updateMaxComputeTableSchema(descriptor); + } + + @Override + public synchronized void updateSchema() { + Map descriptorMap = ((ProtoMessageParser) getMessageParser()).getDescriptorMap(); + Descriptors.Descriptor descriptor = descriptorMap.get(getSchemaClass()); + updateMaxComputeTableSchema(descriptor); + } + + private void updateMaxComputeTableSchema(Descriptors.Descriptor descriptor) { + MaxComputeSchema localSchema = maxComputeSchemaHelper.build(descriptor); + protobufConverterOrchestrator.clearCache(); + try { + maxComputeClient.createOrUpdateTable(localSchema.getTableSchema()); + log.info("MaxCompute table upserted successfully"); + TableSchema serverSideTableSchema = maxComputeClient.getLatestTableSchema(); + maxComputeSchema = new MaxComputeSchema( + serverSideTableSchema, + localSchema.getMetadataColumns() + ); + } catch (OdpsException e) { + throw new MaxComputeTableOperationException("Error while updating MaxCompute table", e); + } + } + + private String getSchemaClass() { + return sinkConfig.getSinkConnectorSchemaMessageMode() == SinkConnectorSchemaMessageMode.LOG_MESSAGE + ? sinkConfig.getSinkConnectorSchemaProtoMessageClass() : sinkConfig.getSinkConnectorSchemaProtoKeyClass(); + } +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/schema/MaxComputeSchemaCacheFactory.java b/src/main/java/com/gotocompany/depot/maxcompute/schema/MaxComputeSchemaCacheFactory.java new file mode 100644 index 00000000..c1db0779 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/schema/MaxComputeSchemaCacheFactory.java @@ -0,0 +1,22 @@ +package com.gotocompany.depot.maxcompute.schema; + +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.config.SinkConfig; +import com.gotocompany.depot.maxcompute.client.MaxComputeClient; +import com.gotocompany.depot.maxcompute.converter.ProtobufConverterOrchestrator; +import com.gotocompany.depot.maxcompute.MaxComputeSchemaHelper; +import com.gotocompany.depot.maxcompute.schema.partition.PartitioningStrategy; + +public class MaxComputeSchemaCacheFactory { + + public static MaxComputeSchemaCache createMaxComputeSchemaCache( + ProtobufConverterOrchestrator protobufConverterOrchestrator, + MaxComputeSinkConfig maxComputeSinkConfig, + PartitioningStrategy partitioningStrategy, + SinkConfig sinkConfig, + MaxComputeClient maxComputeClient + ) { + return new MaxComputeSchemaCache(new MaxComputeSchemaHelper(protobufConverterOrchestrator, maxComputeSinkConfig, partitioningStrategy), sinkConfig, + protobufConverterOrchestrator, maxComputeClient); + } +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/schema/SchemaDifferenceUtils.java b/src/main/java/com/gotocompany/depot/maxcompute/schema/SchemaDifferenceUtils.java new file mode 100644 index 00000000..e7e8ac7f --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/schema/SchemaDifferenceUtils.java @@ -0,0 +1,88 @@ +package com.gotocompany.depot.maxcompute.schema; + +import com.aliyun.odps.TableSchema; +import com.aliyun.odps.type.ArrayTypeInfo; +import com.aliyun.odps.type.StructTypeInfo; +import com.aliyun.odps.type.TypeInfo; +import com.gotocompany.depot.maxcompute.model.MaxComputeColumnDetail; +import org.apache.commons.lang.StringUtils; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; +import java.util.stream.Collectors; + +import static com.gotocompany.depot.maxcompute.util.TypeInfoUtils.isPrimitiveArrayType; +import static com.gotocompany.depot.maxcompute.util.TypeInfoUtils.isPrimitiveType; +import static com.gotocompany.depot.maxcompute.util.TypeInfoUtils.isStructArrayType; +import static com.gotocompany.depot.maxcompute.util.TypeInfoUtils.isStructType; + +public class SchemaDifferenceUtils { + + private static final String ALTER_TABLE_QUERY_TEMPLATE = "ALTER TABLE %s.%s ADD COLUMN IF NOT EXISTS %s;"; + + public static List getSchemaDifferenceSql(TableSchema oldSchema, TableSchema newSchema, String schemaName, String tableName) { + List maxComputeColumnDetailDifference = getMaxComputeColumnDetailDifference(oldSchema, newSchema, tableName); + + return maxComputeColumnDetailDifference.stream() + .map(MaxComputeColumnDetail -> String.format(ALTER_TABLE_QUERY_TEMPLATE, schemaName, tableName, MaxComputeColumnDetail.getDDL())) + .collect(Collectors.toList()); + } + + private static List getMaxComputeColumnDetailDifference(TableSchema oldSchema, TableSchema newSchema, String tableName) { + Map oldMaxComputeColumnDetail = buildMaxComputeColumnDetailMap(oldSchema); + Map newMaxComputeColumnDetail = buildMaxComputeColumnDetailMap(newSchema); + Iterator> newMaxComputeColumnDetailIterator = newMaxComputeColumnDetail.entrySet().iterator(); + List changedMetadata = new ArrayList<>(); + + while (newMaxComputeColumnDetailIterator.hasNext()) { + Map.Entry entry = newMaxComputeColumnDetailIterator.next(); + String columnName = entry.getKey(); + MaxComputeColumnDetail oldMetadata = oldMaxComputeColumnDetail.get(columnName); + if (Objects.isNull(oldMetadata)) { //handle new column / struct field + changedMetadata.add(entry.getValue()); + if (isStructType(entry.getValue().getTypeInfo()) || isStructArrayType(entry.getValue().getTypeInfo())) { + skipStructFields(entry, newMaxComputeColumnDetailIterator); + } + } + } + return changedMetadata; + } + + private static void skipStructFields(Map.Entry entry, Iterator> newMaxComputeColumnDetailIterator) { + StructTypeInfo structTypeInfo = isStructType(entry.getValue().getTypeInfo()) ? (StructTypeInfo) entry.getValue().getTypeInfo() + : ((StructTypeInfo) ((ArrayTypeInfo) entry.getValue().getTypeInfo()).getElementTypeInfo()); + for (int i = 0; i < structTypeInfo.getFieldCount(); i++) { + newMaxComputeColumnDetailIterator.next(); + } + } + + private static Map buildMaxComputeColumnDetailMap(TableSchema schema) { + Map maxComputeColumnDetailMap = new TreeMap<>(); + schema.getColumns().forEach(column -> fieldMetadataHelper(column.getTypeInfo(), "", column.getName(), maxComputeColumnDetailMap, false)); + return maxComputeColumnDetailMap; + } + + private static void fieldMetadataHelper(TypeInfo typeInfo, String prefix, String name, Map result, boolean isArrayElement) { + if (isPrimitiveType(typeInfo) || isPrimitiveArrayType(typeInfo)) { + result.put(getPathName(prefix, name, isArrayElement), new MaxComputeColumnDetail(prefix, name, typeInfo, isArrayElement)); + } + if (isStructType(typeInfo) || isStructArrayType(typeInfo)) { + StructTypeInfo structTypeInfo = isStructType(typeInfo) ? (StructTypeInfo) typeInfo : ((StructTypeInfo) ((ArrayTypeInfo) typeInfo).getElementTypeInfo()); + result.put(getPathName(prefix, name, isArrayElement), new MaxComputeColumnDetail(prefix, name, typeInfo, isArrayElement)); + + for (int i = 0; i < structTypeInfo.getFieldCount(); i++) { + TypeInfo fieldType = structTypeInfo.getFieldTypeInfos().get(i); + String fieldName = structTypeInfo.getFieldNames().get(i); + fieldMetadataHelper(fieldType, getPathName(prefix, name, isArrayElement), fieldName, result, isStructArrayType(typeInfo)); + } + } + } + + private static String getPathName(String prefix, String name, boolean isArrayElement) { + return StringUtils.isBlank(prefix) ? String.format("`%s`", name) : String.format("%s%s.`%s`", prefix, isArrayElement ? ".element" : "", name); + } +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/schema/partition/DefaultPartitioningStrategy.java b/src/main/java/com/gotocompany/depot/maxcompute/schema/partition/DefaultPartitioningStrategy.java new file mode 100644 index 00000000..48e76086 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/schema/partition/DefaultPartitioningStrategy.java @@ -0,0 +1,42 @@ +package com.gotocompany.depot.maxcompute.schema.partition; + +import com.aliyun.odps.Column; +import com.aliyun.odps.PartitionSpec; +import com.aliyun.odps.type.TypeInfo; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import lombok.RequiredArgsConstructor; + +@RequiredArgsConstructor +public class DefaultPartitioningStrategy implements PartitioningStrategy { + + private static final String PARTITION_SPEC_FORMAT = "%s=%s"; + private static final String DEFAULT_PARTITION = "__NULL__"; + + private final TypeInfo typeInfo; + private final MaxComputeSinkConfig maxComputeSinkConfig; + + @Override + public String getOriginalPartitionColumnName() { + return maxComputeSinkConfig.getTablePartitionKey(); + } + + @Override + public boolean shouldReplaceOriginalColumn() { + return true; + } + + @Override + public Column getPartitionColumn() { + return Column.newBuilder(maxComputeSinkConfig.getTablePartitionColumnName(), typeInfo) + .build(); + } + + @Override + public PartitionSpec getPartitionSpec(Object object) { + if (object == null) { + return new PartitionSpec(String.format(PARTITION_SPEC_FORMAT, maxComputeSinkConfig.getTablePartitionColumnName(), DEFAULT_PARTITION)); + } + return new PartitionSpec(String.format(PARTITION_SPEC_FORMAT, maxComputeSinkConfig.getTablePartitionColumnName(), object)); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/schema/partition/PartitioningStrategy.java b/src/main/java/com/gotocompany/depot/maxcompute/schema/partition/PartitioningStrategy.java new file mode 100644 index 00000000..1c9e0b22 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/schema/partition/PartitioningStrategy.java @@ -0,0 +1,11 @@ +package com.gotocompany.depot.maxcompute.schema.partition; + +import com.aliyun.odps.Column; +import com.aliyun.odps.PartitionSpec; + +public interface PartitioningStrategy { + String getOriginalPartitionColumnName(); + boolean shouldReplaceOriginalColumn(); + Column getPartitionColumn(); + PartitionSpec getPartitionSpec(Object object); +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/schema/partition/PartitioningStrategyFactory.java b/src/main/java/com/gotocompany/depot/maxcompute/schema/partition/PartitioningStrategyFactory.java new file mode 100644 index 00000000..5969f801 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/schema/partition/PartitioningStrategyFactory.java @@ -0,0 +1,54 @@ +package com.gotocompany.depot.maxcompute.schema.partition; + +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.common.collect.Sets; +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.converter.ProtobufConverterOrchestrator; +import lombok.RequiredArgsConstructor; + +import java.util.Set; + +@RequiredArgsConstructor +public class PartitioningStrategyFactory { + + private static final Set ALLOWED_PARTITION_KEY_TYPE_INFO = Sets.newHashSet( + TypeInfoFactory.TIMESTAMP_NTZ, + TypeInfoFactory.STRING, + TypeInfoFactory.TINYINT, + TypeInfoFactory.SMALLINT, + TypeInfoFactory.INT, + TypeInfoFactory.BIGINT + ); + + public static PartitioningStrategy createPartitioningStrategy( + ProtobufConverterOrchestrator protobufConverterOrchestrator, + MaxComputeSinkConfig maxComputeSinkConfig, + Descriptors.Descriptor descriptor) { + if (!maxComputeSinkConfig.isTablePartitioningEnabled()) { + return null; + } + String partitionKey = maxComputeSinkConfig.getTablePartitionKey(); + Descriptors.FieldDescriptor fieldDescriptor = descriptor + .findFieldByName(partitionKey); + + if (fieldDescriptor == null) { + throw new IllegalArgumentException("Partition key not found in the descriptor: " + partitionKey); + } + TypeInfo partitionKeyTypeInfo = protobufConverterOrchestrator.convert(fieldDescriptor); + checkPartitionTypePrecondition(partitionKeyTypeInfo); + if (TypeInfoFactory.TIMESTAMP_NTZ.equals(partitionKeyTypeInfo)) { + return new TimestampPartitioningStrategy(maxComputeSinkConfig); + } else { + return new DefaultPartitioningStrategy(partitionKeyTypeInfo, maxComputeSinkConfig); + } + } + + private static void checkPartitionTypePrecondition(TypeInfo typeInfo) { + if (!ALLOWED_PARTITION_KEY_TYPE_INFO.contains(typeInfo)) { + throw new IllegalArgumentException("Partition key type not supported: " + typeInfo.getTypeName()); + } + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/schema/partition/TimestampPartitioningStrategy.java b/src/main/java/com/gotocompany/depot/maxcompute/schema/partition/TimestampPartitioningStrategy.java new file mode 100644 index 00000000..2e251a6b --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/schema/partition/TimestampPartitioningStrategy.java @@ -0,0 +1,55 @@ +package com.gotocompany.depot.maxcompute.schema.partition; + +import com.aliyun.odps.Column; +import com.aliyun.odps.PartitionSpec; +import com.aliyun.odps.data.GenerateExpression; +import com.aliyun.odps.data.Record; +import com.aliyun.odps.expression.TruncTime; +import com.aliyun.odps.type.TypeInfoFactory; +import com.gotocompany.depot.config.MaxComputeSinkConfig; + +public class TimestampPartitioningStrategy implements PartitioningStrategy { + + private final MaxComputeSinkConfig maxComputeSinkConfig; + private final GenerateExpression generateExpression; + + public TimestampPartitioningStrategy(MaxComputeSinkConfig maxComputeSinkConfig) { + this.maxComputeSinkConfig = maxComputeSinkConfig; + this.generateExpression = initializeGenerateExpression(); + } + + @Override + public String getOriginalPartitionColumnName() { + return maxComputeSinkConfig.getTablePartitionKey(); + } + + @Override + public boolean shouldReplaceOriginalColumn() { + return false; + } + + @Override + public Column getPartitionColumn() { + Column column = Column.newBuilder(maxComputeSinkConfig.getTablePartitionColumnName(), TypeInfoFactory.STRING) + .build(); + column.setGenerateExpression(new TruncTime(maxComputeSinkConfig.getTablePartitionKey(), + maxComputeSinkConfig.getTablePartitionByTimestampTimeUnit())); + return column; + } + + @Override + public PartitionSpec getPartitionSpec(Object object) { + PartitionSpec partitionSpec = new PartitionSpec(); + if (object instanceof Record) { + Record record = (Record) object; + partitionSpec.set(maxComputeSinkConfig.getTablePartitionColumnName(), generateExpression.generate(record)); + } + return partitionSpec; + } + + private GenerateExpression initializeGenerateExpression() { + return new TruncTime(maxComputeSinkConfig.getTablePartitionKey(), + maxComputeSinkConfig.getTablePartitionByTimestampTimeUnit()); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/schema/validator/TableValidator.java b/src/main/java/com/gotocompany/depot/maxcompute/schema/validator/TableValidator.java new file mode 100644 index 00000000..b65fc913 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/schema/validator/TableValidator.java @@ -0,0 +1,53 @@ +package com.gotocompany.depot.maxcompute.schema.validator; + +import com.aliyun.odps.TableSchema; +import com.gotocompany.depot.config.MaxComputeSinkConfig; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.regex.Pattern; + +public class TableValidator { + + private final MaxComputeSinkConfig maxComputeSinkConfig; + private final Pattern validTableNamePattern; + + public TableValidator(MaxComputeSinkConfig maxComputeSinkConfig) { + this.maxComputeSinkConfig = maxComputeSinkConfig; + this.validTableNamePattern = Pattern.compile(maxComputeSinkConfig.getTableValidatorNameRegex()); + + } + + public void validate(String tableName, Long lifecycleDays, TableSchema tableSchema) { + List errorHolder = new ArrayList<>(); + validateTableName(tableName, errorHolder); + validateLifecycleDays(lifecycleDays, errorHolder); + validateTableSchema(tableSchema, errorHolder); + if (!errorHolder.isEmpty()) { + throw new IllegalArgumentException(String.join(", ", errorHolder)); + } + } + + private void validateTableName(String tableName, List errorHolder) { + if (!validTableNamePattern.matcher(tableName).matches()) { + errorHolder.add("Table name should match the pattern: " + validTableNamePattern.pattern()); + } + } + + private void validateLifecycleDays(Long lifecycleDays, List errorHolder) { + if (Objects.nonNull(lifecycleDays) && lifecycleDays < 0) { + errorHolder.add("Lifecycle days should be a positive integer"); + } + } + + private void validateTableSchema(TableSchema tableSchema, List errorHolder) { + if (tableSchema.getAllColumns().size() > maxComputeSinkConfig.getTableValidatorMaxColumnsPerTable()) { + errorHolder.add("Table schema should have less or equal than " + maxComputeSinkConfig.getTableValidatorMaxColumnsPerTable() + " columns"); + } + if (tableSchema.getPartitionColumns().size() > maxComputeSinkConfig.getTableValidatorMaxPartitionKeysPerTable()) { + errorHolder.add("Table schema should have less or equal than " + maxComputeSinkConfig.getTableValidatorMaxPartitionKeysPerTable() + " partition keys"); + } + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/util/MetadataUtil.java b/src/main/java/com/gotocompany/depot/maxcompute/util/MetadataUtil.java new file mode 100644 index 00000000..98a35982 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/util/MetadataUtil.java @@ -0,0 +1,74 @@ +package com.gotocompany.depot.maxcompute.util; + +import com.aliyun.odps.type.StructTypeInfo; +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.common.collect.ImmutableMap; +import com.gotocompany.depot.common.TupleString; +import com.gotocompany.depot.config.MaxComputeSinkConfig; + +import java.time.Instant; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class MetadataUtil { + + private static final Map METADATA_TYPE_MAP; + private static final Map> METADATA_MAPPER_MAP; + + private static final String TIMESTAMP = "timestamp"; + private static final String INTEGER = "integer"; + private static final String LONG = "long"; + private static final String FLOAT = "float"; + private static final String DOUBLE = "double"; + private static final String STRING = "string"; + private static final String BOOLEAN = "boolean"; + + static { + METADATA_TYPE_MAP = ImmutableMap.builder() + .put(INTEGER, TypeInfoFactory.INT) + .put(LONG, TypeInfoFactory.BIGINT) + .put(FLOAT, TypeInfoFactory.FLOAT) + .put(DOUBLE, TypeInfoFactory.DOUBLE) + .put(STRING, TypeInfoFactory.STRING) + .put(BOOLEAN, TypeInfoFactory.BOOLEAN) + .put(TIMESTAMP, TypeInfoFactory.TIMESTAMP_NTZ) + .build(); + + METADATA_MAPPER_MAP = ImmutableMap.>builder() + .put(INTEGER, obj -> ((Number) obj).intValue()) + .put(LONG, obj -> ((Number) obj).longValue()) + .put(FLOAT, obj -> ((Number) obj).floatValue()) + .put(DOUBLE, obj -> ((Number) obj).doubleValue()) + .put(STRING, Function.identity()) + .put(BOOLEAN, Function.identity()) + .build(); + } + + public static TypeInfo getMetadataTypeInfo(String type) { + return METADATA_TYPE_MAP.get(type.toLowerCase()); + } + + public static Object getValidMetadataValue(String type, Object value, MaxComputeSinkConfig maxComputeSinkConfig) { + if (TIMESTAMP.equalsIgnoreCase(type) && value instanceof Long) { + return Instant.ofEpochMilli((Long) value) + .atZone(maxComputeSinkConfig.getZoneId()) + .toLocalDateTime(); + } + return METADATA_MAPPER_MAP.get(type.toLowerCase()).apply(value); + } + + public static StructTypeInfo getMetadataTypeInfo(List metadataColumnsTypes) { + return TypeInfoFactory.getStructTypeInfo(metadataColumnsTypes + .stream() + .map(TupleString::getFirst) + .collect(Collectors.toList()), + metadataColumnsTypes + .stream() + .map(tuple -> METADATA_TYPE_MAP.get(tuple.getSecond().toLowerCase())) + .collect(Collectors.toList())); + } + +} diff --git a/src/main/java/com/gotocompany/depot/maxcompute/util/TypeInfoUtils.java b/src/main/java/com/gotocompany/depot/maxcompute/util/TypeInfoUtils.java new file mode 100644 index 00000000..a84ffb6f --- /dev/null +++ b/src/main/java/com/gotocompany/depot/maxcompute/util/TypeInfoUtils.java @@ -0,0 +1,24 @@ +package com.gotocompany.depot.maxcompute.util; + +import com.aliyun.odps.type.ArrayTypeInfo; +import com.aliyun.odps.type.PrimitiveTypeInfo; +import com.aliyun.odps.type.StructTypeInfo; +import com.aliyun.odps.type.TypeInfo; + +public class TypeInfoUtils { + public static boolean isPrimitiveType(TypeInfo typeInfo) { + return typeInfo instanceof PrimitiveTypeInfo; + } + + public static boolean isStructArrayType(TypeInfo typeInfo) { + return typeInfo instanceof ArrayTypeInfo && ((ArrayTypeInfo) typeInfo).getElementTypeInfo() instanceof StructTypeInfo; + } + + public static boolean isPrimitiveArrayType(TypeInfo typeInfo) { + return typeInfo instanceof ArrayTypeInfo && ((ArrayTypeInfo) typeInfo).getElementTypeInfo() instanceof PrimitiveTypeInfo; + } + + public static boolean isStructType(TypeInfo typeInfo) { + return typeInfo instanceof StructTypeInfo; + } +} diff --git a/src/main/java/com/gotocompany/depot/message/ProtoUnknownFieldValidationType.java b/src/main/java/com/gotocompany/depot/message/ProtoUnknownFieldValidationType.java new file mode 100644 index 00000000..e3995b4f --- /dev/null +++ b/src/main/java/com/gotocompany/depot/message/ProtoUnknownFieldValidationType.java @@ -0,0 +1,83 @@ +package com.gotocompany.depot.message; + +import com.google.protobuf.Message; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Stream; + +public enum ProtoUnknownFieldValidationType { + MESSAGE { + @Override + public boolean shouldFilter(Object object) { + return object instanceof Message; + } + + @Override + public Stream getMapper(Object object) { + return Stream.of((Message) object); + } + }, + MESSAGE_ARRAY_FIRST_INDEX { + @Override + public boolean shouldFilter(Object object) { + return isMessageOrMessageListType(object); + } + + @Override + public Stream getMapper(Object object) { + if (object instanceof Message) { + return Stream.of((Message) object); + } + if (object instanceof List) { + return Optional.ofNullable(((List) object).get(0)) + .filter(this::shouldFilter) + .map(o -> Stream.of((Message) o)) + .orElseGet(Stream::empty); + } + return Stream.empty(); + } + }, + MESSAGE_ARRAY_FULL { + @Override + public boolean shouldFilter(Object object) { + return isMessageOrMessageListType(object); + } + + @Override + public Stream getMapper(Object object) { + if (object instanceof Message) { + return Stream.of((Message) object); + } + if (object instanceof List) { + return Optional.of(object) + .filter(this::shouldFilter) + .map(messageList -> ((List) messageList).stream()) + .orElseGet(Stream::empty); + } + return Stream.empty(); + } + }; + + public abstract boolean shouldFilter(Object object); + public abstract Stream getMapper(Object object); + + private static boolean isMessageOrMessageListType(Object object) { + if (Objects.isNull(object)) { + return false; + } + if (object instanceof Message) { + return true; + } + if (object instanceof List) { + List list = (List) object; + + if (list.isEmpty()) { + return false; + } + return list.get(0) instanceof Message; + } + return false; + } +} diff --git a/src/main/java/com/gotocompany/depot/message/proto/ProtoMessageParser.java b/src/main/java/com/gotocompany/depot/message/proto/ProtoMessageParser.java index fdb58072..19fe5644 100644 --- a/src/main/java/com/gotocompany/depot/message/proto/ProtoMessageParser.java +++ b/src/main/java/com/gotocompany/depot/message/proto/ProtoMessageParser.java @@ -114,4 +114,8 @@ public ProtoField getProtoField(String schemaClass) throws IOException { getTypeNameToPackageNameMap(getDescriptorMap())); return protoField; } + + public Descriptors.Descriptor getDescriptor(String schemaClass) { + return stencilClient.get(schemaClass); + } } diff --git a/src/main/java/com/gotocompany/depot/message/proto/ProtoParsedMessage.java b/src/main/java/com/gotocompany/depot/message/proto/ProtoParsedMessage.java index 4b5ef062..089260b8 100644 --- a/src/main/java/com/gotocompany/depot/message/proto/ProtoParsedMessage.java +++ b/src/main/java/com/gotocompany/depot/message/proto/ProtoParsedMessage.java @@ -64,7 +64,8 @@ public JSONObject toJson() { @Override public void validate(SinkConfig config) { - if (!config.getSinkConnectorSchemaProtoAllowUnknownFieldsEnable() && ProtoUtils.hasUnknownField(dynamicMessage)) { + if (!config.getSinkConnectorSchemaProtoAllowUnknownFieldsEnable() && ProtoUtils.hasUnknownField(dynamicMessage, + config.getSinkConnectorSchemaProtoUnknownFieldsValidation())) { log.error("Unknown fields {}", UnknownProtoFields.toString(dynamicMessage.toByteArray())); throw new UnknownFieldsException(dynamicMessage); } diff --git a/src/main/java/com/gotocompany/depot/metrics/MaxComputeMetrics.java b/src/main/java/com/gotocompany/depot/metrics/MaxComputeMetrics.java new file mode 100644 index 00000000..e126f2d1 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/metrics/MaxComputeMetrics.java @@ -0,0 +1,40 @@ +package com.gotocompany.depot.metrics; + +import com.gotocompany.depot.config.SinkConfig; + +public class MaxComputeMetrics extends SinkMetrics { + + public static final String MAXCOMPUTE_SINK_PREFIX = "maxcompute_"; + public static final String MAXCOMPUTE_TABLE_TAG = "table=%s"; + public static final String MAXCOMPUTE_PROJECT_TAG = "project=%s"; + public static final String MAXCOMPUTE_API_TAG = "api=%s"; + public static final String MAXCOMPUTE_ERROR_TAG = "error=%s"; + public static final String MAXCOMPUTE_COMPRESSION_TAG = "compression=%s-%s"; + + public MaxComputeMetrics(SinkConfig config) { + super(config); + } + + public enum MaxComputeAPIType { + TABLE_UPDATE, + TABLE_CREATE, + TABLE_INSERT, + } + + public String getMaxComputeOperationTotalMetric() { + return getApplicationPrefix() + SINK_PREFIX + MAXCOMPUTE_SINK_PREFIX + "operation_total"; + } + + public String getMaxComputeOperationLatencyMetric() { + return getApplicationPrefix() + SINK_PREFIX + MAXCOMPUTE_SINK_PREFIX + "operation_latency_milliseconds"; + } + + public String getMaxComputeFlushSizeMetric() { + return String.format("%s%s%s%s", getApplicationPrefix(), SINK_PREFIX, MAXCOMPUTE_SINK_PREFIX, "flush_size_bytes"); + } + + public String getMaxComputeFlushRecordMetric() { + return String.format("%s%s%s%s", getApplicationPrefix(), SINK_PREFIX, MAXCOMPUTE_SINK_PREFIX, "flush_record_count"); + } + +} diff --git a/src/main/java/com/gotocompany/depot/utils/ProtoUtils.java b/src/main/java/com/gotocompany/depot/utils/ProtoUtils.java index 3ffda4de..90297ffb 100644 --- a/src/main/java/com/gotocompany/depot/utils/ProtoUtils.java +++ b/src/main/java/com/gotocompany/depot/utils/ProtoUtils.java @@ -1,6 +1,7 @@ package com.gotocompany.depot.utils; import com.google.protobuf.Message; +import com.gotocompany.depot.message.ProtoUnknownFieldValidationType; import java.util.Collections; import java.util.LinkedList; @@ -9,14 +10,14 @@ import java.util.stream.Collectors; public class ProtoUtils { - public static boolean hasUnknownField(Message root) { - List messageFields = collectNestedFields(root); + public static boolean hasUnknownField(Message root, ProtoUnknownFieldValidationType protoUnknownFieldValidationType) { + List messageFields = collectNestedFields(root, protoUnknownFieldValidationType); List messageWithUnknownFields = getMessageWithUnknownFields(messageFields); return !messageWithUnknownFields.isEmpty(); } - private static List collectNestedFields(Message node) { + private static List collectNestedFields(Message node, ProtoUnknownFieldValidationType protoUnknownFieldValidationType) { List output = new LinkedList<>(); Queue stack = Collections.asLifoQueue(new LinkedList<>()); stack.add(node); @@ -26,8 +27,8 @@ private static List collectNestedFields(Message node) { break; } List nestedChildNodes = current.getAllFields().values().stream() - .filter(field -> field instanceof Message) - .map(field -> (Message) field) + .filter(protoUnknownFieldValidationType::shouldFilter) + .flatMap(protoUnknownFieldValidationType::getMapper) .collect(Collectors.toList()); stack.addAll(nestedChildNodes); @@ -39,6 +40,5 @@ private static List collectNestedFields(Message node) { private static List getMessageWithUnknownFields(List messages) { return messages.stream().filter(message -> !message.getUnknownFields().asMap().isEmpty()).collect(Collectors.toList()); - } } diff --git a/src/main/java/com/gotocompany/depot/utils/RetryUtils.java b/src/main/java/com/gotocompany/depot/utils/RetryUtils.java new file mode 100644 index 00000000..8bd06cc1 --- /dev/null +++ b/src/main/java/com/gotocompany/depot/utils/RetryUtils.java @@ -0,0 +1,48 @@ +package com.gotocompany.depot.utils; + +import com.gotocompany.depot.exception.NonRetryableException; +import lombok.extern.slf4j.Slf4j; + +import java.util.function.Predicate; + +@Slf4j +public class RetryUtils { + + public static void executeWithRetry(RunnableWithException runnableWithException, + int maxRetries, + long backoffMillis, + Predicate retryPredicate) { + int retryCount = 0; + Exception lastException = null; + while (retryCount < maxRetries) { + try { + runnableWithException.run(); + break; + } catch (Exception e) { + if (retryPredicate.test(e)) { + retryCount++; + lastException = e; + log.info("Retrying operation, retry count: {}", retryCount); + } else { + log.error("Non-retryable exception occurred, aborting operation", e); + throw new NonRetryableException(e.getMessage(), e); + } + } + try { + Thread.sleep(backoffMillis); + } catch (InterruptedException e) { + log.error("Thread interrupted while sleeping", e); + } + } + if (retryCount == maxRetries) { + log.error("Max retries reached, aborting operation"); + throw new NonRetryableException("Max retries reached, aborting operation", lastException); + } + } + + @FunctionalInterface + public interface RunnableWithException { + void run() throws Exception; + } + +} diff --git a/src/test/java/com/gotocompany/depot/config/converter/LocalDateTimeConverterTest.java b/src/test/java/com/gotocompany/depot/config/converter/LocalDateTimeConverterTest.java new file mode 100644 index 00000000..fdcda8a1 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/config/converter/LocalDateTimeConverterTest.java @@ -0,0 +1,30 @@ +package com.gotocompany.depot.config.converter; + +import org.junit.Test; + +import java.time.DateTimeException; +import java.time.LocalDateTime; + +import static org.junit.Assert.assertEquals; + +public class LocalDateTimeConverterTest { + + private final LocalDateTimeConverter localDateTimeConverter = new LocalDateTimeConverter(); + + @Test + public void shouldConvertToLocalDateTime() { + String input = "2024-01-01T00:00:00"; + LocalDateTime expected = LocalDateTime.of(2024, 1, 1, 0, 0, 0); + + LocalDateTime localDateTime = localDateTimeConverter.convert(null, input); + + assertEquals(expected, localDateTime); + } + + @Test(expected = DateTimeException.class) + public void shouldThrowExceptionWhenInputIsInvalid() { + String input = "12-312024-01-01T00:00:00Z"; + localDateTimeConverter.convert(null, input); + } + +} diff --git a/src/test/java/com/gotocompany/depot/config/converter/MaxComputeOdpsGlobalSettingsConverterTest.java b/src/test/java/com/gotocompany/depot/config/converter/MaxComputeOdpsGlobalSettingsConverterTest.java new file mode 100644 index 00000000..84acf948 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/config/converter/MaxComputeOdpsGlobalSettingsConverterTest.java @@ -0,0 +1,51 @@ +package com.gotocompany.depot.config.converter; + +import org.junit.Test; +import org.junit.jupiter.api.Assertions; + +import java.util.Map; + +public class MaxComputeOdpsGlobalSettingsConverterTest { + + @Test + public void shouldParseOdpsGlobalSettings() { + MaxComputeOdpsGlobalSettingsConverter converter = new MaxComputeOdpsGlobalSettingsConverter(); + String odpsGlobalSettings = "odps.schema.evolution.enable=true , odps.task.major.version= sql_flighting_autopt "; + + Map settings = converter.convert(null, odpsGlobalSettings); + + Assertions.assertEquals(2, settings.size()); + Assertions.assertTrue(settings.containsKey("odps.schema.evolution.enable")); + Assertions.assertEquals("true", settings.get("odps.schema.evolution.enable")); + Assertions.assertTrue(settings.containsKey("odps.task.major.version")); + Assertions.assertEquals("sql_flighting_autopt", settings.get("odps.task.major.version")); + } + + @Test + public void shouldParseEmptyMapWhenGivenStringIsEmpty() { + MaxComputeOdpsGlobalSettingsConverter converter = new MaxComputeOdpsGlobalSettingsConverter(); + String odpsGlobalSettings = " "; + + Map settings = converter.convert(null, odpsGlobalSettings); + + Assertions.assertEquals(0, settings.size()); + } + + @Test + public void shouldParseEmptyMapWhenGivenStringIsNull() { + MaxComputeOdpsGlobalSettingsConverter converter = new MaxComputeOdpsGlobalSettingsConverter(); + + Map settings = converter.convert(null, null); + + Assertions.assertEquals(0, settings.size()); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIllegalArgumentException() { + MaxComputeOdpsGlobalSettingsConverter converter = new MaxComputeOdpsGlobalSettingsConverter(); + String odpsGlobalSettings = "odps.schema.evolution.enable+true , odps.task.major.version=sql_flighting_autopt"; + + converter.convert(null, odpsGlobalSettings); + } + +} diff --git a/src/test/java/com/gotocompany/depot/config/converter/ZoneIdConverterTest.java b/src/test/java/com/gotocompany/depot/config/converter/ZoneIdConverterTest.java new file mode 100644 index 00000000..b5b5893c --- /dev/null +++ b/src/test/java/com/gotocompany/depot/config/converter/ZoneIdConverterTest.java @@ -0,0 +1,30 @@ +package com.gotocompany.depot.config.converter; + +import com.gotocompany.depot.exception.ConfigurationException; +import org.junit.Test; + +import java.time.ZoneId; + +import static org.junit.Assert.assertEquals; + +public class ZoneIdConverterTest { + + @Test + public void shouldParseValidZoneId() { + String zoneId = "UTC"; + ZoneIdConverter zoneIdConverter = new ZoneIdConverter(); + + ZoneId result = zoneIdConverter.convert(null, zoneId); + + assertEquals(ZoneId.of(zoneId), result); + } + + @Test(expected = ConfigurationException.class) + public void shouldThrowDateTimeExceptionGivenInvalidZoneId() { + String zoneId = "InvalidZoneId"; + ZoneIdConverter zoneIdConverter = new ZoneIdConverter(); + + zoneIdConverter.convert(null, zoneId); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/MaxComputeSchemaHelperTest.java b/src/test/java/com/gotocompany/depot/maxcompute/MaxComputeSchemaHelperTest.java new file mode 100644 index 00000000..c60fcc89 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/MaxComputeSchemaHelperTest.java @@ -0,0 +1,208 @@ +package com.gotocompany.depot.maxcompute; + +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.TextMaxComputeTable; +import com.gotocompany.depot.common.TupleString; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.converter.ProtobufConverterOrchestrator; +import com.gotocompany.depot.maxcompute.model.MaxComputeSchema; +import com.gotocompany.depot.maxcompute.schema.partition.PartitioningStrategy; +import com.gotocompany.depot.maxcompute.schema.partition.PartitioningStrategyFactory; +import org.assertj.core.groups.Tuple; +import org.junit.Test; +import org.mockito.Mockito; + +import java.time.ZoneId; +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.when; + + +public class MaxComputeSchemaHelperTest { + + private final Descriptors.Descriptor descriptor = TextMaxComputeTable.Table.getDescriptor(); + + @Test + public void shouldBuildPartitionedTableSchemaWithRootLevelMetadata() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.shouldAddMetadata()).thenReturn(Boolean.TRUE); + when(maxComputeSinkConfig.getMetadataColumnsTypes()).thenReturn( + Arrays.asList(new TupleString("__message_timestamp", "timestamp"), + new TupleString("__kafka_topic", "string"), + new TupleString("__kafka_offset", "long") + ) + ); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(Boolean.TRUE); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn("event_timestamp"); + when(maxComputeSinkConfig.getTablePartitionColumnName()).thenReturn("__partitioning_column"); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getTablePartitionByTimestampTimeUnit()).thenReturn("DAY"); + when(maxComputeSinkConfig.getMaxPastYearEventTimeDifference()).thenReturn(999); + when(maxComputeSinkConfig.getMaxFutureYearEventTimeDifference()).thenReturn(999); + PartitioningStrategy partitioningStrategy = PartitioningStrategyFactory.createPartitioningStrategy( + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeSinkConfig, + descriptor + ); + MaxComputeSchemaHelper maxComputeSchemaHelper = new MaxComputeSchemaHelper(new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeSinkConfig, partitioningStrategy); + int expectedNonPartitionColumnCount = 7; + int expectedPartitionColumnCount = 1; + + MaxComputeSchema maxComputeSchema = maxComputeSchemaHelper.build(descriptor); + + assertThat(maxComputeSchema.getTableSchema().getColumns().size()).isEqualTo(expectedNonPartitionColumnCount); + assertThat(maxComputeSchema.getTableSchema().getPartitionColumns().size()).isEqualTo(expectedPartitionColumnCount); + assertThat(maxComputeSchema.getTableSchema().getColumns()) + .extracting("name", "typeInfo") + .containsExactlyInAnyOrder( + Tuple.tuple("id", TypeInfoFactory.STRING), + Tuple.tuple("user", TypeInfoFactory.getStructTypeInfo( + Arrays.asList("id", "contacts"), + Arrays.asList(TypeInfoFactory.STRING, TypeInfoFactory.getArrayTypeInfo(TypeInfoFactory.getStructTypeInfo( + Arrays.asList("number"), + Arrays.asList(TypeInfoFactory.STRING) + ))) + )), + Tuple.tuple("items", TypeInfoFactory.getArrayTypeInfo(TypeInfoFactory.getStructTypeInfo( + Arrays.asList("id", "name"), + Arrays.asList(TypeInfoFactory.STRING, TypeInfoFactory.STRING) + ))), + Tuple.tuple("event_timestamp", TypeInfoFactory.TIMESTAMP_NTZ), + Tuple.tuple("__message_timestamp", TypeInfoFactory.TIMESTAMP_NTZ), + Tuple.tuple("__kafka_topic", TypeInfoFactory.STRING), + Tuple.tuple("__kafka_offset", TypeInfoFactory.BIGINT) + ); + assertThat(maxComputeSchema.getTableSchema().getPartitionColumns()) + .extracting("name", "typeInfo") + .contains(Tuple.tuple("__partitioning_column", TypeInfoFactory.STRING)); + } + + @Test + public void shouldBuildPartitionedTableSchemaWithNestedMetadata() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.shouldAddMetadata()).thenReturn(Boolean.TRUE); + when(maxComputeSinkConfig.getMaxcomputeMetadataNamespace()).thenReturn("meta"); + when(maxComputeSinkConfig.getMetadataColumnsTypes()).thenReturn( + Arrays.asList(new TupleString("__message_timestamp", "timestamp"), + new TupleString("__kafka_topic", "string"), + new TupleString("__kafka_offset", "long") + ) + ); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(Boolean.TRUE); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn("event_timestamp"); + when(maxComputeSinkConfig.getTablePartitionColumnName()).thenReturn("__partitioning_column"); + when(maxComputeSinkConfig.getTablePartitionByTimestampTimeUnit()).thenReturn("DAY"); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getMaxPastYearEventTimeDifference()).thenReturn(999); + when(maxComputeSinkConfig.getMaxFutureYearEventTimeDifference()).thenReturn(999); + int expectedNonPartitionColumnCount = 5; + int expectedPartitionColumnCount = 1; + PartitioningStrategy partitioningStrategy = PartitioningStrategyFactory.createPartitioningStrategy( + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeSinkConfig, + descriptor + ); + MaxComputeSchemaHelper maxComputeSchemaHelper = new MaxComputeSchemaHelper( + new ProtobufConverterOrchestrator(maxComputeSinkConfig), maxComputeSinkConfig, partitioningStrategy); + + MaxComputeSchema maxComputeSchema = maxComputeSchemaHelper.build(descriptor); + + assertThat(maxComputeSchema.getTableSchema().getColumns().size()).isEqualTo(expectedNonPartitionColumnCount); + assertThat(maxComputeSchema.getTableSchema().getPartitionColumns().size()).isEqualTo(expectedPartitionColumnCount); + assertThat(maxComputeSchema.getTableSchema().getColumns()) + .extracting("name", "typeInfo") + .containsExactlyInAnyOrder( + Tuple.tuple("id", TypeInfoFactory.STRING), + Tuple.tuple("user", TypeInfoFactory.getStructTypeInfo( + Arrays.asList("id", "contacts"), + Arrays.asList(TypeInfoFactory.STRING, TypeInfoFactory.getArrayTypeInfo(TypeInfoFactory.getStructTypeInfo( + Arrays.asList("number"), + Arrays.asList(TypeInfoFactory.STRING) + ))) + )), + Tuple.tuple("items", TypeInfoFactory.getArrayTypeInfo(TypeInfoFactory.getStructTypeInfo( + Arrays.asList("id", "name"), + Arrays.asList(TypeInfoFactory.STRING, TypeInfoFactory.STRING) + ))), + Tuple.tuple("event_timestamp", TypeInfoFactory.TIMESTAMP_NTZ), + Tuple.tuple("meta", TypeInfoFactory.getStructTypeInfo( + Arrays.asList("__message_timestamp", "__kafka_topic", "__kafka_offset"), + Arrays.asList(TypeInfoFactory.TIMESTAMP_NTZ, TypeInfoFactory.STRING, TypeInfoFactory.BIGINT) + )) + ); + assertThat(maxComputeSchema.getTableSchema().getPartitionColumns()) + .extracting("name", "typeInfo") + .contains(Tuple.tuple("__partitioning_column", TypeInfoFactory.STRING)); + } + + @Test + public void shouldBuildTableSchemaWithoutPartitionAndMeta() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.shouldAddMetadata()).thenReturn(Boolean.FALSE); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(Boolean.FALSE); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getMaxPastYearEventTimeDifference()).thenReturn(999); + when(maxComputeSinkConfig.getMaxFutureYearEventTimeDifference()).thenReturn(999); + int expectedNonPartitionColumnCount = 4; + int expectedPartitionColumnCount = 0; + PartitioningStrategy partitioningStrategy = PartitioningStrategyFactory.createPartitioningStrategy( + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeSinkConfig, + descriptor + ); + MaxComputeSchemaHelper maxComputeSchemaHelper = new MaxComputeSchemaHelper(new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeSinkConfig, partitioningStrategy); + + MaxComputeSchema maxComputeSchema = maxComputeSchemaHelper.build(descriptor); + + assertThat(maxComputeSchema.getTableSchema().getColumns().size()).isEqualTo(expectedNonPartitionColumnCount); + assertThat(maxComputeSchema.getTableSchema().getPartitionColumns().size()).isEqualTo(expectedPartitionColumnCount); + assertThat(maxComputeSchema.getTableSchema().getColumns()) + .extracting("name", "typeInfo") + .containsExactlyInAnyOrder( + Tuple.tuple("id", TypeInfoFactory.STRING), + Tuple.tuple("user", TypeInfoFactory.getStructTypeInfo( + Arrays.asList("id", "contacts"), + Arrays.asList(TypeInfoFactory.STRING, TypeInfoFactory.getArrayTypeInfo(TypeInfoFactory.getStructTypeInfo( + Arrays.asList("number"), + Arrays.asList(TypeInfoFactory.STRING) + ))) + )), + Tuple.tuple("items", TypeInfoFactory.getArrayTypeInfo(TypeInfoFactory.getStructTypeInfo( + Arrays.asList("id", "name"), + Arrays.asList(TypeInfoFactory.STRING, TypeInfoFactory.STRING) + ))), + Tuple.tuple("event_timestamp", TypeInfoFactory.TIMESTAMP_NTZ) + ); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIllegalArgumentExceptionWhenPartitionKeyIsNotFound() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.shouldAddMetadata()).thenReturn(Boolean.TRUE); + when(maxComputeSinkConfig.getMetadataColumnsTypes()).thenReturn( + Arrays.asList(new TupleString("__message_timestamp", "timestamp"), + new TupleString("__kafka_topic", "string"), + new TupleString("__kafka_offset", "long") + ) + ); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(Boolean.TRUE); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn("non_existent_partition_key"); + when(maxComputeSinkConfig.getMaxPastYearEventTimeDifference()).thenReturn(999); + when(maxComputeSinkConfig.getMaxFutureYearEventTimeDifference()).thenReturn(999); + PartitioningStrategy partitioningStrategy = PartitioningStrategyFactory.createPartitioningStrategy( + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeSinkConfig, + descriptor + ); + MaxComputeSchemaHelper maxComputeSchemaHelper = new MaxComputeSchemaHelper(new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeSinkConfig, partitioningStrategy); + + maxComputeSchemaHelper.build(descriptor); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/MaxComputeSinkTest.java b/src/test/java/com/gotocompany/depot/maxcompute/MaxComputeSinkTest.java new file mode 100644 index 00000000..afd9937e --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/MaxComputeSinkTest.java @@ -0,0 +1,253 @@ +package com.gotocompany.depot.maxcompute; + +import com.aliyun.odps.data.Record; +import com.aliyun.odps.tunnel.TunnelException; +import com.gotocompany.depot.SinkResponse; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.error.ErrorInfo; +import com.gotocompany.depot.error.ErrorType; +import com.gotocompany.depot.maxcompute.client.MaxComputeClient; +import com.gotocompany.depot.maxcompute.converter.record.MessageRecordConverter; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.maxcompute.model.RecordWrappers; +import com.gotocompany.depot.message.Message; +import com.gotocompany.depot.metrics.Instrumentation; +import com.gotocompany.depot.metrics.MaxComputeMetrics; +import org.junit.Assert; +import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.mockito.Mockito; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.mockito.Mockito.when; + +public class MaxComputeSinkTest { + + @Test + public void shouldInsertMaxComputeSinkTest() throws IOException, TunnelException { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeAccessId()) + .thenReturn("accessId"); + when(maxComputeSinkConfig.getMaxComputeAccessKey()) + .thenReturn("accessKey"); + when(maxComputeSinkConfig.getMaxComputeOdpsUrl()) + .thenReturn("odpsUrl"); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("projectId"); + when(maxComputeSinkConfig.getMaxComputeSchema()) + .thenReturn("schema"); + when(maxComputeSinkConfig.getMaxComputeTunnelUrl()) + .thenReturn("tunnelUrl"); + when(maxComputeSinkConfig.getTableValidatorNameRegex()) + .thenReturn("^[A-Za-z][A-Za-z0-9_]{0,127}$"); + when(maxComputeSinkConfig.getTableValidatorMaxColumnsPerTable()) + .thenReturn(1200); + when(maxComputeSinkConfig.getTableValidatorMaxPartitionKeysPerTable()) + .thenReturn(1); + MaxComputeClient maxComputeClient = Mockito.spy(new MaxComputeClient(maxComputeSinkConfig, Mockito.mock(Instrumentation.class), Mockito.mock(MaxComputeMetrics.class))); + Mockito.doNothing() + .when(maxComputeClient) + .insert(Mockito.anyList()); + MessageRecordConverter messageRecordConverter = Mockito.mock(MessageRecordConverter.class); + MaxComputeSink maxComputeSink = new MaxComputeSink(maxComputeClient, messageRecordConverter, Mockito.mock(Instrumentation.class), Mockito.mock(MaxComputeMetrics.class)); + List messages = Arrays.asList( + new Message("key1".getBytes(StandardCharsets.UTF_8), "message1".getBytes(StandardCharsets.UTF_8)), + new Message("key2".getBytes(StandardCharsets.UTF_8), "invalidMessage2".getBytes(StandardCharsets.UTF_8)) + ); + List validRecords = Collections.singletonList(new RecordWrapper(Mockito.mock(Record.class), 0, null, null)); + List invalidRecords = Collections.singletonList(new RecordWrapper(Mockito.mock(Record.class), 1, + new ErrorInfo(new RuntimeException("Invalid Schema"), ErrorType.DESERIALIZATION_ERROR), null)); + when(messageRecordConverter.convert(messages)).thenReturn(new RecordWrappers(validRecords, invalidRecords)); + + SinkResponse sinkResponse = maxComputeSink.pushToSink(messages); + + Mockito.verify(maxComputeClient, Mockito.times(1)).insert(validRecords); + Assertions.assertEquals(1, sinkResponse.getErrors().size()); + Assertions.assertEquals(sinkResponse.getErrors().get(1L).getException().getMessage(), "Invalid Schema"); + Assertions.assertEquals(sinkResponse.getErrors().get(1L).getErrorType(), ErrorType.DESERIALIZATION_ERROR); + } + + @Test + public void shouldMarkAllMessageAsFailedWhenInsertThrowTunnelExceptionError() throws IOException, TunnelException { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeAccessId()) + .thenReturn("accessId"); + when(maxComputeSinkConfig.getMaxComputeAccessKey()) + .thenReturn("accessKey"); + when(maxComputeSinkConfig.getMaxComputeOdpsUrl()) + .thenReturn("odpsUrl"); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("projectId"); + when(maxComputeSinkConfig.getMaxComputeSchema()) + .thenReturn("schema"); + when(maxComputeSinkConfig.getMaxComputeTunnelUrl()) + .thenReturn("tunnelUrl"); + when(maxComputeSinkConfig.getTableValidatorNameRegex()) + .thenReturn("^[A-Za-z][A-Za-z0-9_]{0,127}$"); + when(maxComputeSinkConfig.getTableValidatorMaxColumnsPerTable()) + .thenReturn(1200); + when(maxComputeSinkConfig.getTableValidatorMaxPartitionKeysPerTable()) + .thenReturn(1); + MaxComputeClient maxComputeClient = Mockito.spy(new MaxComputeClient(maxComputeSinkConfig, Mockito.mock(Instrumentation.class), Mockito.mock(MaxComputeMetrics.class))); + Mockito.doNothing() + .when(maxComputeClient) + .insert(Mockito.anyList()); + MessageRecordConverter messageRecordConverter = Mockito.mock(MessageRecordConverter.class); + Mockito.doThrow(new TunnelException("Failed establishing connection")) + .when(maxComputeClient) + .insert(Mockito.anyList()); + MaxComputeSink maxComputeSink = new MaxComputeSink(maxComputeClient, messageRecordConverter, Mockito.mock(Instrumentation.class), Mockito.mock(MaxComputeMetrics.class)); + List messages = Arrays.asList( + new Message("key1".getBytes(StandardCharsets.UTF_8), "message1".getBytes(StandardCharsets.UTF_8)), + new Message("key2".getBytes(StandardCharsets.UTF_8), "invalidMessage2".getBytes(StandardCharsets.UTF_8)) + ); + List validRecords = Arrays.asList( + new RecordWrapper(Mockito.mock(Record.class), 0, null, null), + new RecordWrapper(Mockito.mock(Record.class), 1, null, null) + ); + when(messageRecordConverter.convert(messages)).thenReturn(new RecordWrappers(validRecords, new ArrayList<>())); + + SinkResponse sinkResponse = maxComputeSink.pushToSink(messages); + + Assertions.assertEquals(2, sinkResponse.getErrors().size()); + Assert.assertTrue(sinkResponse.getErrors() + .values() + .stream() + .allMatch(s -> ErrorType.SINK_RETRYABLE_ERROR.equals(s.getErrorType()))); + } + + @Test + public void shouldMarkAllMessageAsFailedWhenInsertThrowIOExceptionError() throws IOException, TunnelException { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeAccessId()) + .thenReturn("accessId"); + when(maxComputeSinkConfig.getMaxComputeAccessKey()) + .thenReturn("accessKey"); + when(maxComputeSinkConfig.getMaxComputeOdpsUrl()) + .thenReturn("odpsUrl"); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("projectId"); + when(maxComputeSinkConfig.getMaxComputeSchema()) + .thenReturn("schema"); + when(maxComputeSinkConfig.getMaxComputeTunnelUrl()) + .thenReturn("tunnelUrl"); + when(maxComputeSinkConfig.getTableValidatorNameRegex()) + .thenReturn("^[A-Za-z][A-Za-z0-9_]{0,127}$"); + when(maxComputeSinkConfig.getTableValidatorMaxColumnsPerTable()) + .thenReturn(1200); + when(maxComputeSinkConfig.getTableValidatorMaxPartitionKeysPerTable()) + .thenReturn(1); + MaxComputeClient maxComputeClient = Mockito.spy(new MaxComputeClient(maxComputeSinkConfig, Mockito.mock(Instrumentation.class), Mockito.mock(MaxComputeMetrics.class))); + Mockito.doNothing() + .when(maxComputeClient) + .insert(Mockito.anyList()); + MessageRecordConverter messageRecordConverter = Mockito.mock(MessageRecordConverter.class); + Mockito.doThrow(new IOException("Failed flushing")) + .when(maxComputeClient) + .insert(Mockito.anyList()); + MaxComputeSink maxComputeSink = new MaxComputeSink(maxComputeClient, messageRecordConverter, Mockito.mock(Instrumentation.class), Mockito.mock(MaxComputeMetrics.class)); + List messages = Arrays.asList( + new Message("key1".getBytes(StandardCharsets.UTF_8), "message1".getBytes(StandardCharsets.UTF_8)), + new Message("key2".getBytes(StandardCharsets.UTF_8), "invalidMessage2".getBytes(StandardCharsets.UTF_8)) + ); + List validRecords = Arrays.asList( + new RecordWrapper(Mockito.mock(Record.class), 0, null, null), + new RecordWrapper(Mockito.mock(Record.class), 1, null, null) + ); + when(messageRecordConverter.convert(messages)).thenReturn(new RecordWrappers(validRecords, new ArrayList<>())); + + SinkResponse sinkResponse = maxComputeSink.pushToSink(messages); + + Assertions.assertEquals(2, sinkResponse.getErrors().size()); + Assert.assertTrue(sinkResponse.getErrors() + .values() + .stream() + .allMatch(s -> ErrorType.SINK_RETRYABLE_ERROR.equals(s.getErrorType()))); + } + + @Test + public void shouldMarkAllMessageAsFailedWhenInsertThrowExceptionError() throws IOException, TunnelException { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeAccessId()) + .thenReturn("accessId"); + when(maxComputeSinkConfig.getMaxComputeAccessKey()) + .thenReturn("accessKey"); + when(maxComputeSinkConfig.getMaxComputeOdpsUrl()) + .thenReturn("odpsUrl"); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("projectId"); + when(maxComputeSinkConfig.getMaxComputeSchema()) + .thenReturn("schema"); + when(maxComputeSinkConfig.getMaxComputeTunnelUrl()) + .thenReturn("tunnelUrl"); + when(maxComputeSinkConfig.getTableValidatorNameRegex()) + .thenReturn("^[A-Za-z][A-Za-z0-9_]{0,127}$"); + when(maxComputeSinkConfig.getTableValidatorMaxColumnsPerTable()) + .thenReturn(1200); + when(maxComputeSinkConfig.getTableValidatorMaxPartitionKeysPerTable()) + .thenReturn(1); + MaxComputeClient maxComputeClient = Mockito.spy(new MaxComputeClient(maxComputeSinkConfig, Mockito.mock(Instrumentation.class), Mockito.mock(MaxComputeMetrics.class))); + Mockito.doNothing() + .when(maxComputeClient) + .insert(Mockito.anyList()); + MessageRecordConverter messageRecordConverter = Mockito.mock(MessageRecordConverter.class); + Mockito.doThrow(new RuntimeException("Unexpected Error")) + .when(maxComputeClient) + .insert(Mockito.anyList()); + MaxComputeSink maxComputeSink = new MaxComputeSink(maxComputeClient, messageRecordConverter, Mockito.mock(Instrumentation.class), Mockito.mock(MaxComputeMetrics.class)); + List messages = Arrays.asList( + new Message("key1".getBytes(StandardCharsets.UTF_8), "message1".getBytes(StandardCharsets.UTF_8)), + new Message("key2".getBytes(StandardCharsets.UTF_8), "invalidMessage2".getBytes(StandardCharsets.UTF_8)) + ); + List validRecords = Arrays.asList( + new RecordWrapper(Mockito.mock(Record.class), 0, null, null), + new RecordWrapper(Mockito.mock(Record.class), 1, null, null) + ); + when(messageRecordConverter.convert(messages)).thenReturn(new RecordWrappers(validRecords, new ArrayList<>())); + + SinkResponse sinkResponse = maxComputeSink.pushToSink(messages); + + Assertions.assertEquals(2, sinkResponse.getErrors().size()); + Assert.assertTrue(sinkResponse.getErrors() + .values() + .stream() + .allMatch(s -> ErrorType.DEFAULT_ERROR.equals(s.getErrorType()))); + } + + @Test + public void shouldDoNothing() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeAccessId()) + .thenReturn("accessId"); + when(maxComputeSinkConfig.getMaxComputeAccessKey()) + .thenReturn("accessKey"); + when(maxComputeSinkConfig.getMaxComputeOdpsUrl()) + .thenReturn("odpsUrl"); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("projectId"); + when(maxComputeSinkConfig.getMaxComputeSchema()) + .thenReturn("schema"); + when(maxComputeSinkConfig.getMaxComputeTunnelUrl()) + .thenReturn("tunnelUrl"); + when(maxComputeSinkConfig.getTableValidatorNameRegex()) + .thenReturn("^[A-Za-z][A-Za-z0-9_]{0,127}$"); + when(maxComputeSinkConfig.getTableValidatorMaxColumnsPerTable()) + .thenReturn(1200); + when(maxComputeSinkConfig.getTableValidatorMaxPartitionKeysPerTable()) + .thenReturn(1); + MaxComputeClient maxComputeClient = Mockito.spy(new MaxComputeClient(maxComputeSinkConfig, Mockito.mock(Instrumentation.class), Mockito.mock(MaxComputeMetrics.class))); + MessageRecordConverter messageRecordConverter = Mockito.mock(MessageRecordConverter.class); + + MaxComputeSink maxComputeSink = new MaxComputeSink(maxComputeClient, messageRecordConverter, Mockito.mock(Instrumentation.class), Mockito.mock(MaxComputeMetrics.class)); + + assertDoesNotThrow(maxComputeSink::close); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/client/insert/InsertManagerFactoryTest.java b/src/test/java/com/gotocompany/depot/maxcompute/client/insert/InsertManagerFactoryTest.java new file mode 100644 index 00000000..d5415fe3 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/client/insert/InsertManagerFactoryTest.java @@ -0,0 +1,36 @@ +package com.gotocompany.depot.maxcompute.client.insert; + +import com.aliyun.odps.tunnel.TableTunnel; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.metrics.Instrumentation; +import com.gotocompany.depot.metrics.MaxComputeMetrics; +import org.junit.Test; +import org.mockito.Mockito; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class InsertManagerFactoryTest { + + @Test + public void shouldCreatePartitionedInsertManager() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + Mockito.when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(true); + + InsertManager insertManager = InsertManagerFactory.createInsertManager(maxComputeSinkConfig, + Mockito.mock(TableTunnel.class), Mockito.mock(Instrumentation.class), Mockito.mock(MaxComputeMetrics.class)); + + assertTrue(insertManager instanceof PartitionedInsertManager); + } + + @Test + public void shouldCreateNonPartitionedInsertManager() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + Mockito.when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(false); + + InsertManager insertManager = InsertManagerFactory.createInsertManager(maxComputeSinkConfig, + Mockito.mock(TableTunnel.class), Mockito.mock(Instrumentation.class), Mockito.mock(MaxComputeMetrics.class)); + + assertTrue(insertManager instanceof NonPartitionedInsertManager); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/client/insert/NonPartitionedInsertManagerTest.java b/src/test/java/com/gotocompany/depot/maxcompute/client/insert/NonPartitionedInsertManagerTest.java new file mode 100644 index 00000000..27e047a2 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/client/insert/NonPartitionedInsertManagerTest.java @@ -0,0 +1,207 @@ +package com.gotocompany.depot.maxcompute.client.insert; + + +import com.aliyun.odps.tunnel.TableTunnel; +import com.aliyun.odps.tunnel.TunnelException; +import com.aliyun.odps.tunnel.io.CompressOption; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.client.insert.session.StreamingSessionManager; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.metrics.Instrumentation; +import com.gotocompany.depot.metrics.MaxComputeMetrics; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class NonPartitionedInsertManagerTest { + + @Test + public void shouldFlushAllTheRecords() throws IOException, TunnelException { + TableTunnel.FlushResult flushResult = Mockito.mock(TableTunnel.FlushResult.class); + when(flushResult.getRecordCount()) + .thenReturn(2L); + TableTunnel.StreamRecordPack streamRecordPack = Mockito.mock(TableTunnel.StreamRecordPack.class); + TableTunnel.StreamUploadSession streamUploadSession = Mockito.spy(TableTunnel.StreamUploadSession.class); + when(streamRecordPack.flush(Mockito.any(TableTunnel.FlushOption.class))) + .thenReturn(flushResult); + when(streamUploadSession.newRecordPack()) + .thenReturn(streamRecordPack); + when(streamRecordPack.flush()) + .thenReturn("traceId"); + TableTunnel tableTunnel = Mockito.mock(TableTunnel.class); + TableTunnel.StreamUploadSession.Builder builder = Mockito.mock(TableTunnel.StreamUploadSession.Builder.class); + when(tableTunnel.buildStreamUploadSession(Mockito.anyString(), Mockito.anyString())) + .thenReturn(builder); + when(builder.allowSchemaMismatch(Mockito.anyBoolean())) + .thenReturn(builder); + when(builder.build()) + .thenReturn(streamUploadSession); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("project"); + when(maxComputeSinkConfig.getMaxComputeTableName()) + .thenReturn("table"); + when(maxComputeSinkConfig.getMaxComputeRecordPackFlushTimeoutMs()) + .thenReturn(1000L); + when(maxComputeSinkConfig.getStreamingInsertMaximumSessionCount()) + .thenReturn(1); + Instrumentation instrumentation = Mockito.mock(Instrumentation.class); + Mockito.doNothing() + .when(instrumentation) + .captureCount(Mockito.anyString(), Mockito.anyLong()); + MaxComputeMetrics maxComputeMetrics = Mockito.mock(MaxComputeMetrics.class); + when(maxComputeMetrics.getMaxComputeFlushRecordMetric()) + .thenReturn("flush_record"); + when(maxComputeMetrics.getMaxComputeFlushSizeMetric()) + .thenReturn("flush_size"); + StreamingSessionManager streamingSessionManager = StreamingSessionManager.createNonPartitioned( + tableTunnel, maxComputeSinkConfig + ); + NonPartitionedInsertManager nonPartitionedInsertManager = new NonPartitionedInsertManager(maxComputeSinkConfig, instrumentation, maxComputeMetrics, streamingSessionManager); + List recordWrappers = Collections.singletonList( + Mockito.mock(RecordWrapper.class) + ); + + nonPartitionedInsertManager.insert(recordWrappers); + + verify(streamRecordPack, Mockito.times(1)) + .flush(Mockito.any(TableTunnel.FlushOption.class)); + } + + @Test + public void shouldFlushAllTheRecordsWithCompressOption() throws IOException, TunnelException { + TableTunnel.FlushResult flushResult = Mockito.mock(TableTunnel.FlushResult.class); + when(flushResult.getRecordCount()) + .thenReturn(2L); + TableTunnel.StreamRecordPack streamRecordPack = Mockito.mock(TableTunnel.StreamRecordPack.class); + TableTunnel.StreamUploadSession streamUploadSession = Mockito.spy(TableTunnel.StreamUploadSession.class); + when(streamRecordPack.flush(Mockito.any(TableTunnel.FlushOption.class))) + .thenReturn(flushResult); + ArgumentCaptor compressOptionArgumentCaptor = ArgumentCaptor.forClass(CompressOption.class); + when(streamUploadSession.newRecordPack(compressOptionArgumentCaptor.capture())) + .thenReturn(streamRecordPack); + when(streamRecordPack.flush()) + .thenReturn("traceId"); + TableTunnel tableTunnel = Mockito.mock(TableTunnel.class); + TableTunnel.StreamUploadSession.Builder builder = Mockito.mock(TableTunnel.StreamUploadSession.Builder.class); + when(tableTunnel.buildStreamUploadSession(Mockito.anyString(), Mockito.anyString())) + .thenReturn(builder); + when(builder.allowSchemaMismatch(Mockito.anyBoolean())) + .thenReturn(builder); + when(builder.build()) + .thenReturn(streamUploadSession); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("project"); + when(maxComputeSinkConfig.getMaxComputeTableName()) + .thenReturn("table"); + when(maxComputeSinkConfig.getMaxComputeRecordPackFlushTimeoutMs()) + .thenReturn(1000L); + when(maxComputeSinkConfig.isStreamingInsertCompressEnabled()) + .thenReturn(true); + when(maxComputeSinkConfig.getMaxComputeCompressionAlgorithm()) + .thenReturn(CompressOption.CompressAlgorithm.ODPS_RAW); + when(maxComputeSinkConfig.getMaxComputeCompressionLevel()) + .thenReturn(1); + when(maxComputeSinkConfig.getMaxComputeCompressionStrategy()) + .thenReturn(1); + when(maxComputeSinkConfig.getStreamingInsertMaximumSessionCount()) + .thenReturn(1); + Instrumentation instrumentation = Mockito.mock(Instrumentation.class); + Mockito.doNothing() + .when(instrumentation) + .captureCount(Mockito.anyString(), Mockito.anyLong()); + MaxComputeMetrics maxComputeMetrics = Mockito.mock(MaxComputeMetrics.class); + when(maxComputeMetrics.getMaxComputeFlushRecordMetric()) + .thenReturn("flush_record"); + when(maxComputeMetrics.getMaxComputeFlushSizeMetric()) + .thenReturn("flush_size"); + StreamingSessionManager streamingSessionManager = StreamingSessionManager.createNonPartitioned( + tableTunnel, maxComputeSinkConfig + ); + NonPartitionedInsertManager nonPartitionedInsertManager = new NonPartitionedInsertManager(maxComputeSinkConfig, instrumentation, maxComputeMetrics, streamingSessionManager); + List recordWrappers = Collections.singletonList( + Mockito.mock(RecordWrapper.class) + ); + + nonPartitionedInsertManager.insert(recordWrappers); + + verify(streamRecordPack, Mockito.times(1)) + .flush(Mockito.any(TableTunnel.FlushOption.class)); + assertEquals(compressOptionArgumentCaptor.getValue().algorithm, CompressOption.CompressAlgorithm.ODPS_RAW); + assertEquals(compressOptionArgumentCaptor.getValue().strategy, 1); + assertEquals(compressOptionArgumentCaptor.getValue().level, 1); + } + + @Test(expected = IOException.class) + public void shouldRefreshSessionWhenIOExceptionOccurred() throws IOException, TunnelException { + TableTunnel.FlushResult flushResult = Mockito.mock(TableTunnel.FlushResult.class); + when(flushResult.getRecordCount()) + .thenReturn(2L); + TableTunnel.StreamRecordPack streamRecordPack = Mockito.mock(TableTunnel.StreamRecordPack.class); + TableTunnel.StreamUploadSession streamUploadSession = Mockito.spy(TableTunnel.StreamUploadSession.class); + when(streamRecordPack.flush(Mockito.any(TableTunnel.FlushOption.class))) + .thenReturn(flushResult); + ArgumentCaptor compressOptionArgumentCaptor = ArgumentCaptor.forClass(CompressOption.class); + when(streamUploadSession.newRecordPack(compressOptionArgumentCaptor.capture())) + .thenReturn(streamRecordPack); + Mockito.doThrow(IOException.class) + .when(streamRecordPack) + .append(Mockito.any()); + TableTunnel tableTunnel = Mockito.mock(TableTunnel.class); + TableTunnel.StreamUploadSession.Builder builder = Mockito.mock(TableTunnel.StreamUploadSession.Builder.class); + when(tableTunnel.buildStreamUploadSession(Mockito.anyString(), Mockito.anyString())) + .thenReturn(builder); + when(builder.allowSchemaMismatch(Mockito.anyBoolean())) + .thenReturn(builder); + when(builder.build()) + .thenReturn(streamUploadSession); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("project"); + when(maxComputeSinkConfig.getMaxComputeTableName()) + .thenReturn("table"); + when(maxComputeSinkConfig.getMaxComputeRecordPackFlushTimeoutMs()) + .thenReturn(1000L); + when(maxComputeSinkConfig.isStreamingInsertCompressEnabled()) + .thenReturn(true); + when(maxComputeSinkConfig.getMaxComputeCompressionAlgorithm()) + .thenReturn(CompressOption.CompressAlgorithm.ODPS_RAW); + when(maxComputeSinkConfig.getMaxComputeCompressionLevel()) + .thenReturn(1); + when(maxComputeSinkConfig.getMaxComputeCompressionStrategy()) + .thenReturn(1); + when(maxComputeSinkConfig.getStreamingInsertMaximumSessionCount()) + .thenReturn(1); + Instrumentation instrumentation = Mockito.mock(Instrumentation.class); + Mockito.doNothing() + .when(instrumentation) + .captureCount(Mockito.anyString(), Mockito.anyLong()); + MaxComputeMetrics maxComputeMetrics = Mockito.mock(MaxComputeMetrics.class); + when(maxComputeMetrics.getMaxComputeFlushRecordMetric()) + .thenReturn("flush_record"); + when(maxComputeMetrics.getMaxComputeFlushSizeMetric()) + .thenReturn("flush_size"); + StreamingSessionManager streamingSessionManager = Mockito.spy(StreamingSessionManager.createNonPartitioned( + tableTunnel, maxComputeSinkConfig + )); + NonPartitionedInsertManager nonPartitionedInsertManager = new NonPartitionedInsertManager(maxComputeSinkConfig, instrumentation, maxComputeMetrics, streamingSessionManager); + List recordWrappers = Collections.singletonList( + Mockito.mock(RecordWrapper.class) + ); + + nonPartitionedInsertManager.insert(recordWrappers); + + verify(streamingSessionManager, Mockito.times(1)) + .refreshSession(Mockito.anyString()); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/client/insert/PartitionedInsertManagerTest.java b/src/test/java/com/gotocompany/depot/maxcompute/client/insert/PartitionedInsertManagerTest.java new file mode 100644 index 00000000..315ae1a0 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/client/insert/PartitionedInsertManagerTest.java @@ -0,0 +1,231 @@ +package com.gotocompany.depot.maxcompute.client.insert; + +import com.aliyun.odps.PartitionSpec; +import com.aliyun.odps.tunnel.TableTunnel; +import com.aliyun.odps.tunnel.TunnelException; +import com.aliyun.odps.tunnel.io.CompressOption; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.client.insert.session.StreamingSessionManager; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.metrics.Instrumentation; +import com.gotocompany.depot.metrics.MaxComputeMetrics; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class PartitionedInsertManagerTest { + + @Test + public void shouldGroupRecordsBasedOnPartitionSpecAndFlushAll() throws IOException, TunnelException { + TableTunnel.FlushResult flushResult = Mockito.mock(TableTunnel.FlushResult.class); + when(flushResult.getRecordCount()) + .thenReturn(2L); + TableTunnel.StreamRecordPack streamRecordPack = Mockito.mock(TableTunnel.StreamRecordPack.class); + TableTunnel.StreamUploadSession streamUploadSession = Mockito.spy(TableTunnel.StreamUploadSession.class); + when(streamRecordPack.flush(Mockito.any(TableTunnel.FlushOption.class))) + .thenReturn(flushResult); + when(streamUploadSession.newRecordPack()) + .thenReturn(streamRecordPack); + when(streamRecordPack.flush()) + .thenReturn("traceId"); + TableTunnel tableTunnel = Mockito.mock(TableTunnel.class); + TableTunnel.StreamUploadSession.Builder builder = Mockito.mock(TableTunnel.StreamUploadSession.Builder.class); + when(tableTunnel.buildStreamUploadSession(Mockito.anyString(), Mockito.anyString())) + .thenReturn(builder); + when(builder.setCreatePartition(Mockito.anyBoolean())) + .thenReturn(builder); + when(builder.setPartitionSpec(Mockito.anyString())) + .thenReturn(builder); + when(builder.allowSchemaMismatch(Mockito.anyBoolean())) + .thenReturn(builder); + when(builder.build()) + .thenReturn(streamUploadSession); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("project"); + when(maxComputeSinkConfig.getMaxComputeTableName()) + .thenReturn("table"); + when(maxComputeSinkConfig.getMaxComputeRecordPackFlushTimeoutMs()) + .thenReturn(1000L); + when(maxComputeSinkConfig.getStreamingInsertMaximumSessionCount()) + .thenReturn(1); + RecordWrapper firstPartitionRecordWrapper = Mockito.mock(RecordWrapper.class); + when(firstPartitionRecordWrapper.getPartitionSpec()) + .thenReturn(new PartitionSpec("ds=1")); + RecordWrapper secondPartitionRecordWrapper = Mockito.mock(RecordWrapper.class); + when(secondPartitionRecordWrapper.getPartitionSpec()) + .thenReturn(new PartitionSpec("ds=2")); + List recordWrappers = Arrays.asList( + firstPartitionRecordWrapper, + secondPartitionRecordWrapper + ); + Instrumentation instrumentation = Mockito.mock(Instrumentation.class); + Mockito.doNothing() + .when(instrumentation) + .captureCount(Mockito.anyString(), Mockito.anyLong()); + StreamingSessionManager streamingSessionManager = StreamingSessionManager.createPartitioned( + tableTunnel, maxComputeSinkConfig + ); + PartitionedInsertManager partitionedInsertManager = new PartitionedInsertManager(maxComputeSinkConfig, instrumentation, Mockito.mock(MaxComputeMetrics.class), streamingSessionManager); + int expectedPartitionFlushInvocation = 2; + + partitionedInsertManager.insert(recordWrappers); + + verify(streamRecordPack, Mockito.times(expectedPartitionFlushInvocation)) + .flush(Mockito.any(TableTunnel.FlushOption.class)); + } + + @Test + public void shouldGroupRecordsBasedOnPartitionSpecAndFlushAllWithCompression() throws IOException, TunnelException { + TableTunnel.FlushResult flushResult = Mockito.mock(TableTunnel.FlushResult.class); + when(flushResult.getRecordCount()) + .thenReturn(2L); + TableTunnel.StreamRecordPack streamRecordPack = Mockito.mock(TableTunnel.StreamRecordPack.class); + TableTunnel.StreamUploadSession streamUploadSession = Mockito.spy(TableTunnel.StreamUploadSession.class); + when(streamRecordPack.flush(Mockito.any(TableTunnel.FlushOption.class))) + .thenReturn(flushResult); + ArgumentCaptor compressOptionArgumentCaptor = ArgumentCaptor.forClass(CompressOption.class); + when(streamUploadSession.newRecordPack(compressOptionArgumentCaptor.capture())) + .thenReturn(streamRecordPack); + when(streamRecordPack.flush()) + .thenReturn("traceId"); + TableTunnel tableTunnel = Mockito.mock(TableTunnel.class); + TableTunnel.StreamUploadSession.Builder builder = Mockito.mock(TableTunnel.StreamUploadSession.Builder.class); + when(tableTunnel.buildStreamUploadSession(Mockito.anyString(), Mockito.anyString())) + .thenReturn(builder); + when(builder.setCreatePartition(Mockito.anyBoolean())) + .thenReturn(builder); + when(builder.setPartitionSpec(Mockito.anyString())) + .thenReturn(builder); + when(builder.allowSchemaMismatch(Mockito.anyBoolean())) + .thenReturn(builder); + when(builder.build()) + .thenReturn(streamUploadSession); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("project"); + when(maxComputeSinkConfig.getMaxComputeTableName()) + .thenReturn("table"); + when(maxComputeSinkConfig.getMaxComputeRecordPackFlushTimeoutMs()) + .thenReturn(1000L); + when(maxComputeSinkConfig.isStreamingInsertCompressEnabled()) + .thenReturn(true); + when(maxComputeSinkConfig.getMaxComputeCompressionAlgorithm()) + .thenReturn(CompressOption.CompressAlgorithm.ODPS_RAW); + when(maxComputeSinkConfig.getMaxComputeCompressionLevel()) + .thenReturn(1); + when(maxComputeSinkConfig.getMaxComputeCompressionStrategy()) + .thenReturn(1); + when(maxComputeSinkConfig.getStreamingInsertMaximumSessionCount()) + .thenReturn(1); + RecordWrapper firstPartitionRecordWrapper = Mockito.mock(RecordWrapper.class); + when(firstPartitionRecordWrapper.getPartitionSpec()) + .thenReturn(new PartitionSpec("ds=1")); + RecordWrapper secondPartitionRecordWrapper = Mockito.mock(RecordWrapper.class); + when(secondPartitionRecordWrapper.getPartitionSpec()) + .thenReturn(new PartitionSpec("ds=2")); + List recordWrappers = Arrays.asList( + firstPartitionRecordWrapper, + secondPartitionRecordWrapper + ); + Instrumentation instrumentation = Mockito.mock(Instrumentation.class); + Mockito.doNothing() + .when(instrumentation) + .captureCount(Mockito.anyString(), Mockito.anyLong()); + StreamingSessionManager streamingSessionManager = StreamingSessionManager.createPartitioned( + tableTunnel, maxComputeSinkConfig + ); + PartitionedInsertManager partitionedInsertManager = new PartitionedInsertManager(maxComputeSinkConfig, + instrumentation, Mockito.mock(MaxComputeMetrics.class), streamingSessionManager); + int expectedPartitionFlushInvocation = 2; + + partitionedInsertManager.insert(recordWrappers); + + assertEquals(compressOptionArgumentCaptor.getValue().algorithm, CompressOption.CompressAlgorithm.ODPS_RAW); + assertEquals(compressOptionArgumentCaptor.getValue().level, 1); + assertEquals(compressOptionArgumentCaptor.getValue().strategy, 1); + verify(streamRecordPack, Mockito.times(expectedPartitionFlushInvocation)) + .flush(Mockito.any(TableTunnel.FlushOption.class)); + } + + @Test(expected = IOException.class) + public void shouldRefreshSessionWhenIOExceptionOccurred() throws IOException, TunnelException { + TableTunnel.FlushResult flushResult = Mockito.mock(TableTunnel.FlushResult.class); + when(flushResult.getRecordCount()) + .thenReturn(2L); + TableTunnel.StreamRecordPack streamRecordPack = Mockito.mock(TableTunnel.StreamRecordPack.class); + TableTunnel.StreamUploadSession streamUploadSession = Mockito.spy(TableTunnel.StreamUploadSession.class); + when(streamRecordPack.flush(Mockito.any(TableTunnel.FlushOption.class))) + .thenReturn(flushResult); + ArgumentCaptor compressOptionArgumentCaptor = ArgumentCaptor.forClass(CompressOption.class); + when(streamUploadSession.newRecordPack(compressOptionArgumentCaptor.capture())) + .thenReturn(streamRecordPack); + Mockito.doThrow(new IOException()) + .when(streamRecordPack) + .append(Mockito.any()); + when(streamRecordPack.flush()) + .thenReturn("traceId"); + TableTunnel tableTunnel = Mockito.mock(TableTunnel.class); + TableTunnel.StreamUploadSession.Builder builder = Mockito.mock(TableTunnel.StreamUploadSession.Builder.class); + when(tableTunnel.buildStreamUploadSession(Mockito.anyString(), Mockito.anyString())) + .thenReturn(builder); + when(builder.setCreatePartition(Mockito.anyBoolean())) + .thenReturn(builder); + when(builder.setPartitionSpec(Mockito.anyString())) + .thenReturn(builder); + when(builder.allowSchemaMismatch(Mockito.anyBoolean())) + .thenReturn(builder); + when(builder.build()) + .thenReturn(streamUploadSession); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("project"); + when(maxComputeSinkConfig.getMaxComputeTableName()) + .thenReturn("table"); + when(maxComputeSinkConfig.getMaxComputeRecordPackFlushTimeoutMs()) + .thenReturn(1000L); + when(maxComputeSinkConfig.isStreamingInsertCompressEnabled()) + .thenReturn(true); + when(maxComputeSinkConfig.getMaxComputeCompressionAlgorithm()) + .thenReturn(CompressOption.CompressAlgorithm.ODPS_RAW); + when(maxComputeSinkConfig.getMaxComputeCompressionLevel()) + .thenReturn(1); + when(maxComputeSinkConfig.getMaxComputeCompressionStrategy()) + .thenReturn(1); + when(maxComputeSinkConfig.getStreamingInsertMaximumSessionCount()) + .thenReturn(1); + RecordWrapper firstPartitionRecordWrapper = Mockito.mock(RecordWrapper.class); + when(firstPartitionRecordWrapper.getPartitionSpec()) + .thenReturn(new PartitionSpec("ds=1")); + RecordWrapper secondPartitionRecordWrapper = Mockito.mock(RecordWrapper.class); + when(secondPartitionRecordWrapper.getPartitionSpec()) + .thenReturn(new PartitionSpec("ds=2")); + List recordWrappers = Arrays.asList( + firstPartitionRecordWrapper, + secondPartitionRecordWrapper + ); + Instrumentation instrumentation = Mockito.mock(Instrumentation.class); + Mockito.doNothing() + .when(instrumentation) + .captureCount(Mockito.anyString(), Mockito.anyLong()); + StreamingSessionManager streamingSessionManager = Mockito.spy(StreamingSessionManager.createPartitioned( + tableTunnel, maxComputeSinkConfig + )); + PartitionedInsertManager partitionedInsertManager = new PartitionedInsertManager(maxComputeSinkConfig, + instrumentation, Mockito.mock(MaxComputeMetrics.class), streamingSessionManager); + + partitionedInsertManager.insert(recordWrappers); + + verify(streamingSessionManager, Mockito.times(1)) + .refreshSession(Mockito.anyString()); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/client/insert/session/StreamingSessionManagerTest.java b/src/test/java/com/gotocompany/depot/maxcompute/client/insert/session/StreamingSessionManagerTest.java new file mode 100644 index 00000000..d9c899e4 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/client/insert/session/StreamingSessionManagerTest.java @@ -0,0 +1,207 @@ +package com.gotocompany.depot.maxcompute.client.insert.session; + +import com.aliyun.odps.tunnel.TableTunnel; +import com.aliyun.odps.tunnel.TunnelException; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import org.junit.Test; +import org.mockito.Mockito; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class StreamingSessionManagerTest { + + @Test + public void shouldCreateNewPartitionedSessionIfCacheIsEmpty() throws TunnelException { + TableTunnel tableTunnel = Mockito.mock(TableTunnel.class); + TableTunnel.StreamUploadSession.Builder builder = Mockito.mock(TableTunnel.StreamUploadSession.Builder.class); + when(tableTunnel.buildStreamUploadSession(Mockito.anyString(), Mockito.anyString())) + .thenReturn(builder); + when(builder.setCreatePartition(Mockito.anyBoolean())) + .thenReturn(builder); + when(builder.setPartitionSpec(Mockito.anyString())) + .thenReturn(builder); + when(builder.allowSchemaMismatch(Mockito.anyBoolean())) + .thenReturn(builder); + TableTunnel.StreamUploadSession streamUploadSessionMock = Mockito.mock(TableTunnel.StreamUploadSession.class); + when(builder.build()) + .thenReturn(streamUploadSessionMock); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("test_project"); + when(maxComputeSinkConfig.getMaxComputeTableName()) + .thenReturn("test_table"); + when(maxComputeSinkConfig.getStreamingInsertMaximumSessionCount()) + .thenReturn(1); + StreamingSessionManager partitionedStreamingSessionManager = + StreamingSessionManager.createPartitioned(tableTunnel, maxComputeSinkConfig); + + TableTunnel.StreamUploadSession streamUploadSession = + partitionedStreamingSessionManager.getSession("test_session"); + + verify(tableTunnel, Mockito.times(1)) + .buildStreamUploadSession("test_project", "test_table"); + assertEquals(streamUploadSessionMock, streamUploadSession); + } + + @Test + public void shouldReturnSameInstanceIfCacheNotEmpty() throws TunnelException { + TableTunnel tableTunnel = Mockito.mock(TableTunnel.class); + TableTunnel.StreamUploadSession.Builder builder = Mockito.mock(TableTunnel.StreamUploadSession.Builder.class); + when(tableTunnel.buildStreamUploadSession(Mockito.anyString(), Mockito.anyString())) + .thenReturn(builder); + when(builder.setCreatePartition(Mockito.anyBoolean())) + .thenReturn(builder); + when(builder.setPartitionSpec(Mockito.anyString())) + .thenReturn(builder); + when(builder.allowSchemaMismatch(Mockito.anyBoolean())) + .thenReturn(builder); + TableTunnel.StreamUploadSession streamUploadSessionMock = Mockito.mock(TableTunnel.StreamUploadSession.class); + when(builder.build()) + .thenReturn(streamUploadSessionMock); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("test_project"); + when(maxComputeSinkConfig.getMaxComputeTableName()) + .thenReturn("test_table"); + when(maxComputeSinkConfig.getStreamingInsertMaximumSessionCount()) + .thenReturn(1); + StreamingSessionManager partitionedStreamingSessionManager = + StreamingSessionManager.createPartitioned(tableTunnel, maxComputeSinkConfig); + + TableTunnel.StreamUploadSession streamUploadSession = + partitionedStreamingSessionManager.getSession("test_session"); + TableTunnel.StreamUploadSession secondStreamUploadSession = + partitionedStreamingSessionManager.getSession("test_session"); + + verify(tableTunnel, Mockito.times(1)) + .buildStreamUploadSession("test_project", "test_table"); + assertEquals(streamUploadSessionMock, streamUploadSession); + assertEquals(streamUploadSession, secondStreamUploadSession); + } + + @Test + public void shouldEvictOldLatestInstanceWhenCapacityExceeded() throws TunnelException { + TableTunnel tableTunnel = Mockito.mock(TableTunnel.class); + TableTunnel.StreamUploadSession.Builder builder = Mockito.mock(TableTunnel.StreamUploadSession.Builder.class); + when(tableTunnel.buildStreamUploadSession(Mockito.anyString(), Mockito.anyString())) + .thenReturn(builder); + when(builder.setCreatePartition(Mockito.anyBoolean())) + .thenReturn(builder); + when(builder.setPartitionSpec(Mockito.anyString())) + .thenReturn(builder); + when(builder.allowSchemaMismatch(Mockito.anyBoolean())) + .thenReturn(builder); + TableTunnel.StreamUploadSession streamUploadSessionMock = Mockito.mock(TableTunnel.StreamUploadSession.class); + when(builder.build()) + .thenReturn(streamUploadSessionMock); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("test_project"); + when(maxComputeSinkConfig.getMaxComputeTableName()) + .thenReturn("test_table"); + when(maxComputeSinkConfig.getStreamingInsertMaximumSessionCount()) + .thenReturn(1); + StreamingSessionManager partitionedStreamingSessionManager = + StreamingSessionManager.createPartitioned(tableTunnel, maxComputeSinkConfig); + + TableTunnel.StreamUploadSession streamUploadSession = + partitionedStreamingSessionManager.getSession("test_session"); + TableTunnel.StreamUploadSession secondStreamUploadSession = + partitionedStreamingSessionManager.getSession("different_test_session"); + + verify(tableTunnel, Mockito.times(2)) + .buildStreamUploadSession("test_project", "test_table"); + assertEquals(streamUploadSessionMock, streamUploadSession); + assertEquals(streamUploadSession, secondStreamUploadSession); + } + + @Test + public void shouldCreateNewNonPartitionedSessionIfCacheIsEmpty() throws TunnelException { + TableTunnel tableTunnel = Mockito.mock(TableTunnel.class); + TableTunnel.StreamUploadSession.Builder builder = Mockito.mock(TableTunnel.StreamUploadSession.Builder.class); + when(tableTunnel.buildStreamUploadSession(Mockito.anyString(), Mockito.anyString())) + .thenReturn(builder); + when(builder.allowSchemaMismatch(Mockito.anyBoolean())) + .thenReturn(builder); + TableTunnel.StreamUploadSession streamUploadSessionMock = Mockito.mock(TableTunnel.StreamUploadSession.class); + when(builder.build()) + .thenReturn(streamUploadSessionMock); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("test_project"); + when(maxComputeSinkConfig.getMaxComputeTableName()) + .thenReturn("test_table"); + StreamingSessionManager nonPartitionedStreamingSessionManager = + StreamingSessionManager.createNonPartitioned(tableTunnel, maxComputeSinkConfig); + + TableTunnel.StreamUploadSession streamUploadSession = + nonPartitionedStreamingSessionManager.getSession("test_session"); + + verify(tableTunnel, Mockito.times(1)) + .buildStreamUploadSession("test_project", "test_table"); + assertEquals(streamUploadSessionMock, streamUploadSession); + } + + @Test + public void shouldReturnSameNonPartitionedSessionIfCacheNotEmpty() throws TunnelException { + TableTunnel tableTunnel = Mockito.mock(TableTunnel.class); + TableTunnel.StreamUploadSession.Builder builder = Mockito.mock(TableTunnel.StreamUploadSession.Builder.class); + when(tableTunnel.buildStreamUploadSession(Mockito.anyString(), Mockito.anyString())) + .thenReturn(builder); + when(builder.allowSchemaMismatch(Mockito.anyBoolean())) + .thenReturn(builder); + TableTunnel.StreamUploadSession streamUploadSessionMock = Mockito.mock(TableTunnel.StreamUploadSession.class); + when(builder.build()) + .thenReturn(streamUploadSessionMock); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("test_project"); + when(maxComputeSinkConfig.getMaxComputeTableName()) + .thenReturn("test_table"); + when(maxComputeSinkConfig.getStreamingInsertMaximumSessionCount()) + .thenReturn(1); + StreamingSessionManager nonPartitionedStreamingSessionManager = + StreamingSessionManager.createNonPartitioned(tableTunnel, maxComputeSinkConfig); + + TableTunnel.StreamUploadSession streamUploadSession = + nonPartitionedStreamingSessionManager.getSession("test_session"); + TableTunnel.StreamUploadSession secondStreamUploadSession = + nonPartitionedStreamingSessionManager.getSession("test_session"); + + verify(tableTunnel, Mockito.times(1)) + .buildStreamUploadSession("test_project", "test_table"); + assertEquals(streamUploadSessionMock, streamUploadSession); + assertEquals(streamUploadSession, secondStreamUploadSession); + } + + @Test + public void shouldReturnRefreshTheSession() throws TunnelException { + TableTunnel tableTunnel = Mockito.mock(TableTunnel.class); + TableTunnel.StreamUploadSession.Builder builder = Mockito.mock(TableTunnel.StreamUploadSession.Builder.class); + when(tableTunnel.buildStreamUploadSession(Mockito.anyString(), Mockito.anyString())) + .thenReturn(builder); + when(builder.allowSchemaMismatch(Mockito.anyBoolean())) + .thenReturn(builder); + TableTunnel.StreamUploadSession streamUploadSessionMock = Mockito.mock(TableTunnel.StreamUploadSession.class); + when(builder.build()) + .thenReturn(streamUploadSessionMock); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxComputeProjectId()) + .thenReturn("test_project"); + when(maxComputeSinkConfig.getMaxComputeTableName()) + .thenReturn("test_table"); + when(maxComputeSinkConfig.getStreamingInsertMaximumSessionCount()) + .thenReturn(1); + StreamingSessionManager nonPartitionedStreamingSessionManager = + StreamingSessionManager.createNonPartitioned(tableTunnel, maxComputeSinkConfig); + + nonPartitionedStreamingSessionManager.getSession("test_session"); + nonPartitionedStreamingSessionManager.refreshSession("test_session"); + + verify(tableTunnel, Mockito.times(2)) + .buildStreamUploadSession("test_project", "test_table"); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/converter/ProtobufConverterOrchestratorTest.java b/src/test/java/com/gotocompany/depot/maxcompute/converter/ProtobufConverterOrchestratorTest.java new file mode 100644 index 00000000..ba24deb8 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/converter/ProtobufConverterOrchestratorTest.java @@ -0,0 +1,140 @@ +package com.gotocompany.depot.maxcompute.converter; + +import com.aliyun.odps.data.SimpleStruct; +import com.aliyun.odps.type.StructTypeInfo; +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Duration; +import com.google.protobuf.Message; +import com.google.protobuf.Struct; +import com.google.protobuf.Timestamp; +import com.google.protobuf.Value; + +import com.gotocompany.depot.TestMaxComputeTypeInfo; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.lang.reflect.Field; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.when; + +public class ProtobufConverterOrchestratorTest { + + private final Descriptors.Descriptor descriptor = TestMaxComputeTypeInfo.TestRoot.getDescriptor(); + private ProtobufConverterOrchestrator protobufConverterOrchestrator; + + @Before + public void init() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + Mockito.when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:00", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("9999-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + protobufConverterOrchestrator = new ProtobufConverterOrchestrator(maxComputeSinkConfig); + } + + @Test + public void shouldConvertPayloadToTypeInfo() { + String expectedStringTypeInfoRepresentation = "STRING"; + String expectedMessageTypeRepresentation = "STRUCT<`string_field`:STRING,`another_inner_field`:STRUCT<`string_field`:STRING>,`another_inner_list_field`:ARRAY>>"; + String expectedRepeatedMessageTypeRepresentation = String.format("ARRAY<%s>", expectedMessageTypeRepresentation); + String expectedTimestampTypeInfoRepresentation = "TIMESTAMP_NTZ"; + String expectedDurationTypeInfoRepresentation = "STRUCT<`seconds`:BIGINT,`nanos`:INT>"; + String expectedStructTypeInfoRepresentation = "STRING"; + + TypeInfo stringTypeInfo = protobufConverterOrchestrator.convert(descriptor.findFieldByName("string_field")); + TypeInfo messageTypeInfo = protobufConverterOrchestrator.convert(descriptor.findFieldByName("inner_field")); + TypeInfo repeatedTypeInfo = protobufConverterOrchestrator.convert(descriptor.findFieldByName("inner_list_field")); + TypeInfo timestampTypeInfo = protobufConverterOrchestrator.convert(descriptor.findFieldByName("timestamp_field")); + TypeInfo durationTypeInfo = protobufConverterOrchestrator.convert(descriptor.findFieldByName("duration_field")); + TypeInfo structTypeInfo = protobufConverterOrchestrator.convert(descriptor.findFieldByName("struct_field")); + + assertEquals(expectedStringTypeInfoRepresentation, stringTypeInfo.toString()); + assertEquals(expectedMessageTypeRepresentation, messageTypeInfo.toString()); + assertEquals(expectedRepeatedMessageTypeRepresentation, repeatedTypeInfo.toString()); + assertEquals(expectedTimestampTypeInfoRepresentation, timestampTypeInfo.toString()); + assertEquals(expectedDurationTypeInfoRepresentation, durationTypeInfo.toString()); + assertEquals(expectedStructTypeInfoRepresentation, structTypeInfo.toString()); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIllegalArgumentExceptionForUnsupportedType() { + Descriptors.FieldDescriptor unsupportedFieldDescriptor = descriptor.findFieldByName("empty_field"); + protobufConverterOrchestrator.convert(unsupportedFieldDescriptor); + } + + @Test + public void shouldConvertPayloadToRecord() { + Struct.Builder structBuilder = Struct.newBuilder(); + structBuilder.putFields("intField", Value.newBuilder().setNumberValue(1.0).build()); + structBuilder.putFields("stringField", Value.newBuilder().setStringValue("String").build()); + TestMaxComputeTypeInfo.TestAnotherInner testAnotherInner = TestMaxComputeTypeInfo.TestAnotherInner.newBuilder() + .setStringField("inner_string_field") + .build(); + TestMaxComputeTypeInfo.TestInner testInner = TestMaxComputeTypeInfo.TestInner.newBuilder() + .setAnotherInnerField(testAnotherInner) + .addAllAnotherInnerListField(Collections.singletonList(testAnotherInner)) + .setStringField("string_field") + .build(); + Message messagePayload = TestMaxComputeTypeInfo.TestRoot.newBuilder() + .setStringField("string_field") + .setTimestampField(Timestamp.newBuilder() + .setSeconds(100) + .setNanos(0) + .build()) + .setDurationField(Duration.newBuilder() + .setSeconds(100) + .setNanos(0) + .build()) + .setStructField(structBuilder.build()) + .setInnerField(testInner) + .addAllInnerListField(Collections.singletonList(testInner)) + .build(); + StructTypeInfo messageTypeInfo = TypeInfoFactory.getStructTypeInfo( + Arrays.asList("string_field", "another_inner_field", "another_inner_list_field"), + Arrays.asList(TypeInfoFactory.STRING, TypeInfoFactory.getStructTypeInfo(Collections.singletonList("string_field"), Collections.singletonList(TypeInfoFactory.STRING)), + TypeInfoFactory.getArrayTypeInfo(TypeInfoFactory.getStructTypeInfo(Collections.singletonList("string_field"), Collections.singletonList(TypeInfoFactory.STRING)))) + ); + List messageValues = Arrays.asList("string_field", new SimpleStruct(TypeInfoFactory.getStructTypeInfo(Collections.singletonList("string_field"), Collections.singletonList(TypeInfoFactory.STRING)), Collections.singletonList("inner_string_field")), + Collections.singletonList(new SimpleStruct(TypeInfoFactory.getStructTypeInfo(Collections.singletonList("string_field"), Collections.singletonList(TypeInfoFactory.STRING)), Collections.singletonList("inner_string_field")))); + SimpleStruct expectedMessage = new SimpleStruct(messageTypeInfo, messageValues); + + Object stringRecord = protobufConverterOrchestrator.convert(descriptor.findFieldByName("string_field"), messagePayload.getField(descriptor.findFieldByName("string_field"))); + Object messageRecord = protobufConverterOrchestrator.convert(descriptor.findFieldByName("inner_field"), messagePayload.getField(descriptor.findFieldByName("inner_field"))); + Object repeatedMessageRecord = protobufConverterOrchestrator.convert(descriptor.findFieldByName("inner_list_field"), messagePayload.getField(descriptor.findFieldByName("inner_list_field"))); + Object timestampRecord = protobufConverterOrchestrator.convert(descriptor.findFieldByName("timestamp_field"), messagePayload.getField(descriptor.findFieldByName("timestamp_field"))); + Object durationRecord = protobufConverterOrchestrator.convert(descriptor.findFieldByName("duration_field"), messagePayload.getField(descriptor.findFieldByName("duration_field"))); + Object structRecord = protobufConverterOrchestrator.convert(descriptor.findFieldByName("struct_field"), messagePayload.getField(descriptor.findFieldByName("struct_field"))); + + assertEquals("string_field", stringRecord); + assertEquals(LocalDateTime.ofEpochSecond(100, 0, ZoneOffset.UTC), timestampRecord); + assertEquals(new SimpleStruct(TypeInfoFactory.getStructTypeInfo(Arrays.asList("seconds", "nanos"), Arrays.asList(TypeInfoFactory.BIGINT, TypeInfoFactory.INT)), Arrays.asList(100L, 0)), durationRecord); + assertEquals(expectedMessage, messageRecord); + assertEquals(Collections.singletonList(expectedMessage), repeatedMessageRecord); + assertEquals("{\"intField\":1.0,\"stringField\":\"String\"}", structRecord); + } + + @Test + public void shouldClearTheTypeInfoCache() throws NoSuchFieldException, IllegalAccessException { + protobufConverterOrchestrator.convert(descriptor.findFieldByName("inner_list_field")); + Field field = protobufConverterOrchestrator.getClass() + .getDeclaredField("typeInfoCache"); + field.setAccessible(true); + assertEquals(1, ((Map) field.get(protobufConverterOrchestrator)).size()); + + protobufConverterOrchestrator.clearCache(); + + assertEquals(0, ((Map) field.get(protobufConverterOrchestrator)).size()); + } +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/DurationProtobufPayloadConverterTest.java b/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/DurationProtobufPayloadConverterTest.java new file mode 100644 index 00000000..46a30676 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/DurationProtobufPayloadConverterTest.java @@ -0,0 +1,77 @@ +package com.gotocompany.depot.maxcompute.converter.payload; + +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Duration; +import com.gotocompany.depot.TestMaxComputeTypeInfo; +import com.gotocompany.depot.maxcompute.converter.type.DurationProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.model.ProtoPayload; +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +public class DurationProtobufPayloadConverterTest { + + private final DurationProtobufTypeInfoConverter durationTypeInfoConverter = new DurationProtobufTypeInfoConverter(); + private final DurationProtobufPayloadConverter durationPayloadConverter = new DurationProtobufPayloadConverter(durationTypeInfoConverter); + private final Descriptors.Descriptor descriptor = TestMaxComputeTypeInfo.TestRoot.getDescriptor(); + private final Descriptors.Descriptor repeatedDescriptor = TestMaxComputeTypeInfo.TestRootRepeated.getDescriptor(); + + @Test + public void shouldConvertDurationPayloadToStruct() { + Duration duration = Duration.newBuilder() + .setSeconds(1) + .setNanos(1) + .build(); + TestMaxComputeTypeInfo.TestRoot message = TestMaxComputeTypeInfo.TestRoot.newBuilder() + .setDurationField(duration) + .build(); + List expectedFieldNames = Arrays.asList("seconds", "nanos"); + List expectedTypeInfos = Arrays.asList(TypeInfoFactory.BIGINT, TypeInfoFactory.INT); + List values = Arrays.asList(1L, 1); + Object result = durationPayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(5), message.getField(descriptor.getFields().get(5)), true)); + + assertThat(result) + .isInstanceOf(com.aliyun.odps.data.SimpleStruct.class) + .extracting("typeInfo", "values") + .containsExactly(TypeInfoFactory.getStructTypeInfo(expectedFieldNames, expectedTypeInfos), values); + } + + @Test + public void shouldConvertRepeatedDurationPayloadToStructList() { + Duration duration1 = Duration.newBuilder() + .setSeconds(1) + .setNanos(1) + .build(); + Duration duration2 = Duration.newBuilder() + .setSeconds(2) + .setNanos(2) + .build(); + TestMaxComputeTypeInfo.TestRootRepeated message = TestMaxComputeTypeInfo.TestRootRepeated.newBuilder() + .addAllDurationFields(Arrays.asList(duration1, duration2)) + .build(); + List expectedFieldNames = Arrays.asList("seconds", "nanos"); + List expectedTypeInfos = Arrays.asList(TypeInfoFactory.BIGINT, TypeInfoFactory.INT); + List values1 = Arrays.asList(1L, 1); + List values2 = Arrays.asList(2L, 2); + + Object result = durationPayloadConverter.convert(new ProtoPayload(repeatedDescriptor.getFields().get(5), message.getField(repeatedDescriptor.getFields().get(5)), true)); + + assertThat(result) + .isInstanceOf(List.class); + assertThat((List) result) + .hasSize(2) + .allMatch(element -> element instanceof com.aliyun.odps.data.SimpleStruct) + .extracting("typeInfo", "values") + .containsExactly( + Assertions.tuple(TypeInfoFactory.getStructTypeInfo(expectedFieldNames, expectedTypeInfos), values1), + Assertions.tuple(TypeInfoFactory.getStructTypeInfo(expectedFieldNames, expectedTypeInfos), values2) + ); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/MessageProtobufPayloadConverterTest.java b/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/MessageProtobufPayloadConverterTest.java new file mode 100644 index 00000000..7170bfbf --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/MessageProtobufPayloadConverterTest.java @@ -0,0 +1,128 @@ +package com.gotocompany.depot.maxcompute.converter.payload; + +import com.aliyun.odps.data.SimpleStruct; +import com.aliyun.odps.type.StructTypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Duration; +import com.google.protobuf.Timestamp; +import com.gotocompany.depot.TestMaxComputeTypeInfo; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.converter.type.DurationProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.converter.type.MessageProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.converter.type.PrimitiveProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.converter.type.StructProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.converter.type.TimestampProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.converter.type.ProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.model.ProtoPayload; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.when; + +public class MessageProtobufPayloadConverterTest { + + private MessageProtobufPayloadConverter messagePayloadConverter; + private Descriptors.Descriptor descriptor = TestMaxComputeTypeInfo.TestBuyerWrapper.getDescriptor(); + + @Before + public void init() { + MessageProtobufTypeInfoConverter messageTypeInfoConverter = initializeTypeInfoConverters(); + List protobufPayloadConverters = initializePayloadConverter(messageTypeInfoConverter); + messagePayloadConverter = new MessageProtobufPayloadConverter(messageTypeInfoConverter, protobufPayloadConverters); + } + + @Test + public void shouldConvertToStruct() { + Timestamp timestamp = Timestamp.newBuilder() + .setSeconds(1704067200) + .setNanos(0) + .build(); + Duration duration = Duration.newBuilder() + .setSeconds(100) + .build(); + TestMaxComputeTypeInfo.TestBuyer message = TestMaxComputeTypeInfo.TestBuyer.newBuilder() + .setName("buyerName") + .setCart(TestMaxComputeTypeInfo.TestCart.newBuilder() + .setCartId("cart_id") + .addAllItems(Arrays.asList( + TestMaxComputeTypeInfo.TestItem.newBuilder() + .setId("item1") + .setQuantity(1) + .build(), + TestMaxComputeTypeInfo.TestItem.newBuilder() + .setId("item2") + .build())) + .setCreatedAt(timestamp) + .setCartAge(duration) + ) + .setCreatedAt(timestamp) + .build(); + TestMaxComputeTypeInfo.TestBuyerWrapper wrapper = TestMaxComputeTypeInfo.TestBuyerWrapper + .newBuilder() + .setBuyer(message) + .build(); + StructTypeInfo durationTypeInfo = TypeInfoFactory.getStructTypeInfo(Arrays.asList("seconds", "nanos"), Arrays.asList(TypeInfoFactory.BIGINT, TypeInfoFactory.INT)); + StructTypeInfo itemTypeInfo = TypeInfoFactory.getStructTypeInfo(Arrays.asList("id", "quantity"), Arrays.asList(TypeInfoFactory.STRING, TypeInfoFactory.INT)); + StructTypeInfo cartTypeInfo = TypeInfoFactory.getStructTypeInfo( + Arrays.asList("cart_id", "items", "created_at", "cart_age"), + Arrays.asList(TypeInfoFactory.STRING, TypeInfoFactory.getArrayTypeInfo(itemTypeInfo), TypeInfoFactory.TIMESTAMP_NTZ, durationTypeInfo) + ); + StructTypeInfo expectedStructTypeInfo = TypeInfoFactory.getStructTypeInfo( + Arrays.asList("name", "cart", "created_at"), + Arrays.asList(TypeInfoFactory.STRING, cartTypeInfo, TypeInfoFactory.TIMESTAMP_NTZ) + ); + List expectedStructValues = Arrays.asList( + "buyerName", + new SimpleStruct(cartTypeInfo, + Arrays.asList( + "cart_id", + Arrays.asList(new SimpleStruct(itemTypeInfo, Arrays.asList("item1", 1)), new SimpleStruct(itemTypeInfo, Arrays.asList("item2", null))), + LocalDateTime.ofEpochSecond(timestamp.getSeconds(), 0, java.time.ZoneOffset.UTC), + new SimpleStruct(durationTypeInfo, Arrays.asList(duration.getSeconds(), duration.getNanos())))), + LocalDateTime.ofEpochSecond(timestamp.getSeconds(), 0, java.time.ZoneOffset.UTC) + ); + + Object object = messagePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(0), wrapper.getField(descriptor.getFields().get(0)), true)); + + assertThat(object) + .extracting("typeInfo", "values") + .containsExactly(expectedStructTypeInfo, expectedStructValues); + } + + + private MessageProtobufTypeInfoConverter initializeTypeInfoConverters() { + List converters = new ArrayList<>(); + converters.add(new PrimitiveProtobufTypeInfoConverter()); + converters.add(new DurationProtobufTypeInfoConverter()); + converters.add(new StructProtobufTypeInfoConverter()); + converters.add(new TimestampProtobufTypeInfoConverter()); + MessageProtobufTypeInfoConverter messageTypeInfoConverter = new MessageProtobufTypeInfoConverter(converters); + converters.add(messageTypeInfoConverter); + return messageTypeInfoConverter; + } + + private List initializePayloadConverter(MessageProtobufTypeInfoConverter messageTypeInfoConverter) { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:00", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("9999-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + List protobufPayloadConverters = new ArrayList<>(); + protobufPayloadConverters.add(new DurationProtobufPayloadConverter(new DurationProtobufTypeInfoConverter())); + protobufPayloadConverters.add(new PrimitiveProtobufPayloadConverter(new PrimitiveProtobufTypeInfoConverter())); + protobufPayloadConverters.add(new StructProtobufPayloadConverter(new StructProtobufTypeInfoConverter())); + protobufPayloadConverters.add(new TimestampProtobufPayloadConverter(new TimestampProtobufTypeInfoConverter(), maxComputeSinkConfig)); + protobufPayloadConverters.add(new MessageProtobufPayloadConverter(messageTypeInfoConverter, protobufPayloadConverters)); + return protobufPayloadConverters; + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/PrimitiveProtobufPayloadConverterTest.java b/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/PrimitiveProtobufPayloadConverterTest.java new file mode 100644 index 00000000..9c86cba2 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/PrimitiveProtobufPayloadConverterTest.java @@ -0,0 +1,504 @@ +package com.gotocompany.depot.maxcompute.converter.payload; + +import com.aliyun.odps.data.Binary; +import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.gotocompany.depot.TestMaxComputeTypeInfo; +import com.gotocompany.depot.exception.InvalidMessageException; +import com.gotocompany.depot.maxcompute.converter.type.PrimitiveProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.model.ProtoPayload; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class PrimitiveProtobufPayloadConverterTest { + + private final PrimitiveProtobufTypeInfoConverter primitiveTypeInfoConverter = new PrimitiveProtobufTypeInfoConverter(); + private final PrimitiveProtobufPayloadConverter primitivePayloadConverter = new PrimitiveProtobufPayloadConverter(primitiveTypeInfoConverter); + private final Descriptors.Descriptor descriptor = TestMaxComputeTypeInfo.TestFields.getDescriptor(); + private final Descriptors.Descriptor descriptorRepeated = TestMaxComputeTypeInfo.TestFieldsRepeated.getDescriptor(); + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsByteArray() { + byte[] bytes = "bytes".getBytes(StandardCharsets.UTF_8); + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setBytesField(ByteString.copyFrom(bytes)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(0), message.getField(descriptor.getFields().get(0)), true)); + + assertTrue(result instanceof Binary); + assertArrayEquals(bytes, ((Binary) result).data()); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsString() { + String value = "test"; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setStringField(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(1), message.getField(descriptor.getFields().get(1)), true)); + + assertTrue(result instanceof String); + assertEquals(value, result); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsEnum() { + TestMaxComputeTypeInfo.TestEnum value = TestMaxComputeTypeInfo.TestEnum.TEST_1; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setEnumField(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(2), message.getField(descriptor.getFields().get(2)), true)); + + assertTrue(result instanceof String); + assertEquals(value.name(), result); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsDouble() { + double value = 1.23; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setDoubleField(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(3), message.getField(descriptor.getFields().get(3)), true)); + + assertTrue(result instanceof Double); + assertEquals(value, result); + } + + @Test(expected = InvalidMessageException.class) + public void shouldThrowInvalidMessageExceptionWhenDoublePositiveInfinity() { + double value = Double.POSITIVE_INFINITY; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setDoubleField(value) + .build(); + + primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(3), message.getField(descriptor.getFields().get(3)), true)); + } + + @Test(expected = InvalidMessageException.class) + public void shouldThrowInvalidMessageExceptionWhenDoubleNegativeInfinity() { + double value = Double.NEGATIVE_INFINITY; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setDoubleField(value) + .build(); + + primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(3), message.getField(descriptor.getFields().get(3)), true)); + } + + @Test(expected = InvalidMessageException.class) + public void shouldThrowInvalidMessageExceptionWhenDoubleNaN() { + double value = Double.NaN; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setDoubleField(value) + .build(); + + primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(3), message.getField(descriptor.getFields().get(3)), true)); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsFloat() { + float value = 1.23f; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setFloatField(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(4), message.getField(descriptor.getFields().get(4)), true)); + + assertTrue(result instanceof Float); + assertEquals(value, result); + } + + @Test(expected = InvalidMessageException.class) + public void shouldThrowInvalidMessageExceptionWhenFloatPositiveInfinity() { + float value = Float.POSITIVE_INFINITY; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setFloatField(value) + .build(); + + primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(4), message.getField(descriptor.getFields().get(4)), true)); + } + + @Test(expected = InvalidMessageException.class) + public void shouldThrowInvalidMessageExceptionWhenFloatNegativeInfinity() { + float value = Float.NEGATIVE_INFINITY; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setFloatField(value) + .build(); + + primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(4), message.getField(descriptor.getFields().get(4)), true)); + } + + @Test(expected = InvalidMessageException.class) + public void shouldThrowInvalidMessageExceptionWhenFloatNaN() { + float value = Float.NaN; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setFloatField(value) + .build(); + + primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(4), message.getField(descriptor.getFields().get(4)), true)); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsBoolean() { + boolean value = true; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setBoolField(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(5), message.getField(descriptor.getFields().get(5)), true)); + + assertTrue(result instanceof Boolean); + assertEquals(value, result); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsInt64() { + long value = 123L; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setInt64Field(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(6), message.getField(descriptor.getFields().get(6)), true)); + + assertTrue(result instanceof Long); + assertEquals(value, result); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsUInt64() { + long value = 123L; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setUint64Field(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(7), message.getField(descriptor.getFields().get(7)), true)); + + assertTrue(result instanceof Long); + assertEquals(value, result); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsInt32() { + int value = 123; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setInt32Field(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(8), message.getField(descriptor.getFields().get(8)), true)); + + assertTrue(result instanceof Integer); + assertEquals(value, result); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsUInt32() { + int value = 123; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setUint32Field(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(9), message.getField(descriptor.getFields().get(9)), true)); + + assertTrue(result instanceof Integer); + assertEquals(value, result); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsFixed64() { + long value = 123L; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setFixed64Field(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(10), message.getField(descriptor.getFields().get(10)), true)); + + assertTrue(result instanceof Long); + assertEquals(value, result); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsFixed32() { + int value = 123; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setFixed32Field(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(11), message.getField(descriptor.getFields().get(11)), true)); + + assertTrue(result instanceof Integer); + assertEquals(value, result); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsSFixed32() { + int value = 123; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setSfixed32Field(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(12), message.getField(descriptor.getFields().get(12)), true)); + + assertTrue(result instanceof Integer); + assertEquals(value, result); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsSFixed64() { + long value = 123L; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setSfixed64Field(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(13), message.getField(descriptor.getFields().get(13)), true)); + + assertTrue(result instanceof Long); + assertEquals(value, result); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsSInt32() { + int value = 123; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setSint32Field(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(14), message.getField(descriptor.getFields().get(14)), true)); + + assertTrue(result instanceof Integer); + assertEquals(value, result); + } + + @Test + public void shouldReturnObjectAsItIsWhenTypeIsSInt64() { + long value = 123L; + Message message = TestMaxComputeTypeInfo.TestFields.newBuilder() + .setSint64Field(value) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(15), message.getField(descriptor.getFields().get(15)), true)); + + assertTrue(result instanceof Long); + assertEquals(value, result); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsByteArrayList() { + byte[] bytes = "bytes".getBytes(StandardCharsets.UTF_8); + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllBytesFields(Collections.singletonList(ByteString.copyFrom(bytes))) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(0), message.getField(descriptorRepeated.getFields().get(0)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Binary)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsStringList() { + String value = "test"; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllStringFields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(1), message.getField(descriptorRepeated.getFields().get(1)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof String)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsEnumList() { + TestMaxComputeTypeInfo.TestEnum value = TestMaxComputeTypeInfo.TestEnum.TEST_1; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllEnumFields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(2), message.getField(descriptorRepeated.getFields().get(2)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof String)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsDoubleList() { + double value = 1.23; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllDoubleFields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(3), message.getField(descriptorRepeated.getFields().get(3)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Double)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsFloatList() { + float value = 1.23f; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllFloatFields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(4), message.getField(descriptorRepeated.getFields().get(4)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Float)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsBooleanList() { + boolean value = true; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllBoolFields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(5), message.getField(descriptorRepeated.getFields().get(5)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Boolean)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsInt64List() { + long value = 123L; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllInt64Fields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(6), message.getField(descriptorRepeated.getFields().get(6)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Long)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsUInt64List() { + long value = 123L; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllUint64Fields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(7), message.getField(descriptorRepeated.getFields().get(7)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Long)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsInt32List() { + int value = 123; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllInt32Fields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(8), message.getField(descriptorRepeated.getFields().get(8)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Integer)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsUInt32List() { + int value = 123; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllUint32Fields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(9), message.getField(descriptorRepeated.getFields().get(9)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Integer)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsFixed64List() { + long value = 123L; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllFixed64Fields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(10), message.getField(descriptorRepeated.getFields().get(10)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Long)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsFixed32List() { + int value = 123; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllFixed32Fields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(11), message.getField(descriptorRepeated.getFields().get(11)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Integer)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsSFixed32List() { + int value = 123; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllSfixed32Fields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(12), message.getField(descriptorRepeated.getFields().get(12)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Integer)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsSFixed64List() { + long value = 123L; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllSfixed64Fields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(13), message.getField(descriptorRepeated.getFields().get(13)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Long)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsSInt32List() { + int value = 123; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllSint32Fields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(14), message.getField(descriptorRepeated.getFields().get(14)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Integer)); + } + + @Test + public void shouldReturnListObjectAsItIsWhenTypeIsSInt64List() { + long value = 123L; + Message message = TestMaxComputeTypeInfo.TestFieldsRepeated.newBuilder() + .addAllSint64Fields(Collections.singletonList(value)) + .build(); + + Object result = primitivePayloadConverter.convert(new ProtoPayload(descriptorRepeated.getFields().get(15), message.getField(descriptorRepeated.getFields().get(15)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(element -> element instanceof Long)); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/StructProtobufPayloadConverterTest.java b/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/StructProtobufPayloadConverterTest.java new file mode 100644 index 00000000..17d5ea94 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/StructProtobufPayloadConverterTest.java @@ -0,0 +1,61 @@ +package com.gotocompany.depot.maxcompute.converter.payload; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.Struct; +import com.google.protobuf.Value; +import com.gotocompany.depot.TestMaxComputeTypeInfo; +import com.gotocompany.depot.maxcompute.converter.type.StructProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.model.ProtoPayload; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class StructProtobufPayloadConverterTest { + + private final StructProtobufTypeInfoConverter structTypeInfoConverter = new StructProtobufTypeInfoConverter(); + private final StructProtobufPayloadConverter structPayloadConverter = new StructProtobufPayloadConverter(structTypeInfoConverter); + private final Descriptors.Descriptor descriptor = TestMaxComputeTypeInfo.TestRoot.getDescriptor(); + private final Descriptors.Descriptor repeatedDescriptor = TestMaxComputeTypeInfo.TestRootRepeated.getDescriptor(); + + @Test + public void shouldConvertStructPayloadToJsonString() { + Struct.Builder structBuilder = Struct.newBuilder(); + structBuilder.putFields("intField", Value.newBuilder().setNumberValue(1.0).build()); + structBuilder.putFields("stringField", Value.newBuilder().setStringValue("String").build()); + Message message = TestMaxComputeTypeInfo.TestRoot.newBuilder() + .setStructField(structBuilder.build()) + .build(); + String expected = "{\"intField\":1.0,\"stringField\":\"String\"}"; + + Object result = structPayloadConverter.convert(new ProtoPayload(descriptor.getFields().get(4), message.getField(descriptor.getFields().get(4)), true)); + + assertTrue(result instanceof String); + assertEquals(expected, result); + } + + @Test + public void shouldConvertRepeatedStructPayloadToJsonString() { + Struct.Builder structBuilder = Struct.newBuilder(); + structBuilder.putFields("intField", Value.newBuilder().setNumberValue(1.0).build()); + structBuilder.putFields("stringField", Value.newBuilder().setStringValue("String").build()); + List structs = new ArrayList<>(); + structs.add(structBuilder.build()); + structs.add(structBuilder.build()); + Message message = TestMaxComputeTypeInfo.TestRootRepeated.newBuilder() + .addAllStructFields(structs) + .build(); + String expected = "[{\"intField\":1.0,\"stringField\":\"String\"}, {\"intField\":1.0,\"stringField\":\"String\"}]"; + + Object result = structPayloadConverter.convert(new ProtoPayload(repeatedDescriptor.getFields().get(4), message.getField(repeatedDescriptor.getFields().get(4)), true)); + + assertTrue(result instanceof List); + assertTrue(((List) result).stream().allMatch(e -> e instanceof String)); + assertEquals(expected, result.toString()); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/TimestampProtobufPayloadConverterTest.java b/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/TimestampProtobufPayloadConverterTest.java new file mode 100644 index 00000000..6356dfd4 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/converter/payload/TimestampProtobufPayloadConverterTest.java @@ -0,0 +1,230 @@ +package com.gotocompany.depot.maxcompute.converter.payload; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.Timestamp; +import com.gotocompany.depot.TestMaxComputeTypeInfo; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.exception.InvalidMessageException; +import com.gotocompany.depot.maxcompute.converter.type.TimestampProtobufTypeInfoConverter; +import com.gotocompany.depot.maxcompute.model.ProtoPayload; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.time.Duration; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.when; + +public class TimestampProtobufPayloadConverterTest { + + private final TimestampProtobufTypeInfoConverter timestampTypeInfoConverter = new TimestampProtobufTypeInfoConverter(); + private final Descriptors.Descriptor descriptor = TestMaxComputeTypeInfo.TestRoot.getDescriptor(); + private final Descriptors.Descriptor repeatedDescriptor = TestMaxComputeTypeInfo.TestRootRepeated.getDescriptor(); + private TimestampProtobufPayloadConverter timestampPayloadConverter = new TimestampProtobufPayloadConverter(timestampTypeInfoConverter, Mockito.mock(MaxComputeSinkConfig.class)); + + @Before + public void setUp() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:01", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("9999-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(true); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn("timestamp_field"); + when(maxComputeSinkConfig.getMaxPastYearEventTimeDifference()).thenReturn(999); + when(maxComputeSinkConfig.getMaxFutureYearEventTimeDifference()).thenReturn(999); + timestampPayloadConverter = new TimestampProtobufPayloadConverter(timestampTypeInfoConverter, maxComputeSinkConfig); + } + + @Test + public void shouldConvertToTimestampNtz() { + Timestamp timestamp = Timestamp.newBuilder() + .setSeconds(2500) + .setNanos(100) + .build(); + TestMaxComputeTypeInfo.TestRoot message = TestMaxComputeTypeInfo.TestRoot.newBuilder() + .setTimestampField(timestamp) + .build(); + LocalDateTime expectedLocalDateTime = LocalDateTime.ofEpochSecond( + timestamp.getSeconds(), timestamp.getNanos(), java.time.ZoneOffset.UTC); + + Object result = timestampPayloadConverter.convertSingular(new ProtoPayload(descriptor.getFields().get(3), message.getField(descriptor.getFields().get(3)), true)); + + assertThat(result) + .isEqualTo(expectedLocalDateTime); + } + + @Test + public void shouldConvertRepeatedTimestampPayloadToTimestampList() { + Timestamp timestamp1 = Timestamp.newBuilder() + .setSeconds(2500) + .setNanos(100) + .build(); + Timestamp timestamp2 = Timestamp.newBuilder() + .setSeconds(3600) + .setNanos(200) + .build(); + TestMaxComputeTypeInfo.TestRootRepeated message = TestMaxComputeTypeInfo.TestRootRepeated.newBuilder() + .addAllTimestampFields(Arrays.asList(timestamp1, timestamp2)) + .build(); + LocalDateTime expectedLocalDateTime1 = LocalDateTime.ofEpochSecond( + timestamp1.getSeconds(), timestamp1.getNanos(), java.time.ZoneOffset.UTC); + LocalDateTime expectedLocalDateTime2 = LocalDateTime.ofEpochSecond( + timestamp2.getSeconds(), timestamp2.getNanos(), java.time.ZoneOffset.UTC); + + Object result = timestampPayloadConverter.convert(new ProtoPayload(repeatedDescriptor.getFields().get(3), message.getField(repeatedDescriptor.getFields().get(3)), true)); + + assertThat(result) + .isInstanceOf(List.class); + assertThat(((List) result).stream().map(LocalDateTime.class::cast)) + .hasSize(2) + .containsExactly(expectedLocalDateTime1, expectedLocalDateTime2); + } + + @Test(expected = InvalidMessageException.class) + public void shouldThrowInvalidMessageExceptionWhenDateIsOutOfMinValidRange() { + Timestamp timestamp = Timestamp.newBuilder() + .setSeconds(0) + .setNanos(0) + .build(); + TestMaxComputeTypeInfo.TestRoot message = TestMaxComputeTypeInfo.TestRoot.newBuilder() + .setTimestampField(timestamp) + .build(); + LocalDateTime expectedLocalDateTime = LocalDateTime.ofEpochSecond( + timestamp.getSeconds(), timestamp.getNanos(), java.time.ZoneOffset.UTC); + + Object result = timestampPayloadConverter.convertSingular(new ProtoPayload(descriptor.getFields().get(3), message.getField(descriptor.getFields().get(3)), true)); + + assertThat(result) + .isEqualTo(expectedLocalDateTime); + } + + @Test(expected = InvalidMessageException.class) + public void shouldThrowInvalidMessageExceptionWhenDateIsOutOfMaxValidRange() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:00", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + timestampPayloadConverter = new TimestampProtobufPayloadConverter(timestampTypeInfoConverter, maxComputeSinkConfig); + + Timestamp timestamp = Timestamp.newBuilder() + .setSeconds(3600 * 48) + .setNanos(0) + .build(); + TestMaxComputeTypeInfo.TestRoot message = TestMaxComputeTypeInfo.TestRoot.newBuilder() + .setTimestampField(timestamp) + .build(); + LocalDateTime expectedLocalDateTime = LocalDateTime.ofEpochSecond( + timestamp.getSeconds(), timestamp.getNanos(), java.time.ZoneOffset.UTC); + + Object result = timestampPayloadConverter.convertSingular(new ProtoPayload(descriptor.getFields().get(3), message.getField(descriptor.getFields().get(3)), true)); + + assertThat(result).isEqualTo(expectedLocalDateTime); + } + + @Test(expected = InvalidMessageException.class) + public void shouldThrowInvalidMessageExceptionWhenTimeDifferenceExceedsMaxPastDuration() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:00", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("9999-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(true); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn("timestamp_field"); + when(maxComputeSinkConfig.getMaxPastYearEventTimeDifference()).thenReturn(5); + when(maxComputeSinkConfig.getMaxPastYearEventTimeDifference()).thenReturn(5); + timestampPayloadConverter = new TimestampProtobufPayloadConverter(timestampTypeInfoConverter, maxComputeSinkConfig); + Timestamp timestamp = Timestamp.newBuilder() + .setSeconds(3600) + .setNanos(0) + .build(); + TestMaxComputeTypeInfo.TestRoot message = TestMaxComputeTypeInfo.TestRoot.newBuilder() + .setTimestampField(timestamp) + .build(); + + timestampPayloadConverter.convertSingular(new ProtoPayload(descriptor.getFields().get(3), message.getField(descriptor.getFields().get(3)), true)); + } + + @Test(expected = InvalidMessageException.class) + public void shouldThrowInvalidMessageExceptionWhenTimeDifferenceExceedsMaxFutureDuration() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:00", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("9999-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(true); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn("timestamp_field"); + when(maxComputeSinkConfig.getMaxPastYearEventTimeDifference()).thenReturn(5); + when(maxComputeSinkConfig.getMaxFutureYearEventTimeDifference()).thenReturn(1); + timestampPayloadConverter = new TimestampProtobufPayloadConverter(timestampTypeInfoConverter, maxComputeSinkConfig); + Timestamp timestamp = Timestamp.newBuilder() + .setSeconds(System.currentTimeMillis() / 1000 + Duration.ofDays(365 * 6).toMinutes() * 60) + .setNanos(0) + .build(); + TestMaxComputeTypeInfo.TestRoot message = TestMaxComputeTypeInfo.TestRoot.newBuilder() + .setTimestampField(timestamp) + .build(); + + timestampPayloadConverter.convertSingular(new ProtoPayload(descriptor.getFields().get(3), message.getField(descriptor.getFields().get(3)), true)); + } + + @Test + public void shouldSkipDifferenceValidationWhenPartitionDisabled() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:00", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("9999-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(false); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn("timestamp_field"); + when(maxComputeSinkConfig.getMaxPastYearEventTimeDifference()).thenReturn(5); + when(maxComputeSinkConfig.getMaxFutureYearEventTimeDifference()).thenReturn(1); + timestampPayloadConverter = new TimestampProtobufPayloadConverter(timestampTypeInfoConverter, maxComputeSinkConfig); + Timestamp timestamp = Timestamp.newBuilder() + .setSeconds(System.currentTimeMillis() / 1000 + Duration.ofDays(365 * 6).toMinutes() * 60) + .setNanos(0) + .build(); + TestMaxComputeTypeInfo.TestRoot message = TestMaxComputeTypeInfo.TestRoot.newBuilder() + .setTimestampField(timestamp) + .build(); + LocalDateTime expectedLocalDateTime = LocalDateTime.ofEpochSecond( + timestamp.getSeconds(), timestamp.getNanos(), java.time.ZoneOffset.UTC); + + LocalDateTime result = (LocalDateTime) timestampPayloadConverter.convertSingular( + new ProtoPayload(descriptor.getFields().get(3), message.getField(descriptor.getFields().get(3)), true)); + + assertThat(result) + .isEqualTo(expectedLocalDateTime); + } + + @Test + public void shouldSkipDifferenceValidationWhenIsNotRootLevel() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:00", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("9999-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(true); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn("timestamp_field"); + when(maxComputeSinkConfig.getMaxPastYearEventTimeDifference()).thenReturn(5); + when(maxComputeSinkConfig.getMaxFutureYearEventTimeDifference()).thenReturn(1); + timestampPayloadConverter = new TimestampProtobufPayloadConverter(timestampTypeInfoConverter, maxComputeSinkConfig); + Timestamp timestamp = Timestamp.newBuilder() + .setSeconds(System.currentTimeMillis() / 1000 + Duration.ofDays(365 * 6).toMinutes() * 60) + .setNanos(0) + .build(); + TestMaxComputeTypeInfo.TestRoot message = TestMaxComputeTypeInfo.TestRoot.newBuilder() + .setTimestampField(timestamp) + .build(); + LocalDateTime expectedLocalDateTime = LocalDateTime.ofEpochSecond( + timestamp.getSeconds(), timestamp.getNanos(), java.time.ZoneOffset.UTC); + + LocalDateTime result = (LocalDateTime) timestampPayloadConverter.convertSingular(new ProtoPayload(descriptor.getFields().get(3), + message.getField(descriptor.getFields().get(3)), false)); + + assertThat(result) + .isEqualTo(expectedLocalDateTime); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/converter/record/ProtoMessageRecordConverterTest.java b/src/test/java/com/gotocompany/depot/maxcompute/converter/record/ProtoMessageRecordConverterTest.java new file mode 100644 index 00000000..1ee91a4a --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/converter/record/ProtoMessageRecordConverterTest.java @@ -0,0 +1,254 @@ +package com.gotocompany.depot.maxcompute.converter.record; + +import com.aliyun.odps.data.SimpleStruct; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Timestamp; +import com.gotocompany.depot.TestMaxComputeRecord; +import com.gotocompany.depot.common.Tuple; +import com.gotocompany.depot.common.TupleString; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.config.SinkConfig; +import com.gotocompany.depot.error.ErrorInfo; +import com.gotocompany.depot.error.ErrorType; +import com.gotocompany.depot.exception.InvalidMessageException; +import com.gotocompany.depot.exception.UnknownFieldsException; +import com.gotocompany.depot.maxcompute.converter.ProtobufConverterOrchestrator; +import com.gotocompany.depot.maxcompute.MaxComputeSchemaHelper; +import com.gotocompany.depot.maxcompute.model.MaxComputeSchema; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.maxcompute.model.RecordWrappers; +import com.gotocompany.depot.maxcompute.record.ProtoDataColumnRecordDecorator; +import com.gotocompany.depot.maxcompute.record.ProtoMetadataColumnRecordDecorator; +import com.gotocompany.depot.maxcompute.record.RecordDecorator; +import com.gotocompany.depot.maxcompute.schema.MaxComputeSchemaCache; +import com.gotocompany.depot.maxcompute.schema.partition.PartitioningStrategy; +import com.gotocompany.depot.maxcompute.schema.partition.PartitioningStrategyFactory; +import com.gotocompany.depot.message.Message; +import com.gotocompany.depot.message.ParsedMessage; +import com.gotocompany.depot.message.SinkConnectorSchemaMessageMode; +import com.gotocompany.depot.message.proto.ProtoMessageParser; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.io.Serializable; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.when; + +public class ProtoMessageRecordConverterTest { + + private final Descriptors.Descriptor descriptor = TestMaxComputeRecord.MaxComputeRecord.getDescriptor(); + private MaxComputeSinkConfig maxComputeSinkConfig; + private ProtobufConverterOrchestrator protobufConverterOrchestrator; + private ProtoMessageParser protoMessageParser; + private MaxComputeSchemaHelper maxComputeSchemaHelper; + private SinkConfig sinkConfig; + private MaxComputeSchemaCache maxComputeSchemaCache; + private ProtoMessageRecordConverter protoMessageRecordConverter; + + @Before + public void setup() throws IOException { + maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.shouldAddMetadata()).thenReturn(Boolean.TRUE); + when(maxComputeSinkConfig.getMetadataColumnsTypes()).thenReturn( + Arrays.asList(new TupleString("__message_timestamp", "timestamp"), + new TupleString("__kafka_topic", "string"), + new TupleString("__kafka_offset", "long") + ) + ); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(Boolean.TRUE); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn("timestamp"); + when(maxComputeSinkConfig.getTablePartitionColumnName()).thenReturn("__partition_column"); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getTablePartitionByTimestampTimeUnit()).thenReturn("DAY"); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:00", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("9999-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getMaxPastYearEventTimeDifference()).thenReturn(999); + when(maxComputeSinkConfig.getMaxFutureYearEventTimeDifference()).thenReturn(999); + protobufConverterOrchestrator = new ProtobufConverterOrchestrator(maxComputeSinkConfig); + protoMessageParser = Mockito.mock(ProtoMessageParser.class); + ParsedMessage parsedMessage = Mockito.mock(ParsedMessage.class); + when(parsedMessage.getRaw()).thenReturn(getMockedMessage()); + when(protoMessageParser.parse(Mockito.any(), Mockito.any(), Mockito.any())) + .thenReturn(parsedMessage); + sinkConfig = Mockito.mock(SinkConfig.class); + when(sinkConfig.getSinkConnectorSchemaMessageMode()) + .thenReturn(SinkConnectorSchemaMessageMode.LOG_MESSAGE); + PartitioningStrategy partitioningStrategy = PartitioningStrategyFactory.createPartitioningStrategy( + protobufConverterOrchestrator, + maxComputeSinkConfig, + descriptor + ); + maxComputeSchemaHelper = new MaxComputeSchemaHelper(protobufConverterOrchestrator, maxComputeSinkConfig, partitioningStrategy); + maxComputeSchemaCache = Mockito.mock(MaxComputeSchemaCache.class); + MaxComputeSchema maxComputeSchema = maxComputeSchemaHelper.build(descriptor); + when(maxComputeSchemaCache.getMaxComputeSchema()).thenReturn(maxComputeSchema); + RecordDecorator protoDataColumnRecordDecorator = new ProtoDataColumnRecordDecorator(null, + protobufConverterOrchestrator, + protoMessageParser, sinkConfig, partitioningStrategy); + RecordDecorator metadataColumnRecordDecorator = new ProtoMetadataColumnRecordDecorator( + protoDataColumnRecordDecorator, maxComputeSinkConfig, maxComputeSchemaCache); + protoMessageRecordConverter = new ProtoMessageRecordConverter(metadataColumnRecordDecorator, maxComputeSchemaCache); + } + + @Test + public void shouldConvertMessageToRecordWrapper() { + Message message = new Message( + null, + getMockedMessage().toByteArray(), + new Tuple<>("__message_timestamp", 123012311L), + new Tuple<>("__kafka_topic", "topic"), + new Tuple<>("__kafka_offset", 100L) + ); + LocalDateTime expectedTimestampLocalDateTime = Instant.ofEpochMilli( + 123012311L).atZone(ZoneId.of("UTC")) + .toLocalDateTime(); + LocalDateTime expectedPayloadLocalDateTime = LocalDateTime.ofEpochSecond( + 10002010L, + 1000, + ZoneOffset.UTC + ); + + RecordWrappers recordWrappers = protoMessageRecordConverter.convert(Collections.singletonList(message)); + + assertThat(recordWrappers.getValidRecords()).size().isEqualTo(1); + RecordWrapper recordWrapper = recordWrappers.getValidRecords().get(0); + assertThat(recordWrapper.getIndex()).isEqualTo(0); + assertThat(recordWrapper.getRecord()) + .extracting("values") + .isEqualTo(new Serializable[]{ + expectedTimestampLocalDateTime, + "topic", + 100L, + "id", + new ArrayList<>(Arrays.asList( + new SimpleStruct( + TypeInfoFactory.getStructTypeInfo( + Arrays.asList("name", "balance"), + Arrays.asList(TypeInfoFactory.STRING, TypeInfoFactory.FLOAT) + ), + Arrays.asList("name_1", 100.2f) + ), + new SimpleStruct( + TypeInfoFactory.getStructTypeInfo( + Arrays.asList("name", "balance"), + Arrays.asList(TypeInfoFactory.STRING, TypeInfoFactory.FLOAT) + ), + Arrays.asList("name_2", 50f) + ) + )), + expectedPayloadLocalDateTime + }); + assertThat(recordWrapper.getErrorInfo()).isNull(); + } + + @Test + public void shouldReturnRecordWrapperWithDeserializationErrorWhenIOExceptionIsThrown() throws IOException { + RecordDecorator recordDecorator = Mockito.mock(RecordDecorator.class); + Mockito.doThrow(new IOException()).when(recordDecorator) + .decorate(Mockito.any(), Mockito.any()); + ProtoMessageRecordConverter recordConverter = new ProtoMessageRecordConverter(recordDecorator, maxComputeSchemaCache); + Message message = new Message( + null, + getMockedMessage().toByteArray(), + new Tuple<>("__message_timestamp", 123012311L), + new Tuple<>("__kafka_topic", "topic"), + new Tuple<>("__kafka_offset", 100L) + ); + + RecordWrappers recordWrappers = recordConverter.convert(Collections.singletonList(message)); + + assertThat(recordWrappers.getInvalidRecords()).size().isEqualTo(1); + RecordWrapper recordWrapper = recordWrappers.getInvalidRecords().get(0); + assertThat(recordWrapper.getIndex()).isEqualTo(0); + assertThat(recordWrapper.getRecord()) + .isNull(); + assertThat(recordWrapper.getErrorInfo()) + .isEqualTo(new ErrorInfo(new IOException(), ErrorType.DESERIALIZATION_ERROR)); + } + + @Test + public void shouldReturnRecordWrapperWithUnknownFieldsErrorWhenUnknownFieldExceptionIsThrown() throws IOException { + RecordDecorator recordDecorator = Mockito.mock(RecordDecorator.class); + com.google.protobuf.Message mockedMessage = getMockedMessage(); + Mockito.doThrow(new UnknownFieldsException(mockedMessage)).when(recordDecorator) + .decorate(Mockito.any(), Mockito.any()); + ProtoMessageRecordConverter recordConverter = new ProtoMessageRecordConverter(recordDecorator, maxComputeSchemaCache); + Message message = new Message( + null, + getMockedMessage().toByteArray(), + new Tuple<>("__message_timestamp", 123012311L), + new Tuple<>("__kafka_topic", "topic"), + new Tuple<>("__kafka_offset", 100L) + ); + + RecordWrappers recordWrappers = recordConverter.convert(Collections.singletonList(message)); + + assertThat(recordWrappers.getInvalidRecords()).size().isEqualTo(1); + RecordWrapper recordWrapper = recordWrappers.getInvalidRecords().get(0); + assertThat(recordWrapper.getIndex()).isEqualTo(0); + assertThat(recordWrapper.getRecord()) + .isNull(); + assertThat(recordWrapper.getErrorInfo()) + .isEqualTo(new ErrorInfo(new UnknownFieldsException(mockedMessage), ErrorType.UNKNOWN_FIELDS_ERROR)); + } + + @Test + public void shouldReturnRecordWrapperWithInvalidMessageErrorWhenInvalidMessageExceptionIsThrown() throws IOException { + RecordDecorator recordDecorator = Mockito.mock(RecordDecorator.class); + String invalidMessage = "Invalid message"; + Mockito.doThrow(new InvalidMessageException(invalidMessage)).when(recordDecorator) + .decorate(Mockito.any(), Mockito.any()); + ProtoMessageRecordConverter recordConverter = new ProtoMessageRecordConverter(recordDecorator, maxComputeSchemaCache); + Message message = new Message( + null, + getMockedMessage().toByteArray(), + new Tuple<>("__message_timestamp", 123012311L), + new Tuple<>("__kafka_topic", "topic"), + new Tuple<>("__kafka_offset", 100L) + ); + + RecordWrappers recordWrappers = recordConverter.convert(Collections.singletonList(message)); + + assertThat(recordWrappers.getInvalidRecords()).size().isEqualTo(1); + RecordWrapper recordWrapper = recordWrappers.getInvalidRecords().get(0); + assertThat(recordWrapper.getIndex()).isEqualTo(0); + assertThat(recordWrapper.getRecord()) + .isNull(); + assertThat(recordWrapper.getErrorInfo()) + .isEqualTo(new ErrorInfo(new InvalidMessageException(invalidMessage), ErrorType.INVALID_MESSAGE_ERROR)); + } + + private static TestMaxComputeRecord.MaxComputeRecord getMockedMessage() { + return TestMaxComputeRecord.MaxComputeRecord + .newBuilder() + .setId("id") + .addAllInnerRecord(Arrays.asList( + TestMaxComputeRecord.InnerRecord.newBuilder() + .setName("name_1") + .setBalance(100.2f) + .build(), + TestMaxComputeRecord.InnerRecord.newBuilder() + .setName("name_2") + .setBalance(50f) + .build() + )) + .setTimestamp(Timestamp.newBuilder() + .setSeconds(10002010) + .setNanos(1000) + .build()) + .build(); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/converter/type/DurationProtobufTypeInfoConverterTest.java b/src/test/java/com/gotocompany/depot/maxcompute/converter/type/DurationProtobufTypeInfoConverterTest.java new file mode 100644 index 00000000..a258e375 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/converter/type/DurationProtobufTypeInfoConverterTest.java @@ -0,0 +1,45 @@ +package com.gotocompany.depot.maxcompute.converter.type; + +import com.aliyun.odps.type.TypeInfo; +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.TestMaxComputeTypeInfo; +import org.junit.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class DurationProtobufTypeInfoConverterTest { + + private static final int DURATION_INDEX = 5; + private final Descriptors.Descriptor descriptor = TestMaxComputeTypeInfo.TestRoot.getDescriptor(); + private final DurationProtobufTypeInfoConverter durationTypeInfoConverter = new DurationProtobufTypeInfoConverter(); + + @Test + public void shouldConvertToStruct() { + Descriptors.FieldDescriptor fieldDescriptor = descriptor.getFields().get(DURATION_INDEX); + + TypeInfo typeInfo = durationTypeInfoConverter.convert(fieldDescriptor); + + assertEquals("STRUCT<`seconds`:BIGINT,`nanos`:INT>", typeInfo.getTypeName()); + } + + @Test + public void shouldReturnTrueForDuration() { + Descriptors.FieldDescriptor fieldDescriptor = descriptor.getFields().get(DURATION_INDEX); + + boolean canConvert = durationTypeInfoConverter.canConvert(fieldDescriptor); + + assertTrue(canConvert); + } + + @Test + public void shouldReturnFalseForNonDuration() { + Descriptors.FieldDescriptor fieldDescriptor = descriptor.getFields().get(0); + + boolean canConvert = durationTypeInfoConverter.canConvert(fieldDescriptor); + + assertFalse(canConvert); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/converter/type/MessageProtobufTypeInfoConverterTest.java b/src/test/java/com/gotocompany/depot/maxcompute/converter/type/MessageProtobufTypeInfoConverterTest.java new file mode 100644 index 00000000..5428709e --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/converter/type/MessageProtobufTypeInfoConverterTest.java @@ -0,0 +1,66 @@ +package com.gotocompany.depot.maxcompute.converter.type; + +import com.aliyun.odps.type.TypeInfo; +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.TestMaxComputeTypeInfo; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class MessageProtobufTypeInfoConverterTest { + + private final Descriptors.Descriptor descriptor = TestMaxComputeTypeInfo.TestRoot.getDescriptor(); + private MessageProtobufTypeInfoConverter messageTypeInfoConverter; + + @Before + public void initialize() { + initializeConverters(); + } + + @Test + public void shouldConvertMessageToProperTypeInfo() { + TypeInfo firstMessageFieldTypeInfo = messageTypeInfoConverter.convert(descriptor.getFields().get(1)); + TypeInfo secondMessageFieldTypeInfo = messageTypeInfoConverter.convert(descriptor.getFields().get(2)); + + String expectedFirstMessageTypeRepresentation = "STRUCT<`string_field`:STRING,`another_inner_field`:STRUCT<`string_field`:STRING>,`another_inner_list_field`:ARRAY>>"; + String expectedSecondMessageTypeRepresentation = String.format("ARRAY<%s>", expectedFirstMessageTypeRepresentation); + + assertEquals(expectedFirstMessageTypeRepresentation, firstMessageFieldTypeInfo.toString()); + assertEquals(expectedSecondMessageTypeRepresentation, secondMessageFieldTypeInfo.toString()); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIllegalArgumentExceptionWhenUnsupportedTypeIsGiven() { + messageTypeInfoConverter = new MessageProtobufTypeInfoConverter(new ArrayList<>()); + Descriptors.FieldDescriptor unsupportedFieldDescriptor = descriptor.getFields().get(1); + + messageTypeInfoConverter.convert(unsupportedFieldDescriptor); + } + + @Test + public void shouldReturnTrueWhenCanConvertIsCalledWithMessageFieldDescriptor() { + assertTrue(messageTypeInfoConverter.canConvert(descriptor.getFields().get(1))); + } + + @Test + public void shouldReturnFalseWhenCanConvertIsCalledWithNonMessageFieldDescriptor() { + assertFalse(messageTypeInfoConverter.canConvert(descriptor.getFields().get(0))); + } + + private void initializeConverters() { + List converters = new ArrayList<>(); + converters.add(new PrimitiveProtobufTypeInfoConverter()); + converters.add(new DurationProtobufTypeInfoConverter()); + converters.add(new StructProtobufTypeInfoConverter()); + converters.add(new TimestampProtobufTypeInfoConverter()); + messageTypeInfoConverter = new MessageProtobufTypeInfoConverter(converters); + converters.add(messageTypeInfoConverter); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/converter/type/PrimitiveProtobufTypeInfoConverterTest.java b/src/test/java/com/gotocompany/depot/maxcompute/converter/type/PrimitiveProtobufTypeInfoConverterTest.java new file mode 100644 index 00000000..4451e176 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/converter/type/PrimitiveProtobufTypeInfoConverterTest.java @@ -0,0 +1,127 @@ +package com.gotocompany.depot.maxcompute.converter.type; + +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.TestMaxComputeTypeInfo; +import org.junit.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class PrimitiveProtobufTypeInfoConverterTest { + + private final PrimitiveProtobufTypeInfoConverter primitiveTypeInfoConverter = new PrimitiveProtobufTypeInfoConverter(); + private final Descriptors.Descriptor descriptor = TestMaxComputeTypeInfo.TestFields.getDescriptor(); + + @Test + public void shouldConvertToBinary() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("bytes_field")); + + assertEquals(TypeInfoFactory.BINARY, typeInfo); + } + + @Test + public void shouldConvertToString() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("string_field")); + + assertEquals(TypeInfoFactory.STRING, typeInfo); + } + + @Test + public void shouldConvertEnumToString() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("enum_field")); + + assertEquals(TypeInfoFactory.STRING, typeInfo); + } + + @Test + public void shouldConvertToDouble() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("double_field")); + + assertEquals(TypeInfoFactory.DOUBLE, typeInfo); + } + + @Test + public void shouldConvertToFloat() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("float_field")); + + assertEquals(TypeInfoFactory.FLOAT, typeInfo); + } + + @Test + public void shouldConvertToBoolean() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("bool_field")); + + assertEquals(TypeInfoFactory.BOOLEAN, typeInfo); + } + + @Test + public void shouldConvertToBigInt() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("int64_field")); + + assertEquals(TypeInfoFactory.BIGINT, typeInfo); + } + + @Test + public void shouldConvertUInt64ToBigInt() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("uint64_field")); + + assertEquals(TypeInfoFactory.BIGINT, typeInfo); + } + + @Test + public void shouldConvertToInt() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("int32_field")); + + assertEquals(TypeInfoFactory.INT, typeInfo); + } + + @Test + public void shouldConvertUInt32ToInt() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("uint32_field")); + + assertEquals(TypeInfoFactory.INT, typeInfo); + } + + @Test + public void shouldConvertFixed64ToBigInt() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("fixed64_field")); + + assertEquals(TypeInfoFactory.BIGINT, typeInfo); + } + + @Test + public void shouldConvertFixed32ToInt() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("fixed32_field")); + + assertEquals(TypeInfoFactory.INT, typeInfo); + } + + @Test + public void shouldConvertSFixed32ToInt() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("sfixed32_field")); + + assertEquals(TypeInfoFactory.INT, typeInfo); + } + + @Test + public void shouldConvertSFixed64ToBigInt() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("sfixed64_field")); + + assertEquals(TypeInfoFactory.BIGINT, typeInfo); + } + + @Test + public void shouldConvertSInt32ToInt() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("sint32_field")); + + assertEquals(TypeInfoFactory.INT, typeInfo); + } + + @Test + public void shouldConvertSInt64ToBigInt() { + TypeInfo typeInfo = primitiveTypeInfoConverter.convert(descriptor.findFieldByName("sint64_field")); + + assertEquals(TypeInfoFactory.BIGINT, typeInfo); + } +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/converter/type/StructProtobufTypeInfoConverterTest.java b/src/test/java/com/gotocompany/depot/maxcompute/converter/type/StructProtobufTypeInfoConverterTest.java new file mode 100644 index 00000000..ea9884db --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/converter/type/StructProtobufTypeInfoConverterTest.java @@ -0,0 +1,36 @@ +package com.gotocompany.depot.maxcompute.converter.type; + +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.TestMaxComputeTypeInfo; +import org.junit.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class StructProtobufTypeInfoConverterTest { + + private static final int STRUCT_INDEX = 4; + private static final Descriptors.Descriptor DESCRIPTOR = TestMaxComputeTypeInfo.TestRoot.getDescriptor(); + private final StructProtobufTypeInfoConverter structTypeInfoConverter = new StructProtobufTypeInfoConverter(); + + @Test + public void shouldConvertToStringTypeInfo() { + TypeInfo typeInfo = structTypeInfoConverter.convert(DESCRIPTOR.getFields().get(STRUCT_INDEX)); + + assertEquals(TypeInfoFactory.STRING, typeInfo); + } + + @Test + public void shouldReturnTrueWhenCanConvertIsCalledWithStructFieldDescriptor() { + assertTrue(structTypeInfoConverter.canConvert(DESCRIPTOR.getFields().get(STRUCT_INDEX))); + } + + @Test + public void shouldReturnFalseWhenCanConvertIsCalledWithNonStructFieldDescriptor() { + assertFalse(structTypeInfoConverter.canConvert(DESCRIPTOR.getFields().get(0))); + assertFalse(structTypeInfoConverter.canConvert(DESCRIPTOR.getFields().get(1))); + } +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/converter/type/TimestampProtobufPayloadConverterTest.java b/src/test/java/com/gotocompany/depot/maxcompute/converter/type/TimestampProtobufPayloadConverterTest.java new file mode 100644 index 00000000..303eebde --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/converter/type/TimestampProtobufPayloadConverterTest.java @@ -0,0 +1,36 @@ +package com.gotocompany.depot.maxcompute.converter.type; + +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.TestMaxComputeTypeInfo; +import org.junit.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TimestampProtobufPayloadConverterTest { + + private static final int TIMESTAMP_INDEX = 3; + private final Descriptors.Descriptor descriptor = TestMaxComputeTypeInfo.TestRoot.getDescriptor(); + private final TimestampProtobufTypeInfoConverter timestampTypeInfoConverter = new TimestampProtobufTypeInfoConverter(); + + @Test + public void shouldConvertToTimestampNtz() { + TypeInfo typeInfo = timestampTypeInfoConverter.convert(descriptor.getFields().get(TIMESTAMP_INDEX)); + + assertEquals(TypeInfoFactory.TIMESTAMP_NTZ, typeInfo); + } + + @Test + public void shouldReturnTrueWhenCanConvertIsCalledWithTimestampFieldDescriptor() { + assertTrue(timestampTypeInfoConverter.canConvert(descriptor.getFields().get(TIMESTAMP_INDEX))); + } + + @Test + public void shouldReturnFalseWhenCanConvertIsCalledWithNonTimestampFieldDescriptor() { + assertFalse(timestampTypeInfoConverter.canConvert(descriptor.getFields().get(0))); + assertFalse(timestampTypeInfoConverter.canConvert(descriptor.getFields().get(1))); + } +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/record/ProtoDataColumnRecordDecoratorTest.java b/src/test/java/com/gotocompany/depot/maxcompute/record/ProtoDataColumnRecordDecoratorTest.java new file mode 100644 index 00000000..34833077 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/record/ProtoDataColumnRecordDecoratorTest.java @@ -0,0 +1,348 @@ +package com.gotocompany.depot.maxcompute.record; + +import com.aliyun.odps.TableSchema; +import com.aliyun.odps.data.ArrayRecord; +import com.aliyun.odps.data.Record; +import com.aliyun.odps.data.SimpleStruct; +import com.aliyun.odps.type.ArrayTypeInfo; +import com.aliyun.odps.type.StructTypeInfo; +import com.aliyun.odps.type.TypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Timestamp; +import com.gotocompany.depot.TestMaxComputeRecord; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.config.SinkConfig; +import com.gotocompany.depot.maxcompute.converter.ProtobufConverterOrchestrator; +import com.gotocompany.depot.maxcompute.MaxComputeSchemaHelper; +import com.gotocompany.depot.maxcompute.model.MaxComputeSchema; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.maxcompute.schema.MaxComputeSchemaCache; +import com.gotocompany.depot.maxcompute.schema.partition.DefaultPartitioningStrategy; +import com.gotocompany.depot.maxcompute.schema.partition.PartitioningStrategy; +import com.gotocompany.depot.maxcompute.schema.partition.TimestampPartitioningStrategy; +import com.gotocompany.depot.message.Message; +import com.gotocompany.depot.message.ParsedMessage; +import com.gotocompany.depot.message.SinkConnectorSchemaMessageMode; +import com.gotocompany.depot.message.proto.ProtoMessageParser; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class ProtoDataColumnRecordDecoratorTest { + + private static final Descriptors.Descriptor DESCRIPTOR = TestMaxComputeRecord.MaxComputeRecord.getDescriptor(); + + private MaxComputeSchemaHelper maxComputeSchemaHelper; + private ProtoDataColumnRecordDecorator protoDataColumnRecordDecorator; + + @Before + public void setup() throws IOException { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxcomputeMetadataNamespace()).thenReturn("__kafka_metadata"); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(Boolean.FALSE); + when(maxComputeSinkConfig.shouldAddMetadata()).thenReturn(Boolean.FALSE); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:00", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("9999-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getMaxPastYearEventTimeDifference()).thenReturn(999); + when(maxComputeSinkConfig.getMaxFutureYearEventTimeDifference()).thenReturn(999); + SinkConfig sinkConfig = Mockito.mock(SinkConfig.class); + when(sinkConfig.getSinkConnectorSchemaMessageMode()).thenReturn(SinkConnectorSchemaMessageMode.LOG_MESSAGE); + instantiateProtoDataColumnRecordDecorator(sinkConfig, maxComputeSinkConfig, null, null, getMockedMessage()); + } + + @Test + public void decorateShouldProcessDataColumnToRecord() throws IOException { + MaxComputeSchema maxComputeSchema = maxComputeSchemaHelper.build(DESCRIPTOR); + Record record = new ArrayRecord(maxComputeSchema.getTableSchema()); + RecordWrapper recordWrapper = new RecordWrapper(record, 0, null, null); + TestMaxComputeRecord.MaxComputeRecord maxComputeRecord = getMockedMessage(); + Message message = new Message(null, maxComputeRecord.toByteArray()); + LocalDateTime expectedLocalDateTime = LocalDateTime.ofEpochSecond( + 10002010L, + 1000, + java.time.ZoneOffset.UTC + ); + StructTypeInfo expectedArrayStructElementTypeInfo = (StructTypeInfo) ((ArrayTypeInfo) getDataColumnTypeByName(maxComputeSchema.getTableSchema(), "inner_record")).getElementTypeInfo(); + + RecordWrapper decoratedRecordWrapper = protoDataColumnRecordDecorator.decorate(recordWrapper, message); + + assertThat(decoratedRecordWrapper.getRecord()) + .extracting("values") + .isEqualTo(new Object[]{"id", + Arrays.asList( + new SimpleStruct(expectedArrayStructElementTypeInfo, Arrays.asList("name_1", 100.2f)), + new SimpleStruct(expectedArrayStructElementTypeInfo, Arrays.asList("name_2", 50f)) + ), + expectedLocalDateTime}); + } + + @Test + public void decorateShouldProcessDataColumnToRecordAndOmitPartitionColumnIfPartitionedByPrimitiveTypes() throws IOException { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxcomputeMetadataNamespace()).thenReturn("__kafka_metadata"); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(Boolean.TRUE); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn("id"); + when(maxComputeSinkConfig.getTablePartitionColumnName()).thenReturn("id"); + when(maxComputeSinkConfig.shouldAddMetadata()).thenReturn(Boolean.FALSE); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:00", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("9999-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + SinkConfig sinkConfig = Mockito.mock(SinkConfig.class); + when(sinkConfig.getSinkConnectorSchemaMessageMode()).thenReturn(SinkConnectorSchemaMessageMode.LOG_MESSAGE); + PartitioningStrategy partitioningStrategy = new DefaultPartitioningStrategy(TypeInfoFactory.STRING, + maxComputeSinkConfig); + instantiateProtoDataColumnRecordDecorator(sinkConfig, maxComputeSinkConfig, null, partitioningStrategy, getMockedMessage()); + MaxComputeSchema maxComputeSchema = maxComputeSchemaHelper.build(DESCRIPTOR); + Record record = new ArrayRecord(maxComputeSchema.getTableSchema()); + RecordWrapper recordWrapper = new RecordWrapper(record, 0, null, null); + TestMaxComputeRecord.MaxComputeRecord maxComputeRecord = getMockedMessage(); + Message message = new Message(null, maxComputeRecord.toByteArray()); + LocalDateTime expectedLocalDateTime = LocalDateTime.ofEpochSecond( + 10002010L, + 1000, + java.time.ZoneOffset.UTC + ); + StructTypeInfo expectedArrayStructElementTypeInfo = (StructTypeInfo) ((ArrayTypeInfo) getDataColumnTypeByName(maxComputeSchema.getTableSchema(), "inner_record")).getElementTypeInfo(); + + RecordWrapper decoratedRecordWrapper = protoDataColumnRecordDecorator.decorate(recordWrapper, message); + + assertThat(decoratedRecordWrapper.getRecord()) + .extracting("values") + .isEqualTo(new Object[]{ + Arrays.asList( + new SimpleStruct(expectedArrayStructElementTypeInfo, Arrays.asList("name_1", 100.2f)), + new SimpleStruct(expectedArrayStructElementTypeInfo, Arrays.asList("name_2", 50f)) + ), + expectedLocalDateTime}); + } + + @Test + public void decorateShouldProcessDataColumnToRecordAndShouldNotOmitOriginalColumnIfPartitionedByTimestamp() throws IOException { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxcomputeMetadataNamespace()).thenReturn("__kafka_metadata"); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(Boolean.TRUE); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn("timestamp"); + when(maxComputeSinkConfig.getTablePartitionColumnName()).thenReturn("__partition_key"); + when(maxComputeSinkConfig.getTablePartitionByTimestampTimeUnit()).thenReturn("DAY"); + when(maxComputeSinkConfig.shouldAddMetadata()).thenReturn(Boolean.FALSE); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:00", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("9999-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getMaxPastYearEventTimeDifference()).thenReturn(999); + when(maxComputeSinkConfig.getMaxFutureYearEventTimeDifference()).thenReturn(999); + SinkConfig sinkConfig = Mockito.mock(SinkConfig.class); + when(sinkConfig.getSinkConnectorSchemaMessageMode()).thenReturn(SinkConnectorSchemaMessageMode.LOG_MESSAGE); + PartitioningStrategy partitioningStrategy = new TimestampPartitioningStrategy(maxComputeSinkConfig); + instantiateProtoDataColumnRecordDecorator(sinkConfig, maxComputeSinkConfig, null, partitioningStrategy, getMockedMessage()); + MaxComputeSchema maxComputeSchema = maxComputeSchemaHelper.build(DESCRIPTOR); + Record record = new ArrayRecord(maxComputeSchema.getTableSchema()); + RecordWrapper recordWrapper = new RecordWrapper(record, 0, null, null); + TestMaxComputeRecord.MaxComputeRecord maxComputeRecord = getMockedMessage(); + Message message = new Message(null, maxComputeRecord.toByteArray()); + LocalDateTime expectedLocalDateTime = LocalDateTime.ofEpochSecond( + 10002010L, + 1000, + java.time.ZoneOffset.UTC + ); + StructTypeInfo expectedArrayStructElementTypeInfo = (StructTypeInfo) ((ArrayTypeInfo) getDataColumnTypeByName(maxComputeSchema.getTableSchema(), "inner_record")).getElementTypeInfo(); + + RecordWrapper decoratedRecordWrapper = protoDataColumnRecordDecorator.decorate(recordWrapper, message); + + assertThat(decoratedRecordWrapper.getRecord()) + .extracting("values") + .isEqualTo(new Object[]{ + "id", + Arrays.asList( + new SimpleStruct(expectedArrayStructElementTypeInfo, Arrays.asList("name_1", 100.2f)), + new SimpleStruct(expectedArrayStructElementTypeInfo, Arrays.asList("name_2", 50f)) + ), + expectedLocalDateTime}); + } + + @Test + public void decorateShouldSetDefaultPartitioningSpecWhenProtoFieldNotExists() throws IOException { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxcomputeMetadataNamespace()).thenReturn("__kafka_metadata"); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(Boolean.TRUE); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn("timestamp"); + when(maxComputeSinkConfig.getTablePartitionColumnName()).thenReturn("__partition_key"); + when(maxComputeSinkConfig.getTablePartitionByTimestampTimeUnit()).thenReturn("DAY"); + when(maxComputeSinkConfig.shouldAddMetadata()).thenReturn(Boolean.FALSE); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:00", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("9999-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + SinkConfig sinkConfig = Mockito.mock(SinkConfig.class); + when(sinkConfig.getSinkConnectorSchemaMessageMode()).thenReturn(SinkConnectorSchemaMessageMode.LOG_MESSAGE); + PartitioningStrategy partitioningStrategy = new TimestampPartitioningStrategy(maxComputeSinkConfig); + TestMaxComputeRecord.MaxComputeRecord maxComputeRecord = TestMaxComputeRecord.MaxComputeRecord + .newBuilder() + .setId("id") + .addAllInnerRecord(Arrays.asList( + TestMaxComputeRecord.InnerRecord.newBuilder() + .setName("name_1") + .setBalance(100.2f) + .build(), + TestMaxComputeRecord.InnerRecord.newBuilder() + .setName("name_2") + .setBalance(50f) + .build() + )) + .build(); + instantiateProtoDataColumnRecordDecorator(sinkConfig, maxComputeSinkConfig, null, partitioningStrategy, maxComputeRecord); + MaxComputeSchema maxComputeSchema = maxComputeSchemaHelper.build(DESCRIPTOR); + Record record = new ArrayRecord(maxComputeSchema.getTableSchema()); + RecordWrapper recordWrapper = new RecordWrapper(record, 0, null, null); + Message message = new Message(null, maxComputeRecord.toByteArray()); + StructTypeInfo expectedArrayStructElementTypeInfo = (StructTypeInfo) ((ArrayTypeInfo) getDataColumnTypeByName(maxComputeSchema.getTableSchema(), "inner_record")).getElementTypeInfo(); + + RecordWrapper decoratedWrapper = + protoDataColumnRecordDecorator.decorate(recordWrapper, message); + + assertThat(decoratedWrapper.getRecord()) + .extracting("values") + .isEqualTo(new Object[]{ + "id", + Arrays.asList( + new SimpleStruct(expectedArrayStructElementTypeInfo, Arrays.asList("name_1", 100.2f)), + new SimpleStruct(expectedArrayStructElementTypeInfo, Arrays.asList("name_2", 50f)) + ), + null}); + assertThat(decoratedWrapper.getPartitionSpec().toString()) + .isEqualTo("__partition_key='__NULL__'"); + } + + @Test + public void decorateShouldPutDefaultPartitionSpec() throws IOException { + MaxComputeSchema maxComputeSchema = maxComputeSchemaHelper.build(DESCRIPTOR); + Record record = new ArrayRecord(maxComputeSchema.getTableSchema()); + RecordWrapper recordWrapper = new RecordWrapper(record, 0, null, null); + TestMaxComputeRecord.MaxComputeRecord maxComputeRecord = getMockedMessage(); + Message message = new Message(null, maxComputeRecord.toByteArray()); + LocalDateTime expectedLocalDateTime = LocalDateTime.ofEpochSecond( + 10002010L, + 1000, + java.time.ZoneOffset.UTC + ); + StructTypeInfo expectedArrayStructElementTypeInfo = (StructTypeInfo) ((ArrayTypeInfo) getDataColumnTypeByName(maxComputeSchema.getTableSchema(), "inner_record")).getElementTypeInfo(); + + RecordWrapper decoratedRecordWrapper = protoDataColumnRecordDecorator.decorate(recordWrapper, message); + + assertThat(decoratedRecordWrapper.getRecord()) + .extracting("values") + .isEqualTo(new Object[]{"id", + Arrays.asList( + new SimpleStruct(expectedArrayStructElementTypeInfo, Arrays.asList("name_1", 100.2f)), + new SimpleStruct(expectedArrayStructElementTypeInfo, Arrays.asList("name_2", 50f)) + ), + expectedLocalDateTime}); + } + + @Test + public void decorateShouldCallInjectedDecorator() throws IOException { + RecordDecorator recordDecorator = Mockito.mock(RecordDecorator.class); + when(recordDecorator.decorate(Mockito.any(), Mockito.any())) + .thenAnswer(invocation -> invocation.getArgument(0)); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getMaxcomputeMetadataNamespace()).thenReturn("__kafka_metadata"); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(Boolean.FALSE); + when(maxComputeSinkConfig.shouldAddMetadata()).thenReturn(Boolean.FALSE); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + when(maxComputeSinkConfig.getValidMinTimestamp()).thenReturn(LocalDateTime.parse("1970-01-01T00:00:00", DateTimeFormatter.ISO_DATE_TIME)); + when(maxComputeSinkConfig.getValidMaxTimestamp()).thenReturn(LocalDateTime.parse("9999-01-01T23:59:59", DateTimeFormatter.ISO_DATE_TIME)); + SinkConfig sinkConfig = Mockito.mock(SinkConfig.class); + when(sinkConfig.getSinkConnectorSchemaMessageMode()).thenReturn(SinkConnectorSchemaMessageMode.LOG_MESSAGE); + instantiateProtoDataColumnRecordDecorator(sinkConfig, maxComputeSinkConfig, recordDecorator, null, getMockedMessage()); + MaxComputeSchema maxComputeSchema = maxComputeSchemaHelper.build(DESCRIPTOR); + Record record = new ArrayRecord(maxComputeSchema.getTableSchema()); + RecordWrapper recordWrapper = new RecordWrapper(record, 0, null, null); + TestMaxComputeRecord.MaxComputeRecord maxComputeRecord = getMockedMessage(); + Message message = new Message(null, maxComputeRecord.toByteArray()); + LocalDateTime expectedLocalDateTime = LocalDateTime.ofEpochSecond( + 10002010L, + 1000, + java.time.ZoneOffset.UTC); + StructTypeInfo expectedArrayStructElementTypeInfo = (StructTypeInfo) ((ArrayTypeInfo) getDataColumnTypeByName(maxComputeSchema.getTableSchema(), "inner_record")).getElementTypeInfo(); + + RecordWrapper decoratedRecord = protoDataColumnRecordDecorator.decorate(recordWrapper, message); + + assertThat(decoratedRecord.getRecord()) + .extracting("values") + .isEqualTo(new Object[]{"id", + Arrays.asList( + new SimpleStruct(expectedArrayStructElementTypeInfo, Arrays.asList("name_1", 100.2f)), + new SimpleStruct(expectedArrayStructElementTypeInfo, Arrays.asList("name_2", 50f)) + ), + expectedLocalDateTime}); + verify(recordDecorator, Mockito.times(1)) + .decorate(Mockito.any(), Mockito.any()); + } + + private void instantiateProtoDataColumnRecordDecorator(SinkConfig sinkConfig, MaxComputeSinkConfig maxComputeSinkConfig, + RecordDecorator recordDecorator, + PartitioningStrategy partitioningStrategy, + com.google.protobuf.Message mockedMessage) throws IOException { + ProtobufConverterOrchestrator protobufConverterOrchestrator = new ProtobufConverterOrchestrator(maxComputeSinkConfig); + maxComputeSchemaHelper = new MaxComputeSchemaHelper( + protobufConverterOrchestrator, + maxComputeSinkConfig, + partitioningStrategy + ); + MaxComputeSchema maxComputeSchema = maxComputeSchemaHelper.build(DESCRIPTOR); + MaxComputeSchemaCache maxComputeSchemaCache = Mockito.mock(MaxComputeSchemaCache.class); + when(maxComputeSchemaCache.getMaxComputeSchema()).thenReturn(maxComputeSchema); + ProtoMessageParser protoMessageParser = Mockito.mock(ProtoMessageParser.class); + ParsedMessage parsedMessage = Mockito.mock(ParsedMessage.class); + when(parsedMessage.getRaw()).thenReturn(mockedMessage); + when(protoMessageParser.parse(Mockito.any(), Mockito.any(), Mockito.any())) + .thenReturn(parsedMessage); + protoDataColumnRecordDecorator = new ProtoDataColumnRecordDecorator( + recordDecorator, + protobufConverterOrchestrator, + protoMessageParser, + sinkConfig, + partitioningStrategy + ); + } + + private static TypeInfo getDataColumnTypeByName(TableSchema tableSchema, String columnName) { + return tableSchema.getColumns() + .stream() + .filter(column -> column.getName().equals(columnName)) + .findFirst() + .map(com.aliyun.odps.Column::getTypeInfo) + .orElse(null); + } + + private static TestMaxComputeRecord.MaxComputeRecord getMockedMessage() { + return TestMaxComputeRecord.MaxComputeRecord + .newBuilder() + .setId("id") + .addAllInnerRecord(Arrays.asList( + TestMaxComputeRecord.InnerRecord.newBuilder() + .setName("name_1") + .setBalance(100.2f) + .build(), + TestMaxComputeRecord.InnerRecord.newBuilder() + .setName("name_2") + .setBalance(50f) + .build() + )) + .setTimestamp(Timestamp.newBuilder() + .setSeconds(10002010) + .setNanos(1000) + .build()) + .build(); + } +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/record/ProtoMetadataColumnRecordDecoratorTest.java b/src/test/java/com/gotocompany/depot/maxcompute/record/ProtoMetadataColumnRecordDecoratorTest.java new file mode 100644 index 00000000..14588fd3 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/record/ProtoMetadataColumnRecordDecoratorTest.java @@ -0,0 +1,149 @@ +package com.gotocompany.depot.maxcompute.record; + +import com.aliyun.odps.data.ArrayRecord; +import com.aliyun.odps.data.Record; +import com.aliyun.odps.data.SimpleStruct; +import com.aliyun.odps.type.TypeInfoFactory; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Timestamp; +import com.gotocompany.depot.TestMaxComputeRecord; +import com.gotocompany.depot.common.Tuple; +import com.gotocompany.depot.common.TupleString; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.converter.ProtobufConverterOrchestrator; +import com.gotocompany.depot.maxcompute.MaxComputeSchemaHelper; +import com.gotocompany.depot.maxcompute.model.MaxComputeSchema; +import com.gotocompany.depot.maxcompute.model.RecordWrapper; +import com.gotocompany.depot.maxcompute.schema.MaxComputeSchemaCache; +import com.gotocompany.depot.message.Message; +import com.gotocompany.depot.message.proto.ProtoParsedMessage; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.when; + +public class ProtoMetadataColumnRecordDecoratorTest { + + private final Descriptors.Descriptor descriptor = TestMaxComputeRecord.MaxComputeRecord.getDescriptor(); + + private MaxComputeSinkConfig maxComputeSinkConfig; + private MaxComputeSchemaCache maxComputeSchemaCache; + private ProtoMetadataColumnRecordDecorator protoMetadataColumnRecordDecorator; + + @Before + public void setup() { + MaxComputeSinkConfig config = Mockito.mock(MaxComputeSinkConfig.class); + when(config.isTablePartitioningEnabled()).thenReturn(Boolean.FALSE); + when(config.shouldAddMetadata()).thenReturn(Boolean.TRUE); + when(config.getMaxcomputeMetadataNamespace()).thenReturn("__kafka_metadata"); + when(config.getMetadataColumnsTypes()).thenReturn(Arrays.asList( + new TupleString("__message_timestamp", "timestamp"), + new TupleString("__kafka_topic", "string"), + new TupleString("__kafka_offset", "long") + )); + when(config.getZoneId()).thenReturn(ZoneId.of("UTC")); + initializeDecorator(config); + } + + @Test + public void shouldPopulateRecordWithNamespacedMetadata() throws IOException { + Message message = new Message( + null, + new ProtoParsedMessage(getMockedMessage(), null), + new Tuple<>("__message_timestamp", 10002010L), + new Tuple<>("__kafka_topic", "topic"), + new Tuple<>("__kafka_offset", 100L) + ); + Record record = new ArrayRecord(maxComputeSchemaCache.getMaxComputeSchema().getColumns()); + RecordWrapper recordWrapper = new RecordWrapper(record, 0, null, null); + LocalDateTime expectedLocalDateTime = Instant.ofEpochMilli(10002010L) + .atZone(ZoneId.of("UTC")) + .toLocalDateTime(); + + protoMetadataColumnRecordDecorator.decorate(recordWrapper, message); + + assertThat(record.get(maxComputeSinkConfig.getMaxcomputeMetadataNamespace())) + .isEqualTo(new SimpleStruct( + TypeInfoFactory.getStructTypeInfo(Arrays.asList("__message_timestamp", "__kafka_topic", "__kafka_offset"), + Arrays.asList(TypeInfoFactory.TIMESTAMP, TypeInfoFactory.STRING, TypeInfoFactory.BIGINT)), + Arrays.asList(expectedLocalDateTime, "topic", 100L) + )); + } + + @Test + public void shouldPopulateRecordWithNonNamespacedMetadata() throws IOException { + MaxComputeSinkConfig mcSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(mcSinkConfig.isTablePartitioningEnabled()).thenReturn(Boolean.FALSE); + when(mcSinkConfig.shouldAddMetadata()).thenReturn(Boolean.TRUE); + when(mcSinkConfig.getMetadataColumnsTypes()).thenReturn(Arrays.asList( + new TupleString("__message_timestamp", "timestamp"), + new TupleString("__kafka_topic", "string"), + new TupleString("__kafka_offset", "long") + )); + when(mcSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + initializeDecorator(mcSinkConfig); + Message message = new Message( + null, + new ProtoParsedMessage(getMockedMessage(), null), + new Tuple<>("__message_timestamp", 10002010L), + new Tuple<>("__kafka_topic", "topic"), + new Tuple<>("__kafka_offset", 100L) + ); + Record record = new ArrayRecord(maxComputeSchemaCache.getMaxComputeSchema().getColumns()); + RecordWrapper recordWrapper = new RecordWrapper(record, 0, null, null); + LocalDateTime expectedLocalDateTime = Instant.ofEpochMilli(10002010L) + .atZone(ZoneId.of("UTC")) + .toLocalDateTime(); + + protoMetadataColumnRecordDecorator.decorate(recordWrapper, message); + + assertThat(record) + .satisfies(r -> { + assertThat(r.get("__message_timestamp")) + .isEqualTo(expectedLocalDateTime); + assertThat(r.get("__kafka_topic")) + .isEqualTo("topic"); + assertThat(r.get("__kafka_offset")) + .isEqualTo(100L); + }); + } + + private static TestMaxComputeRecord.MaxComputeRecord getMockedMessage() { + return TestMaxComputeRecord.MaxComputeRecord + .newBuilder() + .setId("id") + .addAllInnerRecord(Arrays.asList( + TestMaxComputeRecord.InnerRecord.newBuilder() + .setName("name_1") + .setBalance(100.2f) + .build(), + TestMaxComputeRecord.InnerRecord.newBuilder() + .setName("name_2") + .setBalance(50f) + .build() + )) + .setTimestamp(Timestamp.newBuilder() + .setSeconds(10002010) + .setNanos(1000) + .build()) + .build(); + } + + private void initializeDecorator(MaxComputeSinkConfig sinkConfig) { + this.maxComputeSinkConfig = sinkConfig; + ProtobufConverterOrchestrator protobufConverterOrchestrator = new ProtobufConverterOrchestrator(sinkConfig); + MaxComputeSchemaHelper maxComputeSchemaHelper = new MaxComputeSchemaHelper(protobufConverterOrchestrator, sinkConfig, null); + MaxComputeSchema maxComputeSchema = maxComputeSchemaHelper.build(descriptor); + maxComputeSchemaCache = Mockito.mock(MaxComputeSchemaCache.class); + when(maxComputeSchemaCache.getMaxComputeSchema()).thenReturn(maxComputeSchema); + protoMetadataColumnRecordDecorator = new ProtoMetadataColumnRecordDecorator(null, sinkConfig, maxComputeSchemaCache); + } +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/record/RecordDecoratorFactoryTest.java b/src/test/java/com/gotocompany/depot/maxcompute/record/RecordDecoratorFactoryTest.java new file mode 100644 index 00000000..7beee0ef --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/record/RecordDecoratorFactoryTest.java @@ -0,0 +1,41 @@ +package com.gotocompany.depot.maxcompute.record; + +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import org.junit.Test; +import org.mockito.Mockito; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.when; + +public class RecordDecoratorFactoryTest { + + @Test + public void shouldCreateDataRecordDecorator() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.shouldAddMetadata()).thenReturn(Boolean.FALSE); + + RecordDecorator recordDecorator = RecordDecoratorFactory.createRecordDecorator( + null, null, null, null, maxComputeSinkConfig, null + ); + + assertThat(recordDecorator) + .isInstanceOf(ProtoDataColumnRecordDecorator.class) + .extracting("decorator") + .isNull(); + } + + @Test + public void shouldCreateDataRecordDecoratorWithNamespaceDecorator() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.shouldAddMetadata()).thenReturn(Boolean.TRUE); + + RecordDecorator recordDecorator = RecordDecoratorFactory.createRecordDecorator( + null, null, null, null, maxComputeSinkConfig, null + ); + assertThat(recordDecorator) + .isInstanceOf(ProtoMetadataColumnRecordDecorator.class) + .extracting("decorator") + .isNotNull() + .isInstanceOf(ProtoDataColumnRecordDecorator.class); + } +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/schema/MaxComputeSchemaCacheTest.java b/src/test/java/com/gotocompany/depot/maxcompute/schema/MaxComputeSchemaCacheTest.java new file mode 100644 index 00000000..ecaf6dd8 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/schema/MaxComputeSchemaCacheTest.java @@ -0,0 +1,214 @@ +package com.gotocompany.depot.maxcompute.schema; + +import com.aliyun.odps.OdpsException; +import com.aliyun.odps.TableSchema; +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.config.SinkConfig; +import com.gotocompany.depot.maxcompute.client.MaxComputeClient; +import com.gotocompany.depot.maxcompute.converter.ProtobufConverterOrchestrator; +import com.gotocompany.depot.maxcompute.exception.MaxComputeTableOperationException; +import com.gotocompany.depot.maxcompute.MaxComputeSchemaHelper; +import com.gotocompany.depot.maxcompute.model.MaxComputeSchema; +import com.gotocompany.depot.message.SinkConnectorSchemaMessageMode; +import com.gotocompany.depot.message.proto.ProtoMessageParser; +import org.junit.Test; +import org.mockito.Mockito; + +import java.lang.reflect.Field; +import java.time.ZoneId; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class MaxComputeSchemaCacheTest { + + @Test + public void shouldBuildAndReturnMaxComputeSchema() throws OdpsException { + Map newDescriptor = new HashMap<>(); + newDescriptor.put("class", Mockito.mock(Descriptors.Descriptor.class)); + MaxComputeSchemaHelper maxComputeSchemaHelper = Mockito.mock(MaxComputeSchemaHelper.class); + ProtoMessageParser protoMessageParser = Mockito.mock(ProtoMessageParser.class); + when(protoMessageParser.getDescriptorMap()).thenReturn(newDescriptor); + MaxComputeSchema mockedMaxComputeSchema = new MaxComputeSchema(null, null); + when(maxComputeSchemaHelper.build(Mockito.any())) + .thenReturn(mockedMaxComputeSchema); + SinkConfig sinkConfig = Mockito.mock(SinkConfig.class); + when(sinkConfig.getSinkConnectorSchemaMessageMode()) + .thenReturn(SinkConnectorSchemaMessageMode.LOG_MESSAGE); + when(sinkConfig.getSinkConnectorSchemaProtoMessageClass()) + .thenReturn("class"); + when(sinkConfig.getSinkConnectorSchemaProtoKeyClass()) + .thenReturn("class"); + MaxComputeClient maxComputeClient = Mockito.spy(MaxComputeClient.class); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + MaxComputeSchemaCache maxComputeSchemaCache = new MaxComputeSchemaCache( + maxComputeSchemaHelper, + sinkConfig, + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeClient + ); + maxComputeSchemaCache.setMessageParser(protoMessageParser); + Mockito.doNothing() + .when(maxComputeClient) + .createOrUpdateTable(Mockito.any()); + TableSchema finalMockedTableSchema = Mockito.mock(TableSchema.class); + Mockito.doReturn(finalMockedTableSchema) + .when(maxComputeClient) + .getLatestTableSchema(); + + MaxComputeSchema maxComputeSchema = maxComputeSchemaCache.getMaxComputeSchema(); + + verify(maxComputeClient, Mockito.times(1)) + .createOrUpdateTable(Mockito.any()); + assertEquals(finalMockedTableSchema, maxComputeSchema.getTableSchema()); + } + + @Test + public void shouldReturnMaxComputeSchemaIfExists() throws OdpsException, NoSuchFieldException, IllegalAccessException { + Map newDescriptor = new HashMap<>(); + newDescriptor.put("class", Mockito.mock(Descriptors.Descriptor.class)); + MaxComputeSchemaHelper maxComputeSchemaHelper = Mockito.mock(MaxComputeSchemaHelper.class); + ProtoMessageParser protoMessageParser = Mockito.mock(ProtoMessageParser.class); + when(protoMessageParser.getDescriptorMap()).thenReturn(newDescriptor); + MaxComputeSchema mockedMaxComputeSchema = new MaxComputeSchema(null, null); + when(maxComputeSchemaHelper.build(Mockito.any())) + .thenReturn(mockedMaxComputeSchema); + SinkConfig sinkConfig = Mockito.mock(SinkConfig.class); + when(sinkConfig.getSinkConnectorSchemaMessageMode()) + .thenReturn(SinkConnectorSchemaMessageMode.LOG_MESSAGE); + when(sinkConfig.getSinkConnectorSchemaProtoMessageClass()) + .thenReturn("class"); + MaxComputeClient maxComputeClient = Mockito.spy(MaxComputeClient.class); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + MaxComputeSchemaCache maxComputeSchemaCache = new MaxComputeSchemaCache( + maxComputeSchemaHelper, + sinkConfig, + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeClient + ); + Field field = MaxComputeSchemaCache.class.getDeclaredField("maxComputeSchema"); + field.setAccessible(true); + field.set(maxComputeSchemaCache, mockedMaxComputeSchema); + + MaxComputeSchema maxComputeSchema = maxComputeSchemaCache.getMaxComputeSchema(); + + verify(maxComputeClient, Mockito.times(0)) + .createOrUpdateTable(Mockito.any()); + assertEquals(mockedMaxComputeSchema, maxComputeSchema); + } + + @Test + public void shouldUpdateSchemaBasedOnNewDescriptor() throws OdpsException { + Map newDescriptor = new HashMap<>(); + newDescriptor.put("class", Mockito.mock(Descriptors.Descriptor.class)); + MaxComputeSchemaHelper maxComputeSchemaHelper = Mockito.mock(MaxComputeSchemaHelper.class); + ProtoMessageParser protoMessageParser = Mockito.mock(ProtoMessageParser.class); + MaxComputeSchema mockedMaxComputeSchema = new MaxComputeSchema(null, null); + when(maxComputeSchemaHelper.build(Mockito.any())) + .thenReturn(mockedMaxComputeSchema); + SinkConfig sinkConfig = Mockito.mock(SinkConfig.class); + when(sinkConfig.getSinkConnectorSchemaMessageMode()) + .thenReturn(SinkConnectorSchemaMessageMode.LOG_MESSAGE); + when(sinkConfig.getSinkConnectorSchemaProtoMessageClass()) + .thenReturn("class"); + MaxComputeClient maxComputeClient = Mockito.spy(MaxComputeClient.class); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + MaxComputeSchemaCache maxComputeSchemaCache = new MaxComputeSchemaCache( + maxComputeSchemaHelper, + sinkConfig, + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeClient + ); + maxComputeSchemaCache.setMessageParser(protoMessageParser); + Mockito.doNothing() + .when(maxComputeClient) + .createOrUpdateTable(Mockito.any()); + Mockito.doReturn(Mockito.mock(TableSchema.class)) + .when(maxComputeClient) + .getLatestTableSchema(); + + maxComputeSchemaCache.getMaxComputeSchema(); + maxComputeSchemaCache.onSchemaUpdate(newDescriptor); + + verify(maxComputeClient, Mockito.times(2)) + .createOrUpdateTable(Mockito.any()); + } + + @Test + public void shouldUpdateSchemaUsingLogKeyBasedOnNewDescriptor() throws OdpsException { + Map newDescriptor = new HashMap<>(); + newDescriptor.put("class", Mockito.mock(Descriptors.Descriptor.class)); + MaxComputeSchemaHelper maxComputeSchemaHelper = Mockito.mock(MaxComputeSchemaHelper.class); + ProtoMessageParser protoMessageParser = Mockito.mock(ProtoMessageParser.class); + MaxComputeSchema mockedMaxComputeSchema = new MaxComputeSchema(null, null); + when(maxComputeSchemaHelper.build(Mockito.any())) + .thenReturn(mockedMaxComputeSchema); + SinkConfig sinkConfig = Mockito.mock(SinkConfig.class); + when(sinkConfig.getSinkConnectorSchemaMessageMode()) + .thenReturn(SinkConnectorSchemaMessageMode.LOG_KEY); + when(sinkConfig.getSinkConnectorSchemaProtoKeyClass()) + .thenReturn("class"); + MaxComputeClient maxComputeClient = Mockito.spy(MaxComputeClient.class); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + MaxComputeSchemaCache maxComputeSchemaCache = new MaxComputeSchemaCache( + maxComputeSchemaHelper, + sinkConfig, + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeClient + ); + maxComputeSchemaCache.setMessageParser(protoMessageParser); + Mockito.doNothing() + .when(maxComputeClient) + .createOrUpdateTable(Mockito.any()); + Mockito.doReturn(Mockito.mock(TableSchema.class)) + .when(maxComputeClient) + .getLatestTableSchema(); + + maxComputeSchemaCache.getMaxComputeSchema(); + maxComputeSchemaCache.onSchemaUpdate(newDescriptor); + + verify(maxComputeClient, Mockito.times(2)) + .createOrUpdateTable(Mockito.any()); + } + + @Test(expected = MaxComputeTableOperationException.class) + public void shouldThrowMaxComputeTableOperationExceptionWhenUpsertIsFailing() throws OdpsException { + Map newDescriptor = new HashMap<>(); + newDescriptor.put("class", Mockito.mock(Descriptors.Descriptor.class)); + MaxComputeSchemaHelper maxComputeSchemaHelper = Mockito.mock(MaxComputeSchemaHelper.class); + ProtoMessageParser protoMessageParser = Mockito.mock(ProtoMessageParser.class); + MaxComputeSchema mockedMaxComputeSchema = new MaxComputeSchema(null, null); + when(maxComputeSchemaHelper.build(Mockito.any())) + .thenReturn(mockedMaxComputeSchema); + SinkConfig sinkConfig = Mockito.mock(SinkConfig.class); + when(sinkConfig.getSinkConnectorSchemaMessageMode()) + .thenReturn(SinkConnectorSchemaMessageMode.LOG_MESSAGE); + when(sinkConfig.getSinkConnectorSchemaProtoMessageClass()) + .thenReturn("class"); + MaxComputeClient maxComputeClient = Mockito.spy(MaxComputeClient.class); + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + MaxComputeSchemaCache maxComputeSchemaCache = new MaxComputeSchemaCache( + maxComputeSchemaHelper, + sinkConfig, + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeClient + ); + maxComputeSchemaCache.setMessageParser(protoMessageParser); + Mockito.doThrow(new OdpsException("Invalid schema")) + .when(maxComputeClient) + .createOrUpdateTable(Mockito.any()); + + maxComputeSchemaCache.getMaxComputeSchema(); + maxComputeSchemaCache.onSchemaUpdate(newDescriptor); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/schema/partition/DefaultPartitioningStrategyTest.java b/src/test/java/com/gotocompany/depot/maxcompute/schema/partition/DefaultPartitioningStrategyTest.java new file mode 100644 index 00000000..adff1dc8 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/schema/partition/DefaultPartitioningStrategyTest.java @@ -0,0 +1,72 @@ +package com.gotocompany.depot.maxcompute.schema.partition; + +import com.aliyun.odps.Column; +import com.aliyun.odps.type.TypeInfoFactory; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import org.junit.Test; +import org.mockito.Mockito; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.when; + +public class DefaultPartitioningStrategyTest { + @Test + public void shouldReturnOriginalPartitionColumnName() { + DefaultPartitioningStrategy defaultPartitioningStrategy = new DefaultPartitioningStrategy(TypeInfoFactory.STRING, + getMaxComputeSinkConfig()); + + assertEquals("tablePartitionKey", defaultPartitioningStrategy.getOriginalPartitionColumnName()); + } + + @Test + public void shouldReturnPartitionColumn() { + MaxComputeSinkConfig maxComputeSinkConfig = getMaxComputeSinkConfig(); + DefaultPartitioningStrategy defaultPartitioningStrategy = new DefaultPartitioningStrategy(TypeInfoFactory.STRING, + maxComputeSinkConfig); + Column expectedColumn = Column.newBuilder(maxComputeSinkConfig.getTablePartitionColumnName(), TypeInfoFactory.STRING) + .build(); + + assertEquals(expectedColumn, defaultPartitioningStrategy.getPartitionColumn()); + } + + @Test + public void shouldReturnTrueForReplacingOriginalColumn() { + DefaultPartitioningStrategy defaultPartitioningStrategy = new DefaultPartitioningStrategy(TypeInfoFactory.STRING, + getMaxComputeSinkConfig()); + + assertTrue(defaultPartitioningStrategy.shouldReplaceOriginalColumn()); + } + + @Test + public void shouldReturnValidPartitionSpec() { + DefaultPartitioningStrategy defaultPartitioningStrategy = new DefaultPartitioningStrategy(TypeInfoFactory.STRING, + getMaxComputeSinkConfig()); + String partitionKey = "object"; + String expectedPartitionSpecStringRepresentation = "tablePartitionColumnName='object'"; + + assertEquals(expectedPartitionSpecStringRepresentation, + defaultPartitioningStrategy.getPartitionSpec(partitionKey) + .toString()); + } + + @Test + public void shouldReturnDefaultPartitionSpec() { + String expectedPartitionSpecStringRepresentation = "tablePartitionColumnName='__NULL__'"; + DefaultPartitioningStrategy defaultPartitioningStrategy = new DefaultPartitioningStrategy(TypeInfoFactory.STRING, + getMaxComputeSinkConfig()); + + assertEquals(expectedPartitionSpecStringRepresentation, + defaultPartitioningStrategy.getPartitionSpec(null) + .toString()); + } + + private MaxComputeSinkConfig getMaxComputeSinkConfig() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getTablePartitionColumnName()) + .thenReturn("tablePartitionColumnName"); + when(maxComputeSinkConfig.getTablePartitionKey()) + .thenReturn("tablePartitionKey"); + return maxComputeSinkConfig; + } +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/schema/partition/PartitioningStrategyFactoryTest.java b/src/test/java/com/gotocompany/depot/maxcompute/schema/partition/PartitioningStrategyFactoryTest.java new file mode 100644 index 00000000..0b5dab5a --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/schema/partition/PartitioningStrategyFactoryTest.java @@ -0,0 +1,104 @@ +package com.gotocompany.depot.maxcompute.schema.partition; + +import com.google.protobuf.Descriptors; +import com.gotocompany.depot.TestMaxComputePartition; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.converter.ProtobufConverterOrchestrator; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +import java.time.ZoneId; + +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.when; + +public class PartitioningStrategyFactoryTest { + + private final Descriptors.Descriptor descriptor = TestMaxComputePartition.MaxComputePartition.getDescriptor(); + + @Test + public void shouldReturnDefaultPartitionStrategy() { + String stringFieldName = "string_field"; + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(true); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn(stringFieldName); + when(maxComputeSinkConfig.getTablePartitionColumnName()).thenReturn(stringFieldName); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + + PartitioningStrategy partitioningStrategy = PartitioningStrategyFactory.createPartitioningStrategy( + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeSinkConfig, + descriptor + ); + + assertTrue(partitioningStrategy instanceof DefaultPartitioningStrategy); + } + + @Test + public void shouldReturnTimestampPartitionStrategy() { + String timestampFieldName = "timestamp_field"; + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(true); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn(timestampFieldName); + when(maxComputeSinkConfig.getTablePartitionColumnName()).thenReturn(timestampFieldName); + when(maxComputeSinkConfig.getTablePartitionByTimestampTimeUnit()).thenReturn("DAY"); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + + PartitioningStrategy partitioningStrategy = PartitioningStrategyFactory.createPartitioningStrategy( + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeSinkConfig, + descriptor + ); + + assertTrue(partitioningStrategy instanceof TimestampPartitioningStrategy); + } + + @Test + public void shouldReturnNull() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(Boolean.FALSE); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + + PartitioningStrategy partitioningStrategy = PartitioningStrategyFactory.createPartitioningStrategy( + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeSinkConfig, + descriptor + ); + + Assert.assertNull(partitioningStrategy); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIllegalArgumentExceptionWhenTypeInfoIsNotSupported() { + String unsupportedTypeFieldName = "float_field"; + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(true); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn(unsupportedTypeFieldName); + when(maxComputeSinkConfig.getTablePartitionColumnName()).thenReturn(unsupportedTypeFieldName); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + + PartitioningStrategyFactory.createPartitioningStrategy( + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeSinkConfig, + descriptor + ); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIllegalArgumentExceptionWhenFieldIsNotFoundInDescriptor() { + String fieldName = "non_existent_field"; + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.isTablePartitioningEnabled()).thenReturn(true); + when(maxComputeSinkConfig.getTablePartitionKey()).thenReturn(fieldName); + when(maxComputeSinkConfig.getTablePartitionColumnName()).thenReturn(fieldName); + when(maxComputeSinkConfig.getZoneId()).thenReturn(ZoneId.of("UTC")); + + PartitioningStrategyFactory.createPartitioningStrategy( + new ProtobufConverterOrchestrator(maxComputeSinkConfig), + maxComputeSinkConfig, + descriptor + ); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/schema/partition/TimestampPartitioningStrategyTest.java b/src/test/java/com/gotocompany/depot/maxcompute/schema/partition/TimestampPartitioningStrategyTest.java new file mode 100644 index 00000000..434f6532 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/schema/partition/TimestampPartitioningStrategyTest.java @@ -0,0 +1,132 @@ +package com.gotocompany.depot.maxcompute.schema.partition; + +import com.aliyun.odps.Column; +import com.aliyun.odps.TableSchema; +import com.aliyun.odps.data.ArrayRecord; +import com.aliyun.odps.data.Record; +import com.aliyun.odps.expression.TruncTime; +import com.aliyun.odps.type.TypeInfoFactory; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import com.gotocompany.depot.maxcompute.model.MaxComputeSchema; +import com.gotocompany.depot.maxcompute.schema.MaxComputeSchemaCache; +import org.junit.Test; +import org.mockito.Mockito; + +import java.time.LocalDateTime; +import java.time.ZoneOffset; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.mockito.Mockito.when; + +public class TimestampPartitioningStrategyTest { + + @Test + public void shouldReturnOriginalPartitionColumnName() { + TimestampPartitioningStrategy defaultPartitioningStrategy = + new TimestampPartitioningStrategy(getMaxComputeSinkConfig()); + + assertEquals("event_timestamp", + defaultPartitioningStrategy.getOriginalPartitionColumnName()); + } + + @Test + public void shouldReturnFalseForReplacingOriginalColumn() { + TimestampPartitioningStrategy defaultPartitioningStrategy = + new TimestampPartitioningStrategy(getMaxComputeSinkConfig()); + + assertFalse(defaultPartitioningStrategy.shouldReplaceOriginalColumn()); + } + + @Test + public void shouldReturnValidColumn() { + MaxComputeSinkConfig maxComputeSinkConfig = getMaxComputeSinkConfig(); + TimestampPartitioningStrategy timestampPartitioningStrategy = + new TimestampPartitioningStrategy(maxComputeSinkConfig); + + Column column = Column.newBuilder(maxComputeSinkConfig.getTablePartitionColumnName(), TypeInfoFactory.STRING) + .build(); + + assertEquals(column, timestampPartitioningStrategy.getPartitionColumn()); + } + + @Test + public void shouldReturnValidPartitionSpec() { + //October 29, 2024 12:00:10 AM GMT+07:00 + long epoch = 1730134810; + MaxComputeSinkConfig maxComputeSinkConfig = getMaxComputeSinkConfig(); + TimestampPartitioningStrategy timestampPartitioningStrategy = + new TimestampPartitioningStrategy(maxComputeSinkConfig); + Column partitionColumn = Column.newBuilder("tablePartitionColumnName", TypeInfoFactory.STRING) + .build(); + partitionColumn.setGenerateExpression(new TruncTime("event_timestamp", "DAY")); + TableSchema tableSchema = TableSchema.builder() + .withStringColumn("str") + .withColumn(Column.newBuilder("event_timestamp", TypeInfoFactory.TIMESTAMP_NTZ) + .build()) + .withPartitionColumn(partitionColumn) + .build(); + MaxComputeSchemaCache maxComputeSchemaCache = Mockito.mock(MaxComputeSchemaCache.class); + MaxComputeSchema maxComputeSchema = Mockito.mock(MaxComputeSchema.class); + when(maxComputeSchema.getTableSchema()).thenReturn(tableSchema); + when(maxComputeSchemaCache.getMaxComputeSchema()) + .thenReturn(maxComputeSchema); + String expectedStartOfDayEpoch = "2024-10-28"; + Record record = new ArrayRecord(tableSchema); + record.set("str", "strVal"); + record.set("event_timestamp", LocalDateTime.ofEpochSecond(epoch, 0, ZoneOffset.UTC)); + + assertEquals(String.format("tablePartitionColumnName='%s'", expectedStartOfDayEpoch), + timestampPartitioningStrategy.getPartitionSpec(record).toString()); + } + + @Test + public void shouldEmptyPartitionSpecIfObjectIsNotRecord() { + MaxComputeSinkConfig maxComputeSinkConfig = getMaxComputeSinkConfig(); + TimestampPartitioningStrategy timestampPartitioningStrategy = + new TimestampPartitioningStrategy(maxComputeSinkConfig); + + assertEquals("", + timestampPartitioningStrategy.getPartitionSpec("").toString()); + } + + @Test + public void shouldReturnDefaultPartitionSpec() { + String expectedPartitionSpecStringRepresentation = "tablePartitionColumnName='__NULL__'"; + TimestampPartitioningStrategy timestampPartitioningStrategy = new TimestampPartitioningStrategy(getMaxComputeSinkConfig()); + MaxComputeSchemaCache maxComputeSchemaCache = Mockito.mock(MaxComputeSchemaCache.class); + MaxComputeSchema maxComputeSchema = Mockito.mock(MaxComputeSchema.class); + Column partitionColumn = Column.newBuilder("tablePartitionColumnName", TypeInfoFactory.STRING) + .build(); + partitionColumn.setGenerateExpression(new TruncTime("event_timestamp", "DAY")); + TableSchema tableSchema = TableSchema.builder() + .withStringColumn("str") + .withDatetimeColumn("event_timestamp") + .withPartitionColumn(partitionColumn) + .build(); + when(maxComputeSchema.getTableSchema()).thenReturn(tableSchema); + when(maxComputeSchemaCache.getMaxComputeSchema()) + .thenReturn(maxComputeSchema); + Record record = new ArrayRecord(tableSchema); + record.set("str", "strVal"); + record.set("event_timestamp", null); + + assertEquals(expectedPartitionSpecStringRepresentation, + timestampPartitioningStrategy.getPartitionSpec(record) + .toString()); + } + + private MaxComputeSinkConfig getMaxComputeSinkConfig() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.isTablePartitioningEnabled()) + .thenReturn(Boolean.TRUE); + when(maxComputeSinkConfig.getTablePartitionColumnName()) + .thenReturn("tablePartitionColumnName"); + when(maxComputeSinkConfig.getTablePartitionKey()) + .thenReturn("event_timestamp"); + when(maxComputeSinkConfig.getTablePartitionByTimestampTimeUnit()) + .thenReturn("DAY"); + return maxComputeSinkConfig; + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/schema/validator/TableValidatorTest.java b/src/test/java/com/gotocompany/depot/maxcompute/schema/validator/TableValidatorTest.java new file mode 100644 index 00000000..48c14a1f --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/schema/validator/TableValidatorTest.java @@ -0,0 +1,66 @@ +package com.gotocompany.depot.maxcompute.schema.validator; + +import com.aliyun.odps.Column; +import com.aliyun.odps.TableSchema; +import com.aliyun.odps.type.TypeInfoFactory; +import com.gotocompany.depot.config.MaxComputeSinkConfig; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import static org.mockito.Mockito.when; + +public class TableValidatorTest { + + private TableValidator tableValidator; + + @Before + public void init() { + MaxComputeSinkConfig maxComputeSinkConfig = Mockito.mock(MaxComputeSinkConfig.class); + when(maxComputeSinkConfig.getTableValidatorNameRegex()).thenReturn("^[a-zA-Z_][a-zA-Z0-9_]{0,29}$"); + when(maxComputeSinkConfig.getTableValidatorMaxColumnsPerTable()).thenReturn(1200); + when(maxComputeSinkConfig.getTableValidatorMaxPartitionKeysPerTable()).thenReturn(6); + tableValidator = new TableValidator(maxComputeSinkConfig); + } + + @Test + public void shouldValidateValidTableName() { + TableSchema tableSchema = new TableSchema(); + tableSchema.addColumn(new Column("column1", TypeInfoFactory.STRING)); + tableValidator.validate("ValidTableName", null, tableSchema); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldValidateInvalidTableName() { + TableSchema tableSchema = new TableSchema(); + tableSchema.addColumn(new Column("column1", TypeInfoFactory.STRING)); + tableValidator.validate("1InvalidTableName", 30L, tableSchema); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldValidateNegativeLifecycleDays() { + TableSchema tableSchema = new TableSchema(); + tableSchema.addColumn(new Column("column1", TypeInfoFactory.STRING)); + tableValidator.validate("ValidTableName", -1L, tableSchema); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldValidateExceedMaxColumns() { + TableSchema tableSchema = new TableSchema(); + for (int i = 0; i < 1201; i++) { + tableSchema.addColumn(new Column("column" + i, TypeInfoFactory.STRING)); + } + tableValidator.validate("ValidTableName", 30L, tableSchema); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldValidateExceedMaxPartitionKeys() { + TableSchema tableSchema = new TableSchema(); + tableSchema.addColumn(new Column("column1", TypeInfoFactory.STRING)); + for (int i = 0; i < 7; i++) { + tableSchema.addPartitionColumn(new Column("partition" + i, TypeInfoFactory.STRING)); + } + tableValidator.validate("ValidTableName", 30L, tableSchema); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/util/MetadataUtilTest.java b/src/test/java/com/gotocompany/depot/maxcompute/util/MetadataUtilTest.java new file mode 100644 index 00000000..0c0eb893 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/util/MetadataUtilTest.java @@ -0,0 +1,41 @@ +package com.gotocompany.depot.maxcompute.util; + +import com.aliyun.odps.type.StructTypeInfo; +import com.aliyun.odps.type.TypeInfoFactory; +import com.gotocompany.depot.common.TupleString; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +public class MetadataUtilTest { + + @Test + public void shouldReturnAppropriateStructTypeInfoForNamespacedMetadata() { + List metadataColumnTypes = Arrays.asList(new TupleString("__message_timestamp", "timestamp"), + new TupleString("__kafka_topic", "string"), + new TupleString("__kafka_offset", "long") + ); + + StructTypeInfo structTypeInfo = MetadataUtil.getMetadataTypeInfo(metadataColumnTypes); + + assertThat(structTypeInfo.getFieldNames()).containsExactlyInAnyOrder("__message_timestamp", "__kafka_topic", "__kafka_offset"); + assertThat(structTypeInfo.getFieldTypeInfos()).containsExactlyInAnyOrder( + TypeInfoFactory.TIMESTAMP_NTZ, TypeInfoFactory.STRING, TypeInfoFactory.BIGINT + ); + } + + @Test + public void shouldReturnAppropriateTypeInfoForMetadataType() { + assertThat(MetadataUtil.getMetadataTypeInfo("integer")).isEqualTo(TypeInfoFactory.INT); + assertThat(MetadataUtil.getMetadataTypeInfo("long")).isEqualTo(TypeInfoFactory.BIGINT); + assertThat(MetadataUtil.getMetadataTypeInfo("float")).isEqualTo(TypeInfoFactory.FLOAT); + assertThat(MetadataUtil.getMetadataTypeInfo("double")).isEqualTo(TypeInfoFactory.DOUBLE); + assertThat(MetadataUtil.getMetadataTypeInfo("string")).isEqualTo(TypeInfoFactory.STRING); + assertThat(MetadataUtil.getMetadataTypeInfo("boolean")).isEqualTo(TypeInfoFactory.BOOLEAN); + assertThat(MetadataUtil.getMetadataTypeInfo("timestamp")).isEqualTo(TypeInfoFactory.TIMESTAMP_NTZ); + } + +} diff --git a/src/test/java/com/gotocompany/depot/maxcompute/util/SchemaDifferenceUtilsTest.java b/src/test/java/com/gotocompany/depot/maxcompute/util/SchemaDifferenceUtilsTest.java new file mode 100644 index 00000000..f37c0dd8 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/maxcompute/util/SchemaDifferenceUtilsTest.java @@ -0,0 +1,55 @@ +package com.gotocompany.depot.maxcompute.util; + +import com.aliyun.odps.Column; +import com.aliyun.odps.TableSchema; +import com.aliyun.odps.type.TypeInfoFactory; +import com.gotocompany.depot.maxcompute.schema.SchemaDifferenceUtils; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class SchemaDifferenceUtilsTest { + + @Test + public void testGetSchemaDifferenceDdl() { + TableSchema oldTableSchema = TableSchema.builder() + .withColumn(Column.newBuilder("metadata_1", TypeInfoFactory.STRING).build()) + .withColumn(Column.newBuilder("col1", TypeInfoFactory.STRING).build()) + .withColumn(Column.newBuilder("col2", TypeInfoFactory.getArrayTypeInfo(TypeInfoFactory.BOOLEAN)).build()) + .withColumn(Column.newBuilder("col3", TypeInfoFactory.getStructTypeInfo(Arrays.asList("f1", "f2"), + Arrays.asList(TypeInfoFactory.STRING, TypeInfoFactory.BIGINT))).build()) + .withColumn(Column.newBuilder("col4", TypeInfoFactory.getArrayTypeInfo(TypeInfoFactory.getStructTypeInfo(Arrays.asList("f41"), Arrays.asList(TypeInfoFactory.INT)))).build()) + .build(); + TableSchema newTableSchema = TableSchema.builder() + .withColumn(Column.newBuilder("metadata_1", TypeInfoFactory.STRING).build()) + .withColumn(Column.newBuilder("col1", TypeInfoFactory.STRING).build()) + .withColumn(Column.newBuilder("col2", TypeInfoFactory.getArrayTypeInfo(TypeInfoFactory.BOOLEAN)).build()) + .withColumn(Column.newBuilder("col3", TypeInfoFactory.getStructTypeInfo(Arrays.asList("f1", "f2", "f3"), + Arrays.asList(TypeInfoFactory.STRING, TypeInfoFactory.BIGINT, TypeInfoFactory.getArrayTypeInfo(TypeInfoFactory.STRING)))).build()) + .withColumn(Column.newBuilder("metadata_2", TypeInfoFactory.STRING).build()) + .withColumn(Column.newBuilder("col4", TypeInfoFactory.getArrayTypeInfo(TypeInfoFactory.getStructTypeInfo(Arrays.asList("f41", "f42"), Arrays.asList(TypeInfoFactory.INT, TypeInfoFactory.getStructTypeInfo(Arrays.asList("f421"), Arrays.asList(TypeInfoFactory.STRING)))))).build()) + .withColumn(Column.newBuilder("col5", TypeInfoFactory.getArrayTypeInfo(TypeInfoFactory.getStructTypeInfo(Arrays.asList("f51"), Arrays.asList(TypeInfoFactory.INT)))) + .build()) + .withColumn(Column.newBuilder("col6", TypeInfoFactory.getStructTypeInfo(Arrays.asList("f61"), Arrays.asList(TypeInfoFactory.STRING))) + .build()) + .build(); + Set expectedMetadataColumns = new HashSet<>(Arrays.asList( + "ALTER TABLE test_schema.test_table ADD COLUMN IF NOT EXISTS `col3`.`f3` ARRAY;", + "ALTER TABLE test_schema.test_table ADD COLUMN IF NOT EXISTS `col4`.element.`f42` STRUCT<`f421`:STRING>;", + "ALTER TABLE test_schema.test_table ADD COLUMN IF NOT EXISTS `col5` ARRAY>;", + "ALTER TABLE test_schema.test_table ADD COLUMN IF NOT EXISTS `col6` STRUCT<`f61`:STRING>;", + "ALTER TABLE test_schema.test_table ADD COLUMN IF NOT EXISTS `metadata_2` STRING;" + )); + + Set actualMetadataColumns = new HashSet<>(SchemaDifferenceUtils.getSchemaDifferenceSql(oldTableSchema, newTableSchema, "test_schema", "test_table")); + + assertEquals(actualMetadataColumns.size(), expectedMetadataColumns.size()); + assertTrue(actualMetadataColumns.containsAll(expectedMetadataColumns)); + } + +} diff --git a/src/test/java/com/gotocompany/depot/message/ProtoUnknownFieldValidationTypeTest.java b/src/test/java/com/gotocompany/depot/message/ProtoUnknownFieldValidationTypeTest.java new file mode 100644 index 00000000..c56a0161 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/message/ProtoUnknownFieldValidationTypeTest.java @@ -0,0 +1,133 @@ +package com.gotocompany.depot.message; + +import com.google.protobuf.Message; +import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.mockito.Mockito; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class ProtoUnknownFieldValidationTypeTest { + + @Test + public void shouldFilterMessageType() { + ProtoUnknownFieldValidationType type = ProtoUnknownFieldValidationType.MESSAGE; + Message message = Mockito.mock(Message.class); + + Assertions.assertTrue(type.shouldFilter(message)); + } + + @Test + public void shouldNotFilterNonMessageType() { + ProtoUnknownFieldValidationType type = ProtoUnknownFieldValidationType.MESSAGE; + Integer message = 2; + + Assertions.assertFalse(type.shouldFilter(message)); + } + + @Test + public void shouldReturnStreamOfMessage() { + ProtoUnknownFieldValidationType type = ProtoUnknownFieldValidationType.MESSAGE; + Message message = Mockito.mock(Message.class); + + Assertions.assertEquals(type.getMapper(message).collect(Collectors.toList()), + Stream.of(message).collect(Collectors.toList())); + } + + @Test + public void shouldFilterMessageOrMessageListType() { + ProtoUnknownFieldValidationType type = ProtoUnknownFieldValidationType.MESSAGE_ARRAY_FIRST_INDEX; + Message message = Mockito.mock(com.google.protobuf.Message.class); + List messageList = Collections.singletonList(message); + + Assertions.assertTrue(type.shouldFilter(message)); + Assertions.assertTrue(type.shouldFilter(messageList)); + } + + @Test + public void shouldFilterOutNonMessageType() { + ProtoUnknownFieldValidationType type = ProtoUnknownFieldValidationType.MESSAGE_ARRAY_FIRST_INDEX; + Integer message = 2; + + Assertions.assertFalse(type.shouldFilter(message)); + } + + @Test + public void shouldFilterOutEmptyMessageList() { + ProtoUnknownFieldValidationType type = ProtoUnknownFieldValidationType.MESSAGE_ARRAY_FIRST_INDEX; + List messageList = new ArrayList<>(); + + Assertions.assertFalse(type.shouldFilter(messageList)); + } + + @Test + public void shouldFilterOutNullObject() { + ProtoUnknownFieldValidationType type = ProtoUnknownFieldValidationType.MESSAGE_ARRAY_FIRST_INDEX; + + Assertions.assertFalse(type.shouldFilter(null)); + } + + @Test + public void shouldMapToSingularStreamOfMessage() { + ProtoUnknownFieldValidationType type = ProtoUnknownFieldValidationType.MESSAGE_ARRAY_FIRST_INDEX; + Message message = Mockito.mock(Message.class); + + List result = type.getMapper(message).collect(Collectors.toList()); + + Assertions.assertEquals(Stream.of(message).collect(Collectors.toList()), result); + } + + @Test + public void shouldMapToStreamOfMessagesOnlyFirstIndex() { + ProtoUnknownFieldValidationType type = ProtoUnknownFieldValidationType.MESSAGE_ARRAY_FIRST_INDEX; + Message message1 = Mockito.mock(Message.class); + Message message2 = Mockito.mock(Message.class); + List objects = new ArrayList<>(); + objects.add(message1); + objects.add(message2); + + List result = type.getMapper(objects).collect(Collectors.toList()); + + Assertions.assertEquals(Stream.of(message1).collect(Collectors.toList()), result); + } + + @Test + public void shouldMapToEmptyStreamWhenNonMessageListIsGiven() { + ProtoUnknownFieldValidationType type = ProtoUnknownFieldValidationType.MESSAGE_ARRAY_FIRST_INDEX; + Integer message = 2; + + List result = type.getMapper(message).collect(Collectors.toList()); + + Assertions.assertEquals(Collections.emptyList(), result); + } + + @Test + public void shouldFilterFullArrayMessage() { + ProtoUnknownFieldValidationType type = ProtoUnknownFieldValidationType.MESSAGE_ARRAY_FULL; + Message message1 = Mockito.mock(Message.class); + Message message2 = Mockito.mock(Message.class); + List objects = new ArrayList<>(); + objects.add(message1); + objects.add(message2); + + Assertions.assertTrue(type.shouldFilter(objects)); + } + + @Test + public void shouldMapFullArrayToMessage() { + ProtoUnknownFieldValidationType type = ProtoUnknownFieldValidationType.MESSAGE_ARRAY_FULL; + Message message1 = Mockito.mock(Message.class); + Message message2 = Mockito.mock(Message.class); + List objects = new ArrayList<>(); + objects.add(message1); + objects.add(message2); + + List result = type.getMapper(objects).collect(Collectors.toList()); + + Assertions.assertEquals(objects, result); + } +} diff --git a/src/test/java/com/gotocompany/depot/utils/ProtoUtilTest.java b/src/test/java/com/gotocompany/depot/utils/ProtoUtilTest.java index e5f2a838..5b4e4fdb 100644 --- a/src/test/java/com/gotocompany/depot/utils/ProtoUtilTest.java +++ b/src/test/java/com/gotocompany/depot/utils/ProtoUtilTest.java @@ -5,6 +5,7 @@ import com.google.protobuf.UnknownFieldSet; import com.gotocompany.depot.TestBookingLogMessage; import com.gotocompany.depot.TestLocation; +import com.gotocompany.depot.message.ProtoUnknownFieldValidationType; import org.junit.Test; import static org.junit.Assert.assertFalse; @@ -26,7 +27,7 @@ public void shouldReturnTrueWhenUnknownFieldsExistOnRootLevelFields() { .build()) .build(); - boolean unknownFieldExist = ProtoUtils.hasUnknownField(dynamicMessage); + boolean unknownFieldExist = ProtoUtils.hasUnknownField(dynamicMessage, ProtoUnknownFieldValidationType.MESSAGE); assertTrue(unknownFieldExist); } @@ -45,7 +46,7 @@ public void shouldReturnTrueWhenUnknownFieldsExistOnNestedChildFields() { .build()) .build(); - boolean unknownFieldExist = ProtoUtils.hasUnknownField(dynamicMessage); + boolean unknownFieldExist = ProtoUtils.hasUnknownField(dynamicMessage, ProtoUnknownFieldValidationType.MESSAGE); assertTrue(unknownFieldExist); } @@ -59,13 +60,13 @@ public void shouldReturnFalseWhenNoUnknownFieldsExist() { .setField(fieldDescriptor, DynamicMessage.newBuilder(location).build()) .build(); - boolean unknownFieldExist = ProtoUtils.hasUnknownField(dynamicMessage); + boolean unknownFieldExist = ProtoUtils.hasUnknownField(dynamicMessage, ProtoUnknownFieldValidationType.MESSAGE); assertFalse(unknownFieldExist); } @Test public void shouldReturnFalseWhenRootIsNull() { - boolean unknownFieldExist = ProtoUtils.hasUnknownField(null); + boolean unknownFieldExist = ProtoUtils.hasUnknownField(null, ProtoUnknownFieldValidationType.MESSAGE); assertFalse(unknownFieldExist); } } diff --git a/src/test/java/com/gotocompany/depot/utils/RetryUtilsTest.java b/src/test/java/com/gotocompany/depot/utils/RetryUtilsTest.java new file mode 100644 index 00000000..496b6a73 --- /dev/null +++ b/src/test/java/com/gotocompany/depot/utils/RetryUtilsTest.java @@ -0,0 +1,55 @@ +package com.gotocompany.depot.utils; + +import com.gotocompany.depot.exception.NonRetryableException; +import org.junit.Test; +import org.mockito.Mockito; + +public class RetryUtilsTest { + + @Test + public void shouldRetryUntilSuccess() throws Exception { + int repeatCountBeforeSuccess = 3; + RunnableMock runnableMock = Mockito.spy(new RunnableMock(repeatCountBeforeSuccess)); + + RetryUtils.executeWithRetry(runnableMock::execute, 5, 0, e -> e instanceof Exception); + + Mockito.verify(runnableMock, Mockito.times(repeatCountBeforeSuccess)).execute(); + } + + @Test(expected = NonRetryableException.class) + public void shouldThrowNonRetryableExceptionAfterRetryIsExhausted() throws Exception { + int repeatCountBeforeSuccess = 5; + RunnableMock runnableMock = Mockito.spy(new RunnableMock(repeatCountBeforeSuccess)); + + RetryUtils.executeWithRetry(runnableMock::execute, 3, 0, e -> e instanceof RuntimeException); + + Mockito.verify(runnableMock, Mockito.times(3)).execute(); + } + + @Test(expected = NonRetryableException.class) + public void shouldThrowNonRetryableExceptionWhenNonMatchingExceptionIsThrown() throws Exception { + RunnableMock runnableMock = Mockito.spy(new RunnableMock(3)); + + RetryUtils.executeWithRetry(runnableMock::execute, 3, 0, e -> e instanceof IllegalArgumentException); + + Mockito.verify(runnableMock, Mockito.times(1)).execute(); + } + + private static class RunnableMock { + private final int repeatCountBeforeSuccess; + private int repeatCount; + + RunnableMock(int repeatCountBeforeSuccess) { + this.repeatCountBeforeSuccess = repeatCountBeforeSuccess; + this.repeatCount = 0; + } + + void execute() throws Exception { + repeatCount++; + if (repeatCount < repeatCountBeforeSuccess) { + throw new Exception("Mock exception"); + } + } + } + +} diff --git a/src/test/proto/TestMaxComputePartition.proto b/src/test/proto/TestMaxComputePartition.proto new file mode 100644 index 00000000..805ce5cb --- /dev/null +++ b/src/test/proto/TestMaxComputePartition.proto @@ -0,0 +1,13 @@ +syntax = "proto3"; + +import "google/protobuf/timestamp.proto"; + +package com.gotocompany.depot; + +message MaxComputePartition { + string string_field = 1; + int32 int32_field = 2; + int64 int64_field = 3; + google.protobuf.Timestamp timestamp_field = 4; + float float_field = 5; +} \ No newline at end of file diff --git a/src/test/proto/TestMaxComputeRecord.proto b/src/test/proto/TestMaxComputeRecord.proto new file mode 100644 index 00000000..93e8a30f --- /dev/null +++ b/src/test/proto/TestMaxComputeRecord.proto @@ -0,0 +1,16 @@ +syntax = "proto3"; + +import "google/protobuf/timestamp.proto"; + +package com.gotocompany.depot; + +message MaxComputeRecord { + string id = 1; + repeated InnerRecord inner_record = 2; + google.protobuf.Timestamp timestamp = 3; +} + +message InnerRecord { + string name = 1; + float balance = 2; +} \ No newline at end of file diff --git a/src/test/proto/TestMaxComputeTypeInfo.proto b/src/test/proto/TestMaxComputeTypeInfo.proto new file mode 100644 index 00000000..00eab5b2 --- /dev/null +++ b/src/test/proto/TestMaxComputeTypeInfo.proto @@ -0,0 +1,104 @@ +syntax = "proto3"; + +package com.gotocompany.depot; + +import "google/protobuf/timestamp.proto"; +import "google/protobuf/struct.proto"; +import "google/protobuf/duration.proto"; +import "google/protobuf/empty.proto"; + +message TestFields { + bytes bytes_field = 1; + string string_field = 2; + TestEnum enum_field = 3; + double double_field = 4; + float float_field = 5; + bool bool_field = 6; + int64 int64_field = 7; + uint64 uint64_field = 8; + int32 int32_field = 9; + uint32 uint32_field = 10; + fixed64 fixed64_field = 11; + fixed32 fixed32_field = 12; + sfixed32 sfixed32_field = 13; + sfixed64 sfixed64_field = 14; + sint32 sint32_field = 15; + sint64 sint64_field = 16; +} + +message TestFieldsRepeated { + repeated bytes bytes_fields = 1; + repeated string string_fields = 2; + repeated TestEnum enum_fields = 3; + repeated double double_fields = 4; + repeated float float_fields = 5; + repeated bool bool_fields = 6; + repeated int64 int64_fields = 7; + repeated uint64 uint64_fields = 8; + repeated int32 int32_fields = 9; + repeated uint32 uint32_fields = 10; + repeated fixed64 fixed64_fields = 11; + repeated fixed32 fixed32_fields = 12; + repeated sfixed32 sfixed32_fields = 13; + repeated sfixed64 sfixed64_fields = 14; + repeated sint32 sint32_fields = 15; + repeated sint64 sint64_fields = 16; +} + +message TestRoot { + string string_field = 1; + TestInner inner_field = 2; + repeated TestInner inner_list_field = 3; + google.protobuf.Timestamp timestamp_field = 4; + google.protobuf.Struct struct_field = 5; + google.protobuf.Duration duration_field = 6; + google.protobuf.Empty empty_field = 7; +} + +message TestRootRepeated { + repeated string string_fields = 1; + repeated TestInner inner_fields = 2; + repeated TestInner inner_list_fields = 3; + repeated google.protobuf.Timestamp timestamp_fields = 4; + repeated google.protobuf.Struct struct_fields = 5; + repeated google.protobuf.Duration duration_fields = 6; + repeated google.protobuf.Empty empty_fields = 7; +} + +message TestInner { + string string_field = 1; + TestAnotherInner another_inner_field = 2; + repeated TestAnotherInner another_inner_list_field = 3; +} + +message TestAnotherInner { + string string_field = 1; +} + +message TestBuyerWrapper { + TestBuyer buyer = 1; +} + +message TestBuyer { + string name = 1; + TestCart cart = 2; + google.protobuf.Timestamp created_at = 3; +} + +message TestCart { + string cart_id = 1; + repeated TestItem items = 2; + google.protobuf.Timestamp created_at = 3; + google.protobuf.Duration cart_age = 4; +} + +message TestItem { + string id = 1; + int32 quantity = 2; +} +enum TestEnum { + TEST_1 = 0; + TEST_2 = 1; +} + + diff --git a/src/test/proto/TextMaxComputeTable.proto b/src/test/proto/TextMaxComputeTable.proto new file mode 100644 index 00000000..6db1cd72 --- /dev/null +++ b/src/test/proto/TextMaxComputeTable.proto @@ -0,0 +1,26 @@ +syntax = "proto3"; + +import "google/protobuf/timestamp.proto"; + +package com.gotocompany.depot; + +message Table { + string id = 1; + User user = 2; + repeated Item items = 3; + google.protobuf.Timestamp event_timestamp = 4; +} + +message User { + string id = 1; + repeated Contact contacts = 2; +} + +message Contact { + string number = 1; +} + +message Item { + string id = 1; + string name = 2; +} \ No newline at end of file