Skip to content

Commit

Permalink
[FLINK-26027][Connector/Pulsar] Expose Pulsar producer metrics and ad…
Browse files Browse the repository at this point in the history
…d FLIP-33 sink metrics. (apache#21249)
  • Loading branch information
syhily committed Nov 11, 2022
1 parent f8b3b33 commit 9c970b9
Show file tree
Hide file tree
Showing 24 changed files with 438 additions and 59 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,12 @@
<td><p>Enum</p></td>
<td>Optional delivery guarantee when committing.<br /><br />Possible values:<ul><li>"exactly-once": Records are only delivered exactly-once also under failover scenarios. To build a complete exactly-once pipeline is required that the source and sink support exactly-once and are properly configured.</li><li>"at-least-once": Records are ensured to be delivered but it may happen that the same record is delivered multiple times. Usually, this guarantee is faster than the exactly-once delivery.</li><li>"none": Records are delivered on a best effort basis. It is often the fastest way to process records but it may happen that records are lost or duplicated.</li></ul></td>
</tr>
<tr>
<td><h5>pulsar.sink.enableMetrics</h5></td>
<td style="word-wrap: break-word;">true</td>
<td>Boolean</td>
<td>The metrics from Pulsar Producer are only exposed if you enable this option.You should set the <code class="highlighter-rouge">pulsar.client.statsIntervalSeconds</code> to a positive value if you enable this option.</td>
</tr>
<tr>
<td><h5>pulsar.sink.enableSchemaEvolution</h5></td>
<td style="word-wrap: break-word;">false</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,12 @@
<td>Boolean</td>
<td>Flink commits the consuming position with pulsar transactions on checkpoint. However, if you have disabled the Flink checkpoint or disabled transaction for your Pulsar cluster, ensure that you have set this option to <code class="highlighter-rouge">true</code>.<br />The source would use pulsar client's internal mechanism and commit cursor in two ways.<ul><li>For <code class="highlighter-rouge">Key_Shared</code> and <code class="highlighter-rouge">Shared</code> subscription, the cursor would be committed once the message is consumed.</li><li>For <code class="highlighter-rouge">Exclusive</code> and <code class="highlighter-rouge">Failover</code> subscription, the cursor would be committed in a given interval.</li></ul></td>
</tr>
<tr>
<td><h5>pulsar.source.enableMetrics</h5></td>
<td style="word-wrap: break-word;">true</td>
<td>Boolean</td>
<td>The metrics from Pulsar Consumer are only exposed if you enable this option.You should set the <code class="highlighter-rouge">pulsar.client.statsIntervalSeconds</code> to a positive value if you enable this option.</td>
</tr>
<tr>
<td><h5>pulsar.source.maxFetchRecords</h5></td>
<td style="word-wrap: break-word;">100</td>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* 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:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.connector.pulsar.common.metrics;

/** The constant class for holding all the custom metrics names in Pulsar. */
public final class MetricNames {

private MetricNames() {
// No public constructor.
}

public static final String PULSAR_PRODUCER_METRIC_NAME = "PulsarProducer";
public static final String NUM_MSGS_SENT = "numMsgsSent";
public static final String NUM_BYTES_SENT = "numBytesSent";
public static final String NUM_SEND_FAILED = "numSendFailed";
public static final String NUM_ACKS_RECEIVED = "numAcksReceived";
public static final String SEND_MSGS_RATE = "sendMsgsRate";
public static final String SEND_BYTES_RATE = "sendBytesRate";
public static final String SEND_LATENCY_MILLIS_50_PCT = "sendLatencyMillis50pct";
public static final String SEND_LATENCY_MILLIS_75_PCT = "sendLatencyMillis75pct";
public static final String SEND_LATENCY_MILLIS_95_PCT = "sendLatencyMillis95pct";
public static final String SEND_LATENCY_MILLIS_99_PCT = "sendLatencyMillis99pct";
public static final String SEND_LATENCY_MILLIS_999_PCT = "sendLatencyMillis999pct";
public static final String SEND_LATENCY_MILLIS_MAX = "sendLatencyMillisMax";
public static final String TOTAL_MSGS_SENT = "totalMsgsSent";
public static final String TOTAL_BYTES_SENT = "totalBytesSent";
public static final String TOTAL_SEND_FAILED = "totalSendFailed";
public static final String TOTAL_ACKS_RECEIVED = "totalAcksReceived";
public static final String PENDING_QUEUE_SIZE = "pendingQueueSize";

public static final String PULSAR_CONSUMER_METRIC_NAME = "PulsarConsumer";
public static final String NUM_MSGS_RECEIVED = "numMsgsReceived";
public static final String NUM_BYTES_RECEIVED = "numBytesReceived";
public static final String RATE_MSGS_RECEIVED = "rateMsgsReceived";
public static final String RATE_BYTES_RECEIVED = "rateBytesReceived";
public static final String NUM_ACKS_SENT = "numAcksSent";
public static final String NUM_ACKS_FAILED = "numAcksFailed";
public static final String NUM_RECEIVE_FAILED = "numReceiveFailed";
public static final String NUM_BATCH_RECEIVE_FAILED = "numBatchReceiveFailed";
public static final String TOTAL_MSGS_RECEIVED = "totalMsgsReceived";
public static final String TOTAL_BYTES_RECEIVED = "totalBytesReceived";
public static final String TOTAL_RECEIVED_FAILED = "totalReceivedFailed";
public static final String TOTAL_BATCH_RECEIVED_FAILED = "totalBatchReceivedFailed";
public static final String TOTAL_ACKS_SENT = "totalAcksSent";
public static final String TOTAL_ACKS_FAILED = "totalAcksFailed";
public static final String MSG_NUM_IN_RECEIVER_QUEUE = "msgNumInReceiverQueue";
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* 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:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.connector.pulsar.common.metrics;

import org.apache.flink.metrics.Counter;
import org.apache.flink.metrics.groups.SinkWriterMetricGroup;

import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.interceptor.ProducerInterceptor;

/** The metric statistic for Pulsar's {@link Producer}. */
public class ProducerMetricsInterceptor implements ProducerInterceptor {

private final Counter numRecordsOutErrors;
private final Counter numRecordsOutCounter;
private final Counter numBytesOutCounter;

public ProducerMetricsInterceptor(SinkWriterMetricGroup metricGroup) {
this.numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter();
this.numRecordsOutCounter = metricGroup.getIOMetricGroup().getNumRecordsOutCounter();
this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter();
}

@Override
public void close() {
// Nothing to do by default.
}

@Override
public boolean eligible(Message message) {
return true;
}

@Override
public Message beforeSend(Producer producer, Message message) {
return message;
}

@Override
public void onSendAcknowledgement(
Producer producer, Message message, MessageId msgId, Throwable exception) {
if (exception != null) {
numRecordsOutErrors.inc(1);
} else {
numRecordsOutCounter.inc(1);
numBytesOutCounter.inc(message.size());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import static org.apache.flink.configuration.description.LinkElement.link;
import static org.apache.flink.configuration.description.TextElement.code;
import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_MEMORY_LIMIT_BYTES;
import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_STATS_INTERVAL_SECONDS;
import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PRODUCER_CONFIG_PREFIX;
import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.SINK_CONFIG_PREFIX;
import static org.apache.flink.connector.pulsar.sink.writer.router.MessageKeyHash.MURMUR3_32_HASH;
Expand Down Expand Up @@ -139,6 +140,19 @@ private PulsarSinkOptions() {
"The maximum number of pending messages in one sink parallelism.")
.build());

public static final ConfigOption<Boolean> PULSAR_ENABLE_SINK_METRICS =
ConfigOptions.key(SINK_CONFIG_PREFIX + "enableMetrics")
.booleanType()
.defaultValue(true)
.withDescription(
Description.builder()
.text(
"The metrics from Pulsar Producer are only exposed if you enable this option.")
.text(
"You should set the %s to a positive value if you enable this option.",
code(PULSAR_STATS_INTERVAL_SECONDS.key()))
.build());

///////////////////////////////////////////////////////////////////////////////
//
// The configuration for ProducerConfigurationData part.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,9 @@

import java.util.Objects;

import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_STATS_INTERVAL_SECONDS;
import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_MAX_MESSAGES;
import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_ENABLE_SINK_METRICS;
import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MAX_RECOMMIT_TIMES;
import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MESSAGE_KEY_HASH;
import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL;
Expand All @@ -51,6 +53,7 @@ public class SinkConfiguration extends PulsarConfiguration {
private final MessageKeyHash messageKeyHash;
private final boolean enableSchemaEvolution;
private final int maxRecommitTimes;
private final boolean enableMetrics;

public SinkConfiguration(Configuration configuration) {
super(configuration);
Expand All @@ -62,6 +65,8 @@ public SinkConfiguration(Configuration configuration) {
this.messageKeyHash = get(PULSAR_MESSAGE_KEY_HASH);
this.enableSchemaEvolution = get(PULSAR_WRITE_SCHEMA_EVOLUTION);
this.maxRecommitTimes = get(PULSAR_MAX_RECOMMIT_TIMES);
this.enableMetrics =
get(PULSAR_ENABLE_SINK_METRICS) && get(PULSAR_STATS_INTERVAL_SECONDS) > 0;
}

/** The delivery guarantee changes the behavior of {@link PulsarWriter}. */
Expand Down Expand Up @@ -113,6 +118,11 @@ public int getMaxRecommitTimes() {
return maxRecommitTimes;
}

/** Whether to expose the metrics from Pulsar Producer. */
public boolean isEnableMetrics() {
return enableMetrics;
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand All @@ -130,7 +140,8 @@ public boolean equals(Object o) {
&& partitionSwitchSize == that.partitionSwitchSize
&& enableSchemaEvolution == that.enableSchemaEvolution
&& messageKeyHash == that.messageKeyHash
&& maxRecommitTimes == that.maxRecommitTimes;
&& maxRecommitTimes == that.maxRecommitTimes
&& enableMetrics == that.enableMetrics;
}

@Override
Expand All @@ -142,6 +153,7 @@ public int hashCode() {
partitionSwitchSize,
messageKeyHash,
enableSchemaEvolution,
maxRecommitTimes);
maxRecommitTimes,
enableMetrics);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ public class PulsarWriter<IN> implements PrecommittingSinkWriter<IN, PulsarCommi
private final PulsarSinkContext sinkContext;
private final TopicProducerRegister producerRegister;
private final MailboxExecutor mailboxExecutor;
private final AtomicLong pendingMessages = new AtomicLong(0);
private final AtomicLong pendingMessages;

/**
* Constructor creating a Pulsar writer.
Expand Down Expand Up @@ -122,8 +122,10 @@ public PulsarWriter(
}

// Create this producer register after opening serialization schema!
this.producerRegister = new TopicProducerRegister(sinkConfiguration);
this.producerRegister =
new TopicProducerRegister(sinkConfiguration, initContext.metricGroup());
this.mailboxExecutor = initContext.getMailboxExecutor();
this.pendingMessages = new AtomicLong(0);
}

@Override
Expand Down
Loading

0 comments on commit 9c970b9

Please sign in to comment.