foxus commented on code in PR #22: URL: https://github.com/apache/flink-connector-prometheus/pull/22#discussion_r2044253548
########## flink-connector-prometheus/src/main/java/org/apache/flink/connector/prometheus/table/PrometheusConnectorOption.java: ########## @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.prometheus.table; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +public class PrometheusConnectorOption { + + // ----------------------------------------------------------------------------------------- + // Prometheus connector specific options + // ----------------------------------------------------------------------------------------- + + public static final ConfigOption<String> METRIC_NAME = + ConfigOptions.key("metric.name") + .stringType() + .noDefaultValue() + .withDescription("Prometheus metric name."); + + public static final ConfigOption<String> METRIC_LABEL_KEYS = Review Comment: Related to previous comment, `ConfigOption` has native support for CSVs - ``` ConfigOption<String> METRIC_LABEL_KEYS = ConfigOptions .key("metric.label.keys") .stringType() .asList() .noDefaultValue() .withDescription("Prometheus metric label key name."); ``` You could then eliminate the streaming and hardcoded delimiter work above. ########## flink-connector-prometheus/src/main/java/org/apache/flink/connector/prometheus/table/PrometheusDynamicSink.java: ########## @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.prometheus.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSink; +import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSinkBuilder; +import org.apache.flink.connector.prometheus.sink.PrometheusRequestSigner; +import org.apache.flink.connector.prometheus.sink.PrometheusSink; +import org.apache.flink.connector.prometheus.sink.PrometheusSinkBuilder; +import org.apache.flink.connector.prometheus.sink.PrometheusSinkConfiguration; +import org.apache.flink.connector.prometheus.sink.prometheus.Types; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.Iterator; +import java.util.Map; +import java.util.Optional; +import java.util.ServiceConfigurationError; +import java.util.ServiceLoader; + +public class PrometheusDynamicSink extends AsyncDynamicTableSink<Types.TimeSeries> + implements SupportsPartitioning { + private static final Logger LOG = LoggerFactory.getLogger(PrometheusDynamicSink.class); + + /** Consumed data type of the table. */ + private final DataType physicalDataType; + + private final PrometheusConfig prometheusConfig; + + protected PrometheusDynamicSink( + @Nullable Integer maxBatchSize, + @Nullable Integer maxInFlightRequests, + @Nullable Integer maxBufferedRequests, + @Nullable Long maxBufferSizeInBytes, + @Nullable Long maxTimeInBufferMS, + @Nullable DataType physicalDataType, + PrometheusConfig prometheusConfig) { + + super( + maxBatchSize, // maxBatchSizeInSamples + maxInFlightRequests, + maxBufferedRequests, + maxBufferSizeInBytes, // maxRecordSizeInSamples + maxTimeInBufferMS); + + this.physicalDataType = + Preconditions.checkNotNull(physicalDataType, "Consumed data type must not be null"); + this.prometheusConfig = prometheusConfig; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode changelogMode) { + return ChangelogMode.upsert(); + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + + PrometheusSinkBuilder<RowData> builder = + new PrometheusSinkBuilder<RowData>() + .setRetryConfiguration( + PrometheusSinkConfiguration.RetryConfiguration + .DEFAULT_RETRY_CONFIGURATION) + .setElementConverter( + new RowDataElementConverter(physicalDataType, prometheusConfig)) + .setPrometheusRemoteWriteUrl(prometheusConfig.getRemoteWriteEndpointUrl()); + + Optional.ofNullable(prometheusConfig.getRequestSignerIdentifier()) + .ifPresent(region -> builder.setRequestSigner(getRequestSigner(prometheusConfig))); + Optional.ofNullable(maxBatchSize).ifPresent(builder::setMaxBatchSizeInSamples); + Optional.ofNullable(maxBufferSizeInBytes).ifPresent(builder::setMaxRecordSizeInSamples); + + PrometheusSink<RowData> prometheusSink = builder.build(); + return SinkV2Provider.of(prometheusSink); + } + + @Override + public DynamicTableSink copy() { + return new PrometheusDynamicSink( + maxBatchSize, + maxInFlightRequests, + maxBufferedRequests, + maxBufferSizeInBytes, + maxTimeInBufferMS, + physicalDataType, + prometheusConfig); + } + + @Override + public String asSummaryString() { + return "Prometheus"; + } + + @Override + public void applyStaticPartition(Map<String, String> partitions) { + // We don't need to do anything here because the Prometheus sink handles a static partition + // just like a normal partition. + } + + private PrometheusRequestSigner getRequestSigner(PrometheusConfig prometheusConfig) { + ServiceLoader<PrometheusDynamicRequestSignerFactory> loader = + ServiceLoader.load(PrometheusDynamicRequestSignerFactory.class); + Iterator<PrometheusDynamicRequestSignerFactory> factories = loader.iterator(); + String requestSignerIdentifier = prometheusConfig.getRequestSignerIdentifier(); + + while (true) { + try { + if (!factories.hasNext()) { + break; + } + + PrometheusDynamicRequestSignerFactory factory = factories.next(); Review Comment: Would this be simpler as a for loop as well as incorporating line 129? ``` for(Iterator<PrometheusDynamicRequestSignerFactory> factories = loader.iterator(); factories.hasNext(); ) { PrometheusDynamicRequestSignerFactory factory = factories.next(); ``` ########## flink-connector-prometheus/src/main/java/org/apache/flink/connector/prometheus/table/RowDataToPrometheusTimeSeriesConverter.java: ########## @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.prometheus.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.prometheus.sink.PrometheusTimeSeries; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; + +import java.util.List; + +import static org.apache.flink.table.data.RowData.createFieldGetter; + +/** + * Converts from Flink Table API internal type of {@link RowData} to {@link PrometheusTimeSeries}. + */ +@Internal +public class RowDataToPrometheusTimeSeriesConverter { + + private final DataType physicalDataType; + private final PrometheusConfig prometheusConfig; + + public RowDataToPrometheusTimeSeriesConverter( + DataType physicalDataType, PrometheusConfig prometheusConfig) { + this.physicalDataType = physicalDataType; + this.prometheusConfig = prometheusConfig; + } + + public PrometheusTimeSeries convertRowData(RowData row) { + List<DataTypes.Field> fields = DataType.getFields(physicalDataType); + + PrometheusTimeSeries.Builder builder = PrometheusTimeSeries.builder(); + Double sampleValue = null; + Long sampleTimestamp = null; + + for (int i = 0; i < fields.size(); i++) { + DataTypes.Field field = fields.get(i); + RowData.FieldGetter fieldGetter = + createFieldGetter(fields.get(i).getDataType().getLogicalType(), i); + FieldValue fieldValue = new FieldValue(fieldGetter.getFieldOrNull(row)); + String fieldName = field.getName(); + + if (fieldName.equals(prometheusConfig.getMetricName())) { + builder.withMetricName(fieldValue.getStringValue()); + } else if (fieldName.equals(prometheusConfig.getMetricSampleKey())) { + sampleValue = fieldValue.getDoubleValue(); + } else if (prometheusConfig.getLabelKeys().contains(fieldName)) { + builder.addLabel(fieldName, fieldValue.getStringValue()); + } else if (fieldName.equals(prometheusConfig.getMetricSampleTimestamp())) { + sampleTimestamp = fieldValue.getLongValue(); + } else { + throw new IllegalArgumentException("Unsupported field: " + fieldName); Review Comment: Could we consider providing the list of supported fields in this exception message? ########## flink-connector-prometheus/src/main/java/org/apache/flink/connector/prometheus/table/PrometheusConfig.java: ########## @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.prometheus.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.AWS_REGION; +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.METRIC_LABEL_KEYS; +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.METRIC_NAME; +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.METRIC_REMOTE_WRITE_URL; +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.METRIC_REQUEST_SIGNER; +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.METRIC_SAMPLE_KEY; +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.METRIC_SAMPLE_TIMESTAMP; + +/** Prometheus specific configuration. */ +@Internal +public class PrometheusConfig implements Serializable { + private final ReadableConfig options; + + private static final String DELIMITER = ","; + + public PrometheusConfig(ReadableConfig options) { + this.options = options; + } + + public String getMetricName() { + return options.get(METRIC_NAME); + } + + public List<String> getLabelKeys() { Review Comment: Are `labelKeys` unique? If so, consider returning a set rather than a list to enforce uniqueness, alternatively consider calling unique after the split. This will protect against a user supplied configuration including duplicates and also ensure that a serialized Set is identical regardless of the order of keys provided. ########## flink-connector-prometheus/src/main/java/org/apache/flink/connector/prometheus/table/PrometheusConnectorOption.java: ########## @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.prometheus.table; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +public class PrometheusConnectorOption { + + // ----------------------------------------------------------------------------------------- + // Prometheus connector specific options + // ----------------------------------------------------------------------------------------- + + public static final ConfigOption<String> METRIC_NAME = + ConfigOptions.key("metric.name") + .stringType() + .noDefaultValue() + .withDescription("Prometheus metric name."); + + public static final ConfigOption<String> METRIC_LABEL_KEYS = Review Comment: I would also consider rewording the description to make it clear there can be more than one key name provided. ########## flink-connector-prometheus/src/main/java/org/apache/flink/connector/prometheus/table/PrometheusDynamicSinkFactory.java: ########## @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.prometheus.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.table.AsyncDynamicTableSinkFactory; +import org.apache.flink.connector.base.table.AsyncSinkConnectorOptions; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.types.DataType; + +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.AWS_REGION; +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.METRIC_LABEL_KEYS; +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.METRIC_NAME; +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.METRIC_REMOTE_WRITE_URL; +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.METRIC_REQUEST_SIGNER; +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.METRIC_SAMPLE_KEY; +import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.METRIC_SAMPLE_TIMESTAMP; + +/** Factory for creating {@link PrometheusDynamicSink}. */ +@Internal +public class PrometheusDynamicSinkFactory extends AsyncDynamicTableSinkFactory { + public static final String FACTORY_IDENTIFIER = "prometheus"; + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + FactoryUtil.TableFactoryHelper factoryHelper = + FactoryUtil.createTableFactoryHelper(this, context); + ResolvedCatalogTable catalogTable = context.getCatalogTable(); + + FactoryUtil.validateFactoryOptions(this, factoryHelper.getOptions()); + + DataType physicalDataType = catalogTable.getResolvedSchema().toPhysicalRowDataType(); + PrometheusConfig prometheusConfig = new PrometheusConfig(factoryHelper.getOptions()); + validateMetricConfigKeys(prometheusConfig, physicalDataType); + + PrometheusDynamicSink.PrometheusDynamicSinkBuilder builder = + PrometheusDynamicSink.builder() + .setPhysicalDataType(physicalDataType) + .setPrometheusConfiguration(prometheusConfig); + + addAsyncOptionsToBuilder(getAsyncSinkOptions(factoryHelper.getOptions()), builder); + + return builder.build(); + } + + @Override + public String factoryIdentifier() { + return FACTORY_IDENTIFIER; + } + + @Override + public Set<ConfigOption<?>> requiredOptions() { + final Set<ConfigOption<?>> requiredOptions = new HashSet<>(); + + requiredOptions.add(METRIC_NAME); + requiredOptions.add(METRIC_LABEL_KEYS); + requiredOptions.add(METRIC_SAMPLE_KEY); + requiredOptions.add(METRIC_SAMPLE_TIMESTAMP); + requiredOptions.add(METRIC_REMOTE_WRITE_URL); + + return requiredOptions; + } + + @Override + public Set<ConfigOption<?>> optionalOptions() { + final Set<ConfigOption<?>> optionalOptions = new HashSet<>(); + + optionalOptions.add(AWS_REGION); + optionalOptions.add(METRIC_REQUEST_SIGNER); + + return optionalOptions; + } + + private Properties getAsyncSinkOptions(ReadableConfig config) { + Properties properties = new Properties(); + Optional.ofNullable(config.get(AsyncSinkConnectorOptions.FLUSH_BUFFER_SIZE)) + .ifPresent( + flushBufferSize -> + properties.put( + AsyncSinkConnectorOptions.FLUSH_BUFFER_SIZE.key(), + flushBufferSize)); + Optional.ofNullable(config.get(AsyncSinkConnectorOptions.MAX_BATCH_SIZE)) + .ifPresent( + maxBatchSize -> + properties.put( + AsyncSinkConnectorOptions.MAX_BATCH_SIZE.key(), + maxBatchSize)); + Optional.ofNullable(config.get(AsyncSinkConnectorOptions.MAX_IN_FLIGHT_REQUESTS)) + .ifPresent( + maxInflightRequests -> + properties.put( + AsyncSinkConnectorOptions.MAX_IN_FLIGHT_REQUESTS.key(), + maxInflightRequests)); + Optional.ofNullable(config.get(AsyncSinkConnectorOptions.MAX_BUFFERED_REQUESTS)) + .ifPresent( + maxBufferedRequests -> + properties.put( + AsyncSinkConnectorOptions.MAX_BUFFERED_REQUESTS.key(), + maxBufferedRequests)); + Optional.ofNullable(config.get(AsyncSinkConnectorOptions.FLUSH_BUFFER_TIMEOUT)) + .ifPresent( + timeout -> + properties.put( + AsyncSinkConnectorOptions.FLUSH_BUFFER_TIMEOUT.key(), + timeout)); + return properties; + } + + private void validateMetricConfigKeys( + PrometheusConfig prometheusConfig, DataType physicalDataType) { + Set<String> fieldNames = + DataType.getFields(physicalDataType).stream() + .map(DataTypes.AbstractField::getName) + .collect(Collectors.toSet()); + + List<String> metricConfigKeys = + Stream.concat( + Stream.of( + prometheusConfig.getMetricName(), + prometheusConfig.getMetricSampleKey(), + prometheusConfig.getMetricSampleTimestamp()), + prometheusConfig.getLabelKeys().stream()) + .collect(Collectors.toList()); + + metricConfigKeys.forEach( + metricConfigKey -> + Optional.ofNullable(metricConfigKey) + .filter(fieldNames::contains) + .orElseThrow( + () -> + new ValidationException( + String.format( + "%s not found in fields: %s", Review Comment: Could we consider something along the lines of `%s is not a valid Prometheus metric config key, valid keys are: %s`, this message is exposing the implementation of the validation rather than what the user should do to fix it. ########## flink-connector-prometheus/src/main/java/org/apache/flink/connector/prometheus/table/PrometheusDynamicSink.java: ########## @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.prometheus.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSink; +import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSinkBuilder; +import org.apache.flink.connector.prometheus.sink.PrometheusRequestSigner; +import org.apache.flink.connector.prometheus.sink.PrometheusSink; +import org.apache.flink.connector.prometheus.sink.PrometheusSinkBuilder; +import org.apache.flink.connector.prometheus.sink.PrometheusSinkConfiguration; +import org.apache.flink.connector.prometheus.sink.prometheus.Types; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.Iterator; +import java.util.Map; +import java.util.Optional; +import java.util.ServiceConfigurationError; +import java.util.ServiceLoader; + +public class PrometheusDynamicSink extends AsyncDynamicTableSink<Types.TimeSeries> + implements SupportsPartitioning { + private static final Logger LOG = LoggerFactory.getLogger(PrometheusDynamicSink.class); + + /** Consumed data type of the table. */ + private final DataType physicalDataType; + + private final PrometheusConfig prometheusConfig; + + protected PrometheusDynamicSink( + @Nullable Integer maxBatchSize, + @Nullable Integer maxInFlightRequests, + @Nullable Integer maxBufferedRequests, + @Nullable Long maxBufferSizeInBytes, + @Nullable Long maxTimeInBufferMS, + @Nullable DataType physicalDataType, + PrometheusConfig prometheusConfig) { + + super( + maxBatchSize, // maxBatchSizeInSamples + maxInFlightRequests, + maxBufferedRequests, + maxBufferSizeInBytes, // maxRecordSizeInSamples + maxTimeInBufferMS); + + this.physicalDataType = + Preconditions.checkNotNull(physicalDataType, "Consumed data type must not be null"); + this.prometheusConfig = prometheusConfig; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode changelogMode) { + return ChangelogMode.upsert(); + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + + PrometheusSinkBuilder<RowData> builder = + new PrometheusSinkBuilder<RowData>() + .setRetryConfiguration( + PrometheusSinkConfiguration.RetryConfiguration + .DEFAULT_RETRY_CONFIGURATION) + .setElementConverter( + new RowDataElementConverter(physicalDataType, prometheusConfig)) + .setPrometheusRemoteWriteUrl(prometheusConfig.getRemoteWriteEndpointUrl()); + + Optional.ofNullable(prometheusConfig.getRequestSignerIdentifier()) + .ifPresent(region -> builder.setRequestSigner(getRequestSigner(prometheusConfig))); Review Comment: I'm uncertain why `region` is being created within the context of a request signer identifier only not to be used in the subsequent call. Would this be simpler to leave what's required from the `prometheusConfig` to the `getRequestSigner` method which will return null if there isn't one and only call `builder.setRequestSigner` if it isn't? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org