[FLINK-19130] [core] Extend metric interfaces for backpressure metrics

This commit extends the FunctionTypeMetrics interface and introduces a
new FunctionDispatcherMetrics interface with a goal to expose the
following backpressure-related metrics:

- Number of blocked addresses (per function type)
- Number of inflight async ops (per function type + per-operator)
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FlinkFunctionDispatcherMetrics.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FlinkFunctionDispatcherMetrics.java
new file mode 100644
index 0000000..c060a3d
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FlinkFunctionDispatcherMetrics.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.statefun.flink.core.metrics;
+
+import java.util.Objects;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.MetricGroup;
+
+public class FlinkFunctionDispatcherMetrics implements FunctionDispatcherMetrics {
+  private final Counter inflightAsyncOperations;
+
+  public FlinkFunctionDispatcherMetrics(MetricGroup operatorGroup) {
+    Objects.requireNonNull(operatorGroup, "operatorGroup");
+
+    this.inflightAsyncOperations = operatorGroup.counter("inflight-async-ops");
+  }
+
+  @Override
+  public void asyncOperationRegistered() {
+    inflightAsyncOperations.inc();
+  }
+
+  @Override
+  public void asyncOperationCompleted() {
+    inflightAsyncOperations.dec();
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FlinkFunctionTypeMetrics.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FlinkFunctionTypeMetrics.java
index d9eec0c..553cd68 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FlinkFunctionTypeMetrics.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FlinkFunctionTypeMetrics.java
@@ -27,12 +27,16 @@
   private final Counter outgoingLocalMessage;
   private final Counter outgoingRemoteMessage;
   private final Counter outgoingEgress;
+  private final Counter blockedAddress;
+  private final Counter inflightAsyncOps;
 
   FlinkFunctionTypeMetrics(MetricGroup typeGroup) {
     this.incoming = metered(typeGroup, "in");
     this.outgoingLocalMessage = metered(typeGroup, "out-local");
     this.outgoingRemoteMessage = metered(typeGroup, "out-remote");
     this.outgoingEgress = metered(typeGroup, "out-egress");
+    this.blockedAddress = typeGroup.counter("num-blocked-address");
+    this.inflightAsyncOps = typeGroup.counter("inflight-async-ops");
   }
 
   @Override
@@ -55,6 +59,26 @@
     this.outgoingEgress.inc();
   }
 
+  @Override
+  public void blockedAddress() {
+    this.blockedAddress.inc();
+  }
+
+  @Override
+  public void unblockedAddress() {
+    this.blockedAddress.dec();
+  }
+
+  @Override
+  public void asyncOperationRegistered() {
+    this.inflightAsyncOps.inc();
+  }
+
+  @Override
+  public void asyncOperationCompleted() {
+    this.inflightAsyncOps.dec();
+  }
+
   private static SimpleCounter metered(MetricGroup metrics, String name) {
     SimpleCounter counter = metrics.counter(name, new SimpleCounter());
     metrics.meter(name + "Rate", new MeterView(counter, 60));
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FunctionDispatcherMetrics.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FunctionDispatcherMetrics.java
new file mode 100644
index 0000000..f8d6dc2
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FunctionDispatcherMetrics.java
@@ -0,0 +1,26 @@
+/*
+ * 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.statefun.flink.core.metrics;
+
+public interface FunctionDispatcherMetrics {
+
+  void asyncOperationRegistered();
+
+  void asyncOperationCompleted();
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FunctionTypeMetrics.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FunctionTypeMetrics.java
index 54a84e3..acfc146 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FunctionTypeMetrics.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/metrics/FunctionTypeMetrics.java
@@ -26,4 +26,12 @@
   void outgoingRemoteMessage();
 
   void outgoingEgressMessage();
+
+  void blockedAddress();
+
+  void unblockedAddress();
+
+  void asyncOperationRegistered();
+
+  void asyncOperationCompleted();
 }