[FLINK-23600] Rework remote module component parsing and binding

This closes #247.
diff --git a/statefun-e2e-tests/statefun-exactly-once-remote-e2e/src/test/resources/remote-module/module.yaml b/statefun-e2e-tests/statefun-exactly-once-remote-e2e/src/test/resources/remote-module/module.yaml
index de67146..eb00af7 100644
--- a/statefun-e2e-tests/statefun-exactly-once-remote-e2e/src/test/resources/remote-module/module.yaml
+++ b/statefun-e2e-tests/statefun-exactly-once-remote-e2e/src/test/resources/remote-module/module.yaml
@@ -13,42 +13,29 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-version: "3.0"
-
-module:
-  meta:
-    type: remote
-  spec:
-    endpoints:
-      - endpoint:
-          meta:
-            kind: http
-          spec:
-            functions: org.apache.flink.statefun.e2e.remote/*
-            urlPathTemplate: http://remote-function:8000/service
-            maxNumBatchRequests: 10000
-    ingresses:
-      - ingress:
-          meta:
-            type: io.statefun.kafka/ingress
-            id: org.apache.flink.statefun.e2e.remote/invoke
-          spec:
-            address: kafka-broker:9092
-            consumerGroupId: remote-module-e2e
-            startupPosition:
-              type: earliest
-            topics:
-              - topic: invoke
-                valueType: statefun.e2e/org.apache.flink.statefun.e2e.remote.Invoke
-                targets:
-                  - org.apache.flink.statefun.e2e.remote/counter
-    egresses:
-      - egress:
-          meta:
-            type: io.statefun.kafka/egress
-            id: org.apache.flink.statefun.e2e.remote/invoke-results
-          spec:
-            address: kafka-broker:9092
-            deliverySemantic:
-              type: exactly-once
-              transactionTimeout: 15min
+kind: io.statefun.endpoints.v2/http
+spec:
+  functions: org.apache.flink.statefun.e2e.remote/*
+  urlPathTemplate: http://remote-function:8000/service
+  maxNumBatchRequests: 10000
+---
+kind: io.statefun.kafka.v1/ingress
+spec:
+  id: org.apache.flink.statefun.e2e.remote/invoke
+  address: kafka-broker:9092
+  consumerGroupId: remote-module-e2e
+  startupPosition:
+    type: earliest
+  topics:
+    - topic: invoke
+      valueType: statefun.e2e/org.apache.flink.statefun.e2e.remote.Invoke
+      targets:
+        - org.apache.flink.statefun.e2e.remote/counter
+---
+kind: io.statefun.kafka.v1/egress
+spec:
+  id: org.apache.flink.statefun.e2e.remote/invoke-results
+  address: kafka-broker:9092
+  deliverySemantic:
+    type: exactly-once
+    transactionTimeout: 15min
diff --git a/statefun-e2e-tests/statefun-smoke-e2e-java/src/test/resources/remote-module/module.yaml b/statefun-e2e-tests/statefun-smoke-e2e-java/src/test/resources/remote-module/module.yaml
index d44e2ec..1f9874d 100644
--- a/statefun-e2e-tests/statefun-smoke-e2e-java/src/test/resources/remote-module/module.yaml
+++ b/statefun-e2e-tests/statefun-smoke-e2e-java/src/test/resources/remote-module/module.yaml
@@ -13,17 +13,8 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-version: "3.0"
-
-module:
-  meta:
-    type: remote
-  spec:
-    endpoints:
-      - endpoint:
-          meta:
-            kind: http
-          spec:
-            functions: statefun.smoke.e2e/*
-            urlPathTemplate: http://remote-function-host:8000
-            maxNumBatchRequests: 10000
+kind: io.statefun.endpoints.v2/http
+spec:
+  functions: statefun.smoke.e2e/command-interpreter-fn
+  urlPathTemplate: http://remote-function-host:8000
+  maxNumBatchRequests: 10000
diff --git a/statefun-e2e-tests/statefun-smoke-e2e-multilang-harness/src/test/resources/module.yaml b/statefun-e2e-tests/statefun-smoke-e2e-multilang-harness/src/test/resources/module.yaml
index 394ff0b..e98561f 100644
--- a/statefun-e2e-tests/statefun-smoke-e2e-multilang-harness/src/test/resources/module.yaml
+++ b/statefun-e2e-tests/statefun-smoke-e2e-multilang-harness/src/test/resources/module.yaml
@@ -13,17 +13,8 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-version: "3.0"
-
-module:
-  meta:
-    type: remote
-  spec:
-    endpoints:
-      - endpoint:
-          meta:
-            kind: http
-          spec:
-            functions: statefun.smoke.e2e/command-interpreter-fn
-            urlPathTemplate: http://localhost:8000
-            maxNumBatchRequests: 10000
+kind: io.statefun.endpoints.v2/http
+spec:
+  functions: statefun.smoke.e2e/command-interpreter-fn
+  urlPathTemplate: http://localhost:8000
+  maxNumBatchRequests: 10000
diff --git a/statefun-flink/pom.xml b/statefun-flink/pom.xml
index cc34020..5f56936 100644
--- a/statefun-flink/pom.xml
+++ b/statefun-flink/pom.xml
@@ -32,6 +32,7 @@
 
     <modules>
         <module>statefun-flink-common</module>
+        <module>statefun-flink-extensions</module>
         <module>statefun-flink-launcher</module>
         <module>statefun-flink-io</module>
         <module>statefun-flink-io-bundle</module>
diff --git a/statefun-flink/statefun-flink-common/pom.xml b/statefun-flink/statefun-flink-common/pom.xml
index fedbddb..db0f7af 100644
--- a/statefun-flink/statefun-flink-common/pom.xml
+++ b/statefun-flink/statefun-flink-common/pom.xml
@@ -34,6 +34,12 @@
     </properties>
 
     <dependencies>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>statefun-sdk-embedded</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
         <!-- flink runtime -->
         <dependency>
             <groupId>org.apache.flink</groupId>
diff --git a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/json/StateFunObjectMapper.java b/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/json/StateFunObjectMapper.java
new file mode 100644
index 0000000..2d3e502
--- /dev/null
+++ b/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/json/StateFunObjectMapper.java
@@ -0,0 +1,61 @@
+/*
+ * 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.common.json;
+
+import java.io.IOException;
+import java.time.Duration;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationFeature;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonDeserializer;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.module.SimpleModule;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.util.TimeUtils;
+
+public final class StateFunObjectMapper {
+
+  public static ObjectMapper create() {
+    final ObjectMapper mapper =
+        new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+    final SimpleModule module = new SimpleModule("statefun");
+    module.addDeserializer(Duration.class, new DurationJsonDeserializer());
+    module.addDeserializer(TypeName.class, new TypeNameJsonDeserializer());
+
+    mapper.registerModule(module);
+    return mapper;
+  }
+
+  private static final class DurationJsonDeserializer extends JsonDeserializer<Duration> {
+    @Override
+    public Duration deserialize(
+        JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+      return TimeUtils.parseDuration(jsonParser.getText());
+    }
+  }
+
+  private static final class TypeNameJsonDeserializer extends JsonDeserializer<TypeName> {
+    @Override
+    public TypeName deserialize(
+        JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+      return TypeName.parseFrom(jsonParser.getText());
+    }
+  }
+}
diff --git a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protobuf/FileDescriptorResolver.java b/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protobuf/FileDescriptorResolver.java
deleted file mode 100644
index 71d24ea..0000000
--- a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protobuf/FileDescriptorResolver.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * 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.common.protobuf;
-
-import static java.util.stream.Collectors.toMap;
-
-import com.google.protobuf.DescriptorProtos;
-import com.google.protobuf.DescriptorProtos.FileDescriptorProto;
-import com.google.protobuf.Descriptors;
-import java.util.Map;
-import java.util.function.Function;
-
-/**
- * Extract and resolve all the {@link Descriptors.FileDescriptor} embedded in a {@link
- * DescriptorProtos.FileDescriptorSet}.
- */
-final class FileDescriptorResolver {
-
-  static Map<String, Descriptors.FileDescriptor> resolve(
-      DescriptorProtos.FileDescriptorSet fileDescriptorSet) {
-    FileDescriptorResolver resolver = new FileDescriptorResolver(fileDescriptorSet);
-    return resolver.resolve();
-  }
-
-  private final Map<String, FileDescriptorProtoWrapper> descriptorByName;
-
-  private FileDescriptorResolver(DescriptorProtos.FileDescriptorSet descriptorSet) {
-    // dependencies are specified in the form of a proto filename, therefore we need to index the
-    // FileDescriptorProto's in that file by their name.
-    this.descriptorByName =
-        descriptorSet.getFileList().stream()
-            .map(FileDescriptorProtoWrapper::new)
-            .collect(toMap(FileDescriptorProtoWrapper::name, Function.identity()));
-  }
-
-  /** Resolve each {@code FileDescriptorProto} by name. */
-  private Map<String, Descriptors.FileDescriptor> resolve() {
-    return descriptorByName.entrySet().stream()
-        .collect(toMap(Map.Entry::getKey, e -> e.getValue().resolve()));
-  }
-
-  private final class FileDescriptorProtoWrapper {
-    private final DescriptorProtos.FileDescriptorProto unresolved;
-    private Descriptors.FileDescriptor resolved;
-
-    private FileDescriptorProtoWrapper(FileDescriptorProto unresolved) {
-      this.unresolved = unresolved;
-    }
-
-    public String name() {
-      return unresolved.getName();
-    }
-
-    /** Resolve a given {@code FileDescriptorProto} */
-    private Descriptors.FileDescriptor resolve() {
-      if (resolved != null) {
-        return resolved;
-      }
-      Descriptors.FileDescriptor[] dependencies =
-          unresolved.getDependencyList().stream()
-              .map(descriptorByName::get)
-              .map(FileDescriptorProtoWrapper::resolve)
-              .toArray(Descriptors.FileDescriptor[]::new);
-      try {
-        Descriptors.FileDescriptor resolved =
-            Descriptors.FileDescriptor.buildFrom(unresolved, dependencies, false);
-        this.resolved = resolved;
-        return resolved;
-      } catch (Descriptors.DescriptorValidationException e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-}
diff --git a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protobuf/ProtobufDescriptorMap.java b/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protobuf/ProtobufDescriptorMap.java
deleted file mode 100644
index 008b527..0000000
--- a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protobuf/ProtobufDescriptorMap.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.common.protobuf;
-
-import com.google.protobuf.DescriptorProtos;
-import com.google.protobuf.Descriptors;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
-import java.nio.file.Files;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-
-public final class ProtobufDescriptorMap {
-
-  public static ProtobufDescriptorMap from(String fileDescriptorPath) throws IOException {
-    File file = new File(fileDescriptorPath);
-    byte[] descriptorBytes = Files.readAllBytes(file.toPath());
-    DescriptorProtos.FileDescriptorSet fileDescriptorSet =
-        DescriptorProtos.FileDescriptorSet.parseFrom(descriptorBytes);
-    return from(fileDescriptorSet);
-  }
-
-  public static ProtobufDescriptorMap from(URL fileDescriptorUrl) throws IOException {
-    InputStream stream = fileDescriptorUrl.openStream();
-
-    DescriptorProtos.FileDescriptorSet fileDescriptorSet =
-        DescriptorProtos.FileDescriptorSet.parseFrom(stream);
-    return from(fileDescriptorSet);
-  }
-
-  public static ProtobufDescriptorMap from(DescriptorProtos.FileDescriptorSet fileDescriptorSet) {
-    Map<String, Descriptors.FileDescriptor> resolvedSet =
-        FileDescriptorResolver.resolve(fileDescriptorSet);
-
-    Map<String, Descriptors.GenericDescriptor> messageOrEnumDescriptors = new HashMap<>();
-
-    for (Descriptors.FileDescriptor fileDescriptor : resolvedSet.values()) {
-      addMessages(messageOrEnumDescriptors, fileDescriptor, packageName(fileDescriptor));
-      addEnums(messageOrEnumDescriptors, fileDescriptor, packageName(fileDescriptor));
-    }
-    return new ProtobufDescriptorMap(messageOrEnumDescriptors);
-  }
-
-  private final Map<String, Descriptors.GenericDescriptor> descriptorByName;
-
-  private ProtobufDescriptorMap(Map<String, Descriptors.GenericDescriptor> descriptorByName) {
-    this.descriptorByName = descriptorByName;
-  }
-
-  public Optional<Descriptors.GenericDescriptor> getDescriptorByName(String messageFullName) {
-    Descriptors.GenericDescriptor descriptor = descriptorByName.get(messageFullName);
-    return Optional.ofNullable(descriptor);
-  }
-
-  private static String packageName(Descriptors.FileDescriptor proto) {
-    String packageName = proto.getPackage();
-    if (!packageName.isEmpty()) {
-      packageName = packageName + ".";
-    }
-    return packageName;
-  }
-
-  private static void addMessages(
-      Map<String, Descriptors.GenericDescriptor> descriptors,
-      Descriptors.FileDescriptor proto,
-      String packageName) {
-    for (Descriptors.Descriptor message : proto.getMessageTypes()) {
-      String fullName = packageName + message.getName();
-      descriptors.put(fullName, message);
-    }
-  }
-
-  private static void addEnums(
-      Map<String, Descriptors.GenericDescriptor> descriptors,
-      Descriptors.FileDescriptor descriptor,
-      String packageName) {
-    for (Descriptors.EnumDescriptor message : descriptor.getEnumTypes()) {
-      String fullName = packageName + message.getName();
-      descriptors.put(fullName, message);
-    }
-  }
-}
diff --git a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/PathFragment.java b/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/PathFragment.java
deleted file mode 100644
index b83595a..0000000
--- a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/PathFragment.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.common.protopath;
-
-import java.util.Objects;
-
-final class PathFragment {
-  private final String name;
-  private final int index;
-
-  PathFragment(String name, int index) {
-    this.name = name;
-    this.index = index;
-  }
-
-  PathFragment(String name) {
-    this(name, -1);
-  }
-
-  boolean isRepeated() {
-    return index >= 0;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  int getIndex() {
-    return index;
-  }
-
-  @Override
-  public String toString() {
-    return "ParsedField{" + "name='" + name + '\'' + ", index=" + index + '}';
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-    PathFragment that = (PathFragment) o;
-    return index == that.index && name.equals(that.name);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(name, index);
-  }
-}
diff --git a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/PathFragmentDescriptor.java b/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/PathFragmentDescriptor.java
deleted file mode 100644
index bfb9273..0000000
--- a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/PathFragmentDescriptor.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.common.protopath;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import java.util.Objects;
-
-final class PathFragmentDescriptor {
-  private final Descriptors.FieldDescriptor descriptor;
-  private final PathFragment pathFragment;
-
-  PathFragmentDescriptor(Descriptors.FieldDescriptor descriptor, PathFragment pathFragment) {
-    this.descriptor = Objects.requireNonNull(descriptor);
-    this.pathFragment = Objects.requireNonNull(pathFragment);
-  }
-
-  Object value(Message message) {
-    int index = pathFragment.getIndex();
-    if (index >= 0) {
-      return message.getRepeatedField(descriptor, index);
-    }
-    return message.getField(descriptor);
-  }
-}
diff --git a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/ProtobufDynamicMessageLens.java b/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/ProtobufDynamicMessageLens.java
deleted file mode 100644
index 966aef7..0000000
--- a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/ProtobufDynamicMessageLens.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.common.protopath;
-
-import com.google.protobuf.Message;
-import java.util.List;
-import java.util.function.Function;
-
-final class ProtobufDynamicMessageLens implements Function<Message, Object> {
-  private final PathFragmentDescriptor[] path;
-  private final PathFragmentDescriptor value;
-
-  ProtobufDynamicMessageLens(List<PathFragmentDescriptor> path) {
-    this.path = path.subList(0, path.size() - 1).toArray(new PathFragmentDescriptor[0]);
-    this.value = path.get(path.size() - 1);
-  }
-
-  @Override
-  public Object apply(Message message) {
-    message = traverseToTheLastMessage(message);
-    return value.value(message);
-  }
-
-  /**
-   * Traverse the path from root to the last nested message. At each traversed depth follow the next
-   * FiledDescriptor specified in descriptorPath for that depth. The returned message would be the
-   * last Message which contains the desired value. For example the path defined by: {@code .a.b.c}
-   * would result with {@code b} returned.
-   */
-  private Message traverseToTheLastMessage(Message root) {
-    for (PathFragmentDescriptor p : path) {
-      root = (Message) p.value(root);
-    }
-    return root;
-  }
-}
diff --git a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPath.java b/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPath.java
deleted file mode 100644
index fe21880..0000000
--- a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPath.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.common.protopath;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import java.util.List;
-import java.util.function.Function;
-
-public final class ProtobufPath {
-
-  /**
-   * Compile a (limited) {@code ProtocolBuffer}'s path expression.
-   *
-   * <p>A {@code ProtocolBuffer}'s path expression applied to a {@link com.google.protobuf.Message}
-   * and can be one of the following:
-   *
-   * <ul>
-   *   <li>Field selector - donated by a {@code .field} expression.
-   *   <li>A repeated filed index - donated by {@code .field[index]} expression.
-   * </ul>
-   *
-   * <p>Each path expression starts with a {@code $} symbol to donate the root message. For example,
-   * with this message type:
-   *
-   * <pre>{@code
-   * message Foo {
-   *     string baz = 1;
-   * }
-   * }</pre>
-   *
-   * The following expression can select the field {@code baz}: {@code $.baz}.
-   *
-   * @param pathString an {@code ProtocolBuffer}'s path expression.
-   * @return an ordered list of path fragments.
-   */
-  public static Function<Message, ?> protobufPath(
-      Descriptors.Descriptor messageDescriptor, String pathString) {
-    List<PathFragment> fields = ProtobufPathParser.parse(pathString);
-    List<PathFragmentDescriptor> pathFragments =
-        ProtobufPathCompiler.compile(messageDescriptor, fields);
-    return new ProtobufDynamicMessageLens(pathFragments);
-  }
-}
diff --git a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPathCompiler.java b/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPathCompiler.java
deleted file mode 100644
index 18cda49..0000000
--- a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPathCompiler.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.common.protopath;
-
-import com.google.protobuf.Descriptors;
-import java.util.ArrayList;
-import java.util.List;
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
-
-final class ProtobufPathCompiler {
-
-  static List<PathFragmentDescriptor> compile(
-      Descriptors.Descriptor messageDescriptor, List<PathFragment> pathFragments) {
-    List<PathFragmentDescriptor> accessors = new ArrayList<>();
-    for (int i = 0; i < pathFragments.size(); i++) {
-      PathFragment pathFragment = pathFragments.get(i);
-      Descriptors.FieldDescriptor f = findFieldByName(messageDescriptor, pathFragment);
-      accessors.add(new PathFragmentDescriptor(f, pathFragment));
-      if (i < pathFragments.size() - 1) {
-        // all fragments expect the last one in the path
-        // are of Message type (the last one can be primitive)
-        messageDescriptor = f.getMessageType();
-      }
-    }
-    return accessors;
-  }
-
-  @Nonnull
-  private static Descriptors.FieldDescriptor findFieldByName(
-      Descriptors.Descriptor messageDescriptor, PathFragment pathFragment) {
-    @Nullable
-    Descriptors.FieldDescriptor actualField =
-        messageDescriptor.findFieldByName(pathFragment.getName());
-    if (actualField == null) {
-      throw new IllegalStateException(
-          "Unable to find the field "
-              + pathFragment.getName()
-              + " on "
-              + messageDescriptor.getFullName());
-    }
-    if (pathFragment.isRepeated() && !actualField.isRepeated()) {
-      throw new IllegalArgumentException(
-          "Can't index into a non repeated field " + actualField.getFullName());
-    }
-    return actualField;
-  }
-}
diff --git a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPathParser.java b/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPathParser.java
deleted file mode 100644
index af765af..0000000
--- a/statefun-flink/statefun-flink-common/src/main/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPathParser.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * 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.common.protopath;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-final class ProtobufPathParser {
-  private static final Pattern REPEATED_FIELD_PATTERN = Pattern.compile("(\\w+)\\[(\\d+)]");
-  private static final Pattern PLAIN_FIELD_PATTERN = Pattern.compile("(\\w+)");
-
-  /**
-   * Parses a (limited) {@code ProtocolBuffer}'s path expression.
-   *
-   * <p>A {@code ProtocolBuffer}'s path expression applied to a root {@link
-   * com.google.protobuf.Message} and can be one of the following:
-   *
-   * <ul>
-   *   <li>Field selector - donated by a {@code .field} expression.
-   *   <li>A repeated filed index - donated by {@code .field[index]} expression.
-   * </ul>
-   *
-   * <p>Each path expression starts with a {@code $} symbol to donate the root message. For example,
-   * with this message type:
-   *
-   * <pre>{@code
-   * message Foo {
-   *     string baz = 1;
-   * }
-   * }</pre>
-   *
-   * The following expression can select the field {@code baz}: {@code $.baz}.
-   *
-   * @param protobufPath an {@code ProtocolBuffer}'s path expression.
-   * @return an ordered list of path fragments.
-   */
-  static List<PathFragment> parse(String protobufPath) {
-    validatePrefix(protobufPath);
-    String[] tokens = protobufPath.substring(2).split("\\.");
-
-    List<PathFragment> pathFragments = new ArrayList<>();
-
-    for (String token : tokens) {
-      Matcher repeatedFieldMatcher = REPEATED_FIELD_PATTERN.matcher(token);
-      if (repeatedFieldMatcher.matches()) {
-        pathFragments.add(parseRepeatedField(protobufPath, repeatedFieldMatcher));
-        continue;
-      }
-      Matcher plainFieldMatcher = PLAIN_FIELD_PATTERN.matcher(token);
-      if (plainFieldMatcher.matches()) {
-        pathFragments.add(parsePlainField(plainFieldMatcher));
-        continue;
-      }
-      throw new IllegalArgumentException("Parse error in " + protobufPath + " at " + token);
-    }
-    return pathFragments;
-  }
-
-  private static PathFragment parsePlainField(Matcher plainFieldMatcher) {
-    String fieldName = plainFieldMatcher.group(1);
-    return new PathFragment(fieldName);
-  }
-
-  private static PathFragment parseRepeatedField(
-      String protobufPath, Matcher repeatedFieldMatcher) {
-    String fieldName = repeatedFieldMatcher.group(1);
-    String indexString = repeatedFieldMatcher.group(2);
-    int index = Integer.parseInt(indexString);
-    if (index < 0) {
-      throw new IllegalArgumentException(
-          "Parse error in "
-              + protobufPath
-              + " at a repeated field "
-              + fieldName
-              + " index is negative "
-              + index);
-    }
-    return new PathFragment(fieldName, index);
-  }
-
-  private static void validatePrefix(String protobufPath) {
-    if (protobufPath.length() < 2) {
-      throw new IllegalArgumentException("Path is empty");
-    }
-    if (protobufPath.charAt(0) != '$') {
-      throw new IllegalArgumentException("Path must start with a $ sign");
-    }
-    if (protobufPath.charAt(1) != '.') {
-      throw new IllegalArgumentException("A field access must start with a .");
-    }
-  }
-}
diff --git a/statefun-flink/statefun-flink-common/src/test/java/org/apache/flink/statefun/flink/common/protobuf/ProtobufDescriptorMapTest.java b/statefun-flink/statefun-flink-common/src/test/java/org/apache/flink/statefun/flink/common/protobuf/ProtobufDescriptorMapTest.java
deleted file mode 100644
index 9104307..0000000
--- a/statefun-flink/statefun-flink-common/src/test/java/org/apache/flink/statefun/flink/common/protobuf/ProtobufDescriptorMapTest.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * 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.common.protobuf;
-
-import static org.hamcrest.CoreMatchers.not;
-import static org.hamcrest.MatcherAssert.assertThat;
-
-import com.google.protobuf.DescriptorProtos;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Optional;
-import org.hamcrest.Description;
-import org.hamcrest.Matcher;
-import org.hamcrest.TypeSafeMatcher;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class ProtobufDescriptorMapTest {
-
-  private static final String TEST_DESCRIPTOR_NAME = "test.desc";
-  private static DescriptorProtos.FileDescriptorSet FILE_DESCRIPTOR_SET;
-
-  @BeforeClass
-  public static void setup() throws IOException {
-    // Load the test file descriptor set
-    InputStream descriptorStream =
-        ProtobufDescriptorMap.class.getClassLoader().getResourceAsStream(TEST_DESCRIPTOR_NAME);
-    FILE_DESCRIPTOR_SET = DescriptorProtos.FileDescriptorSet.parseFrom(descriptorStream);
-  }
-
-  @Test
-  public void exampleUsage() {
-    ProtobufDescriptorMap map = ProtobufDescriptorMap.from(FILE_DESCRIPTOR_SET);
-
-    assertThat(map.getDescriptorByName("org.apache.flink.test.NestedMessage"), isPresent());
-    assertThat(
-        map.getDescriptorByName("org.apache.flink.test.Non Existing Type"), not(isPresent()));
-  }
-
-  @Test
-  public void includedTypeIsVisible() {
-    ProtobufDescriptorMap map = ProtobufDescriptorMap.from(FILE_DESCRIPTOR_SET);
-
-    assertThat(map.getDescriptorByName("google.protobuf.Any"), isPresent());
-  }
-
-  @Test
-  public void enumsAreVisible() {
-    ProtobufDescriptorMap map = ProtobufDescriptorMap.from(FILE_DESCRIPTOR_SET);
-
-    assertThat(map.getDescriptorByName("org.apache.flink.test.Letter"), isPresent());
-  }
-
-  private static <T> Matcher<Optional<T>> isPresent() {
-    return new TypeSafeMatcher<Optional<T>>() {
-      @Override
-      protected boolean matchesSafely(Optional<T> t) {
-        return t.isPresent();
-      }
-
-      @Override
-      public void describeTo(Description description) {
-        description.appendText("A present j.u.Optional");
-      }
-    };
-  }
-}
diff --git a/statefun-flink/statefun-flink-common/src/test/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPathParserTest.java b/statefun-flink/statefun-flink-common/src/test/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPathParserTest.java
deleted file mode 100644
index d04bc85..0000000
--- a/statefun-flink/statefun-flink-common/src/test/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPathParserTest.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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.common.protopath;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-
-import java.util.List;
-import org.hamcrest.Matchers;
-import org.junit.Test;
-
-public class ProtobufPathParserTest {
-
-  @Test
-  public void exampleUsage() {
-    List<PathFragment> fragments = ProtobufPathParser.parse("$.foo.bar.baz");
-
-    assertThat(fragments, Matchers.contains(fragment("foo"), fragment("bar"), fragment("baz")));
-  }
-
-  @Test
-  public void repeatedField() {
-    List<PathFragment> fragments = ProtobufPathParser.parse("$.foo[1].bar[7]");
-
-    assertThat(fragments, Matchers.contains(fragment("foo", 1), fragment("bar", 7)));
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void noPrefix() {
-    ProtobufPathParser.parse("foo");
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void badIndex() {
-    ProtobufPathParser.parse("$.foo[-1]");
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void badSyntaxWithIndexing() {
-    ProtobufPathParser.parse("$.foo[[1]");
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void badSyntaxWithMultipleDots() {
-    ProtobufPathParser.parse("$..foo..bar");
-  }
-
-  private static PathFragment fragment(String name) {
-    return new PathFragment(name);
-  }
-
-  private static PathFragment fragment(String name, int index) {
-    return new PathFragment(name, index);
-  }
-}
diff --git a/statefun-flink/statefun-flink-common/src/test/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPathTest.java b/statefun-flink/statefun-flink-common/src/test/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPathTest.java
deleted file mode 100644
index 61d1d7d..0000000
--- a/statefun-flink/statefun-flink-common/src/test/java/org/apache/flink/statefun/flink/common/protopath/ProtobufPathTest.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * 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.common.protopath;
-
-import static org.apache.flink.statefun.flink.common.protopath.ProtobufPath.protobufPath;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-
-import com.google.protobuf.Any;
-import com.google.protobuf.Message;
-import java.util.function.Function;
-import org.apache.flink.statefun.flink.common.protobuf.generated.TestProtos;
-import org.apache.flink.statefun.flink.common.protobuf.generated.TestProtos.NestedMessage;
-import org.apache.flink.statefun.flink.common.protobuf.generated.TestProtos.SimpleMessage;
-import org.junit.Test;
-
-public class ProtobufPathTest {
-
-  @Test
-  public void exampleUsage() {
-    Message message = SimpleMessage.newBuilder().setName("bob").build();
-
-    Function<Message, ?> getter = protobufPath(message.getDescriptorForType(), "$.name");
-
-    assertThat(getter.apply(message), is("bob"));
-  }
-
-  @Test
-  public void repeatedMessage() {
-    Message message =
-        TestProtos.RepeatedMessage.newBuilder()
-            .addSimpleMessage(SimpleMessage.newBuilder().setName("bruce"))
-            .addSimpleMessage(SimpleMessage.newBuilder().setName("lee"))
-            .build();
-
-    Function<Message, ?> getter =
-        protobufPath(message.getDescriptorForType(), "$.simple_message[1].name");
-
-    assertThat(getter.apply(message), is("lee"));
-  }
-
-  @Test
-  public void nestedMessage() {
-    Message message =
-        NestedMessage.newBuilder().setFoo(NestedMessage.Foo.newBuilder().setName("lee")).build();
-
-    Function<Message, ?> getter = protobufPath(message.getDescriptorForType(), "$.foo.name");
-
-    assertThat(getter.apply(message), is("lee"));
-  }
-
-  @Test
-  public void messageWithEnum() {
-    TestProtos.MessageWithEnum message =
-        TestProtos.MessageWithEnum.newBuilder().setLetter(TestProtos.Letter.B).build();
-
-    Function<Message, ?> getter = protobufPath(message.getDescriptorForType(), "$.letter");
-
-    Object apply = getter.apply(message);
-    assertThat(apply, is(TestProtos.Letter.B.getValueDescriptor()));
-  }
-
-  @Test
-  public void importedMessage() {
-    Message message =
-        TestProtos.ImportedMessage.newBuilder().setImported(Any.getDefaultInstance()).build();
-
-    Function<Message, ?> getter = protobufPath(message.getDescriptorForType(), "$.imported");
-
-    assertThat(getter.apply(message), is(Any.getDefaultInstance()));
-  }
-
-  @Test
-  public void oneOfMessage() {
-    Message message = TestProtos.OneOfMessage.newBuilder().setBar(1234).build();
-
-    Function<Message, ?> getter = protobufPath(message.getDescriptorForType(), "$.bar");
-
-    assertThat(getter.apply(message), is(1234L));
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/pom.xml b/statefun-flink/statefun-flink-core/pom.xml
index 9aaec82..50e1028 100644
--- a/statefun-flink/statefun-flink-core/pom.xml
+++ b/statefun-flink/statefun-flink-core/pom.xml
@@ -57,6 +57,11 @@
             <artifactId>statefun-flink-common</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>statefun-flink-extensions</artifactId>
+            <version>${project.version}</version>
+        </dependency>
 
         <!-- flink runtime -->
         <dependency>
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/StatefulFunctionsUniverse.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/StatefulFunctionsUniverse.java
index 4fffffc..71cda2c 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/StatefulFunctionsUniverse.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/StatefulFunctionsUniverse.java
@@ -23,6 +23,7 @@
 import java.util.Map;
 import java.util.Objects;
 import javax.annotation.Nullable;
+import org.apache.flink.statefun.extensions.ExtensionModule;
 import org.apache.flink.statefun.flink.core.message.MessageFactoryKey;
 import org.apache.flink.statefun.flink.core.spi.ExtensionResolver;
 import org.apache.flink.statefun.flink.core.types.StaticallyRegisteredTypes;
@@ -40,7 +41,6 @@
 import org.apache.flink.statefun.sdk.io.IngressIdentifier;
 import org.apache.flink.statefun.sdk.io.IngressSpec;
 import org.apache.flink.statefun.sdk.io.Router;
-import org.apache.flink.statefun.sdk.spi.ExtensionModule;
 import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
 
 public final class StatefulFunctionsUniverse
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/grpcfn/GrpcFunction.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/grpcfn/GrpcFunction.java
deleted file mode 100644
index 9748853..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/grpcfn/GrpcFunction.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.grpcfn;
-
-import java.util.Objects;
-import org.apache.flink.statefun.sdk.Context;
-import org.apache.flink.statefun.sdk.StatefulFunction;
-
-public class GrpcFunction implements StatefulFunction {
-  private final GrpcFunctionSpec functionSpec;
-
-  public GrpcFunction(GrpcFunctionSpec functionSpec) {
-    this.functionSpec = Objects.requireNonNull(functionSpec);
-  }
-
-  @Override
-  public void invoke(Context context, Object input) {
-    throw new UnsupportedOperationException(functionSpec.functionType() + " is not yet supported.");
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/grpcfn/GrpcFunctionProvider.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/grpcfn/GrpcFunctionProvider.java
deleted file mode 100644
index 828cf22..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/grpcfn/GrpcFunctionProvider.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.grpcfn;
-
-import java.util.Map;
-import org.apache.flink.statefun.sdk.FunctionType;
-import org.apache.flink.statefun.sdk.StatefulFunction;
-import org.apache.flink.statefun.sdk.StatefulFunctionProvider;
-
-public class GrpcFunctionProvider implements StatefulFunctionProvider {
-  private final Map<FunctionType, GrpcFunctionSpec> supportedTypes;
-
-  public GrpcFunctionProvider(Map<FunctionType, GrpcFunctionSpec> supportedTypes) {
-    this.supportedTypes = supportedTypes;
-  }
-
-  @Override
-  public StatefulFunction functionOfType(FunctionType type) {
-    GrpcFunctionSpec spec = supportedTypes.get(type);
-    if (spec == null) {
-      throw new IllegalArgumentException("Unsupported type " + type);
-    }
-    return new GrpcFunction(spec);
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/grpcfn/GrpcFunctionSpec.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/grpcfn/GrpcFunctionSpec.java
deleted file mode 100644
index 711d5b7..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/grpcfn/GrpcFunctionSpec.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.grpcfn;
-
-import java.net.SocketAddress;
-import java.util.Objects;
-import org.apache.flink.statefun.flink.core.jsonmodule.FunctionSpec;
-import org.apache.flink.statefun.sdk.FunctionType;
-
-public final class GrpcFunctionSpec implements FunctionSpec {
-  private final FunctionType functionType;
-  private final SocketAddress functionAddress;
-
-  public GrpcFunctionSpec(FunctionType functionType, SocketAddress functionAddress) {
-    this.functionType = Objects.requireNonNull(functionType);
-    this.functionAddress = Objects.requireNonNull(functionAddress);
-  }
-
-  @Override
-  public FunctionType functionType() {
-    return functionType;
-  }
-
-  @Override
-  public Kind kind() {
-    return Kind.GRPC;
-  }
-
-  public SocketAddress address() {
-    return functionAddress;
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/DefaultHttpRequestReplyClientFactory.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/DefaultHttpRequestReplyClientFactory.java
index fd7fcc2..2cfadbb 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/DefaultHttpRequestReplyClientFactory.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/DefaultHttpRequestReplyClientFactory.java
@@ -24,25 +24,28 @@
 import javax.annotation.Nullable;
 import okhttp3.HttpUrl;
 import okhttp3.OkHttpClient;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationFeature;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.flink.statefun.flink.common.SetContextClassLoader;
+import org.apache.flink.statefun.flink.common.json.StateFunObjectMapper;
 import org.apache.flink.statefun.flink.core.reqreply.ClassLoaderSafeRequestReplyClient;
 import org.apache.flink.statefun.flink.core.reqreply.RequestReplyClient;
 import org.apache.flink.statefun.flink.core.reqreply.RequestReplyClientFactory;
 
 public final class DefaultHttpRequestReplyClientFactory implements RequestReplyClientFactory {
 
-  /** Unknown fields in client properties are silently ignored. */
-  private static final ObjectMapper OBJ_MAPPER =
-      new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+  public static final DefaultHttpRequestReplyClientFactory INSTANCE =
+      new DefaultHttpRequestReplyClientFactory();
+
+  private static final ObjectMapper OBJ_MAPPER = StateFunObjectMapper.create();
 
   /** lazily initialized by {@link #createTransportClient} */
   @Nullable private OkHttpClient sharedClient;
 
   private volatile boolean shutdown;
 
+  private DefaultHttpRequestReplyClientFactory() {}
+
   @Override
   public RequestReplyClient createTransportClient(ObjectNode transportProperties, URI endpointUrl) {
     final DefaultHttpRequestReplyClient client = createClient(transportProperties, endpointUrl);
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/DefaultHttpRequestReplyClientSpec.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/DefaultHttpRequestReplyClientSpec.java
index 4639fc2..5aa3785 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/DefaultHttpRequestReplyClientSpec.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/DefaultHttpRequestReplyClientSpec.java
@@ -18,16 +18,10 @@
 
 package org.apache.flink.statefun.flink.core.httpfn;
 
-import java.io.IOException;
 import java.time.Duration;
 import java.util.Objects;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSetter;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonDeserializer;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
-import org.apache.flink.util.TimeUtils;
 
 public final class DefaultHttpRequestReplyClientSpec {
 
@@ -74,25 +68,21 @@
     private Duration writeTimeout = DEFAULT_HTTP_WRITE_TIMEOUT;
 
     @JsonSetter("call")
-    @JsonDeserialize(using = DurationJsonDeserialize.class)
     public void setCallTimeout(Duration callTimeout) {
       this.callTimeout = requireNonZeroDuration(callTimeout);
     }
 
     @JsonSetter("connect")
-    @JsonDeserialize(using = DurationJsonDeserialize.class)
     public void setConnectTimeout(Duration connectTimeout) {
       this.connectTimeout = requireNonZeroDuration(connectTimeout);
     }
 
     @JsonSetter("read")
-    @JsonDeserialize(using = DurationJsonDeserialize.class)
     public void setReadTimeout(Duration readTimeout) {
       this.readTimeout = requireNonZeroDuration(readTimeout);
     }
 
     @JsonSetter("write")
-    @JsonDeserialize(using = DurationJsonDeserialize.class)
     public void setWriteTimeout(Duration writeTimeout) {
       this.writeTimeout = requireNonZeroDuration(writeTimeout);
     }
@@ -122,12 +112,4 @@
       return duration;
     }
   }
-
-  private static final class DurationJsonDeserialize extends JsonDeserializer<Duration> {
-    @Override
-    public Duration deserialize(
-        JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
-      return TimeUtils.parseDuration(jsonParser.getText());
-    }
-  }
 }
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/HttpFunctionEndpointSpec.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/HttpFunctionEndpointSpec.java
index 428e474..c8ce68e 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/HttpFunctionEndpointSpec.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/HttpFunctionEndpointSpec.java
@@ -17,26 +17,34 @@
  */
 package org.apache.flink.statefun.flink.core.httpfn;
 
-import static org.apache.flink.statefun.flink.core.httpfn.TransportClientConstants.OKHTTP_CLIENT_FACTORY_TYPE;
-
 import java.io.Serializable;
 import java.util.Objects;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.flink.statefun.flink.core.jsonmodule.FunctionEndpointSpec;
+import org.apache.flink.statefun.flink.core.httpfn.jsonutils.TargetFunctionsJsonDeserializer;
+import org.apache.flink.statefun.flink.core.httpfn.jsonutils.UrlPathTemplateJsonDeserializer;
 import org.apache.flink.statefun.sdk.TypeName;
 
-public final class HttpFunctionEndpointSpec implements FunctionEndpointSpec, Serializable {
+@JsonDeserialize(builder = HttpFunctionEndpointSpec.Builder.class)
+public final class HttpFunctionEndpointSpec implements Serializable {
 
   private static final long serialVersionUID = 1;
 
   private static final Integer DEFAULT_MAX_NUM_BATCH_REQUESTS = 1000;
+  private static final TransportClientSpec DEFAULT_TRANSPORT_CLIENT_SPEC =
+      new TransportClientSpec(
+          TransportClientConstants.OKHTTP_CLIENT_FACTORY_TYPE,
+          new ObjectMapper().createObjectNode());
 
   // ============================================================
   //  Request-Reply invocation protocol configurations
   // ============================================================
 
-  private final Target target;
+  private final TargetFunctions targetFunctions;
   private final UrlPathTemplate urlPathTemplate;
   private final int maxNumBatchRequests;
 
@@ -47,34 +55,27 @@
   private final TypeName transportClientFactoryType;
   private final ObjectNode transportClientProps;
 
-  public static Builder builder(Target target, UrlPathTemplate urlPathTemplate) {
-    return new Builder(target, urlPathTemplate);
+  public static Builder builder(TargetFunctions targetFunctions, UrlPathTemplate urlPathTemplate) {
+    return new Builder(targetFunctions, urlPathTemplate);
   }
 
   private HttpFunctionEndpointSpec(
-      Target target,
+      TargetFunctions targetFunctions,
       UrlPathTemplate urlPathTemplate,
       int maxNumBatchRequests,
       TypeName transportClientFactoryType,
       ObjectNode transportClientProps) {
-    this.target = target;
+    this.targetFunctions = targetFunctions;
     this.urlPathTemplate = urlPathTemplate;
     this.maxNumBatchRequests = maxNumBatchRequests;
     this.transportClientFactoryType = transportClientFactoryType;
     this.transportClientProps = transportClientProps;
   }
 
-  @Override
-  public Target target() {
-    return target;
+  public TargetFunctions targetFunctions() {
+    return targetFunctions;
   }
 
-  @Override
-  public Kind kind() {
-    return Kind.HTTP;
-  }
-
-  @Override
   public UrlPathTemplate urlPathTemplate() {
     return urlPathTemplate;
   }
@@ -91,43 +92,54 @@
     return transportClientProps;
   }
 
+  @JsonPOJOBuilder
   public static final class Builder {
 
-    private final Target target;
+    private final TargetFunctions targetFunctions;
     private final UrlPathTemplate urlPathTemplate;
+
     private int maxNumBatchRequests = DEFAULT_MAX_NUM_BATCH_REQUESTS;
+    private TransportClientSpec transportClientSpec = DEFAULT_TRANSPORT_CLIENT_SPEC;
 
-    private TypeName transportClientFactoryType = OKHTTP_CLIENT_FACTORY_TYPE;
-    private ObjectNode transportClientProperties = new ObjectMapper().createObjectNode();
-
-    private Builder(Target target, UrlPathTemplate urlPathTemplate) {
-      this.target = Objects.requireNonNull(target);
+    @JsonCreator
+    private Builder(
+        @JsonProperty("functions") @JsonDeserialize(using = TargetFunctionsJsonDeserializer.class)
+            TargetFunctions targetFunctions,
+        @JsonProperty("urlPathTemplate")
+            @JsonDeserialize(using = UrlPathTemplateJsonDeserializer.class)
+            UrlPathTemplate urlPathTemplate) {
+      this.targetFunctions = Objects.requireNonNull(targetFunctions);
       this.urlPathTemplate = Objects.requireNonNull(urlPathTemplate);
     }
 
+    @JsonProperty("maxNumBatchRequests")
     public Builder withMaxNumBatchRequests(int maxNumBatchRequests) {
       this.maxNumBatchRequests = maxNumBatchRequests;
       return this;
     }
 
-    public Builder withTransportClientFactoryType(TypeName transportClientFactoryType) {
-      this.transportClientFactoryType = Objects.requireNonNull(transportClientFactoryType);
+    /**
+     * This is marked with @JsonProperty specifically to tell Jackson to use this method when
+     * deserializing from Json.
+     */
+    @JsonProperty("transport")
+    public Builder withTransport(ObjectNode transportNode) {
+      withTransport(TransportClientSpec.fromJsonNode(transportNode));
       return this;
     }
 
-    public Builder withTransportClientProperties(ObjectNode transportClientProperties) {
-      this.transportClientProperties = Objects.requireNonNull(transportClientProperties);
+    public Builder withTransport(TransportClientSpec transportNode) {
+      this.transportClientSpec = Objects.requireNonNull(transportNode);
       return this;
     }
 
     public HttpFunctionEndpointSpec build() {
-
       return new HttpFunctionEndpointSpec(
-          target,
+          targetFunctions,
           urlPathTemplate,
           maxNumBatchRequests,
-          transportClientFactoryType,
-          transportClientProperties);
+          transportClientSpec.factoryKind(),
+          transportClientSpec.specNode());
     }
   }
 }
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/HttpFunctionProvider.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/HttpFunctionProvider.java
index 2bcb5dd..b30029f 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/HttpFunctionProvider.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/HttpFunctionProvider.java
@@ -18,73 +18,39 @@
 package org.apache.flink.statefun.flink.core.httpfn;
 
 import java.net.URI;
-import java.util.Map;
 import java.util.Objects;
 import javax.annotation.concurrent.NotThreadSafe;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.flink.statefun.flink.core.common.ManagingResources;
-import org.apache.flink.statefun.flink.core.reqreply.RequestReplyClient;
 import org.apache.flink.statefun.flink.core.reqreply.RequestReplyClientFactory;
 import org.apache.flink.statefun.flink.core.reqreply.RequestReplyFunction;
-import org.apache.flink.statefun.flink.core.spi.ExtensionResolver;
 import org.apache.flink.statefun.sdk.FunctionType;
 import org.apache.flink.statefun.sdk.StatefulFunction;
 import org.apache.flink.statefun.sdk.StatefulFunctionProvider;
-import org.apache.flink.statefun.sdk.TypeName;
 
 @NotThreadSafe
 public final class HttpFunctionProvider implements StatefulFunctionProvider, ManagingResources {
 
-  private final Map<FunctionType, HttpFunctionEndpointSpec> specificTypeEndpointSpecs;
-  private final Map<String, HttpFunctionEndpointSpec> perNamespaceEndpointSpecs;
-
-  private final ExtensionResolver extensionResolver;
+  private final HttpFunctionEndpointSpec endpointSpec;
+  private final RequestReplyClientFactory requestReplyClientFactory;
 
   public HttpFunctionProvider(
-      Map<FunctionType, HttpFunctionEndpointSpec> specificTypeEndpointSpecs,
-      Map<String, HttpFunctionEndpointSpec> perNamespaceEndpointSpecs,
-      ExtensionResolver extensionResolver) {
-    this.specificTypeEndpointSpecs = Objects.requireNonNull(specificTypeEndpointSpecs);
-    this.perNamespaceEndpointSpecs = Objects.requireNonNull(perNamespaceEndpointSpecs);
-    this.extensionResolver = Objects.requireNonNull(extensionResolver);
+      HttpFunctionEndpointSpec endpointSpec, RequestReplyClientFactory requestReplyClientFactory) {
+    this.endpointSpec = Objects.requireNonNull(endpointSpec);
+    this.requestReplyClientFactory = Objects.requireNonNull(requestReplyClientFactory);
   }
 
   @Override
   public StatefulFunction functionOfType(FunctionType functionType) {
-    final HttpFunctionEndpointSpec endpointsSpec = getEndpointsSpecOrThrow(functionType);
-    final URI endpointUrl = endpointsSpec.urlPathTemplate().apply(functionType);
+    final URI endpointUrl = endpointSpec.urlPathTemplate().apply(functionType);
 
     return new RequestReplyFunction(
-        endpointsSpec.maxNumBatchRequests(),
-        buildTransportClientFromSpec(endpointUrl, endpointsSpec));
-  }
-
-  private HttpFunctionEndpointSpec getEndpointsSpecOrThrow(FunctionType functionType) {
-    HttpFunctionEndpointSpec endpointSpec = specificTypeEndpointSpecs.get(functionType);
-    if (endpointSpec != null) {
-      return endpointSpec;
-    }
-    endpointSpec = perNamespaceEndpointSpecs.get(functionType.namespace());
-    if (endpointSpec != null) {
-      return endpointSpec;
-    }
-
-    throw new IllegalStateException("Unknown type: " + functionType);
-  }
-
-  private RequestReplyClient buildTransportClientFromSpec(
-      URI endpointUrl, HttpFunctionEndpointSpec endpointsSpec) {
-    final TypeName factoryType = endpointsSpec.transportClientFactoryType();
-    final ObjectNode properties = endpointsSpec.transportClientProperties();
-
-    final RequestReplyClientFactory factory =
-        extensionResolver.resolveExtension(factoryType, RequestReplyClientFactory.class);
-    return factory.createTransportClient(properties, endpointUrl);
+        endpointSpec.maxNumBatchRequests(),
+        requestReplyClientFactory.createTransportClient(
+            endpointSpec.transportClientProperties(), endpointUrl));
   }
 
   @Override
   public void shutdown() {
-    // TODO all RequestReplyClientFactory's need to be shutdown.
-    // TODO This should probably happen in StatefulFunctionsUniverse.
+    requestReplyClientFactory.cleanup();
   }
 }
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/TargetFunctions.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/TargetFunctions.java
new file mode 100644
index 0000000..837c3e4
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/TargetFunctions.java
@@ -0,0 +1,107 @@
+/*
+ * 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.httpfn;
+
+import java.io.Serializable;
+import java.util.Objects;
+import org.apache.flink.statefun.sdk.FunctionType;
+import org.apache.flink.statefun.sdk.FunctionTypeNamespaceMatcher;
+import org.apache.flink.statefun.sdk.TypeName;
+
+public abstract class TargetFunctions implements Serializable {
+
+  public static TargetFunctions fromPatternString(String patternString) {
+    TypeName targetTypeName = TypeName.parseFrom(patternString);
+    if (targetTypeName.namespace().contains("*")) {
+      throw new IllegalArgumentException(
+          "Invalid syntax for target functions. Only <namespace>/<name> or <namespace>/* are supported.");
+    }
+    if (targetTypeName.name().equals("*")) {
+      return TargetFunctions.namespace(targetTypeName.namespace());
+    }
+    if (targetTypeName.name().contains("*")) {
+      throw new IllegalArgumentException(
+          "Invalid syntax for target functions. Only <namespace>/<name> or <namespace>/* are supported.");
+    }
+    final FunctionType functionType =
+        new FunctionType(targetTypeName.namespace(), targetTypeName.name());
+    return TargetFunctions.functionType(functionType);
+  }
+
+  public static TargetFunctions namespace(String namespace) {
+    return new TargetFunctions.NamespaceTarget(
+        FunctionTypeNamespaceMatcher.targetNamespace(namespace));
+  }
+
+  public static TargetFunctions functionType(FunctionType functionType) {
+    return new TargetFunctions.FunctionTypeTarget(functionType);
+  }
+
+  public boolean isSpecificFunctionType() {
+    return this.getClass() == TargetFunctions.FunctionTypeTarget.class;
+  }
+
+  public boolean isNamespace() {
+    return this.getClass() == TargetFunctions.NamespaceTarget.class;
+  }
+
+  public abstract FunctionTypeNamespaceMatcher asNamespace();
+
+  public abstract FunctionType asSpecificFunctionType();
+
+  private static class NamespaceTarget extends TargetFunctions {
+    private static final long serialVersionUID = 1;
+
+    private final FunctionTypeNamespaceMatcher namespaceMatcher;
+
+    private NamespaceTarget(FunctionTypeNamespaceMatcher namespaceMatcher) {
+      this.namespaceMatcher = Objects.requireNonNull(namespaceMatcher);
+    }
+
+    @Override
+    public FunctionTypeNamespaceMatcher asNamespace() {
+      return namespaceMatcher;
+    }
+
+    @Override
+    public FunctionType asSpecificFunctionType() {
+      throw new IllegalStateException("This target is not a specific function type");
+    }
+  }
+
+  private static class FunctionTypeTarget extends TargetFunctions {
+    private static final long serialVersionUID = 1;
+
+    private final FunctionType functionType;
+
+    private FunctionTypeTarget(FunctionType functionType) {
+      this.functionType = Objects.requireNonNull(functionType);
+    }
+
+    @Override
+    public FunctionTypeNamespaceMatcher asNamespace() {
+      throw new IllegalStateException("This target is not a namespace.");
+    }
+
+    @Override
+    public FunctionType asSpecificFunctionType() {
+      return functionType;
+    }
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/TransportClientSpec.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/TransportClientSpec.java
new file mode 100644
index 0000000..b53029a
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/TransportClientSpec.java
@@ -0,0 +1,52 @@
+/*
+ * 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.httpfn;
+
+import java.io.Serializable;
+import java.util.Objects;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.statefun.flink.common.json.Selectors;
+import org.apache.flink.statefun.sdk.TypeName;
+
+public final class TransportClientSpec implements Serializable {
+
+  private static JsonPointer FACTORY_KIND = JsonPointer.compile("/type");
+
+  public static TransportClientSpec fromJsonNode(ObjectNode node) {
+    final TypeName factoryKind = TypeName.parseFrom(Selectors.textAt(node, FACTORY_KIND));
+    return new TransportClientSpec(factoryKind, node);
+  }
+
+  private final TypeName factoryKind;
+  private final ObjectNode specNode;
+
+  public TransportClientSpec(TypeName factoryKind, ObjectNode properties) {
+    this.factoryKind = Objects.requireNonNull(factoryKind);
+    this.specNode = Objects.requireNonNull(properties);
+  }
+
+  public TypeName factoryKind() {
+    return factoryKind;
+  }
+
+  public ObjectNode specNode() {
+    return specNode;
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/TransportClientsModule.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/TransportClientsModule.java
index 1bad8db..df97432 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/TransportClientsModule.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/TransportClientsModule.java
@@ -20,7 +20,7 @@
 
 import com.google.auto.service.AutoService;
 import java.util.Map;
-import org.apache.flink.statefun.sdk.spi.ExtensionModule;
+import org.apache.flink.statefun.extensions.ExtensionModule;
 
 @AutoService(ExtensionModule.class)
 public class TransportClientsModule implements ExtensionModule {
@@ -28,6 +28,6 @@
   public void configure(Map<String, String> globalConfigurations, Binder binder) {
     binder.bindExtension(
         TransportClientConstants.OKHTTP_CLIENT_FACTORY_TYPE,
-        new DefaultHttpRequestReplyClientFactory());
+        DefaultHttpRequestReplyClientFactory.INSTANCE);
   }
 }
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/UrlPathTemplate.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/UrlPathTemplate.java
new file mode 100644
index 0000000..bd8a77a
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/UrlPathTemplate.java
@@ -0,0 +1,40 @@
+/*
+ * 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.httpfn;
+
+import java.io.Serializable;
+import java.net.URI;
+import java.util.Objects;
+import org.apache.flink.statefun.sdk.FunctionType;
+
+public final class UrlPathTemplate implements Serializable {
+  private static final long serialVersionUID = 1;
+
+  private static final String FUNCTION_NAME_HOLDER = "{function.name}";
+
+  private final String template;
+
+  public UrlPathTemplate(String template) {
+    this.template = Objects.requireNonNull(template);
+  }
+
+  public URI apply(FunctionType functionType) {
+    return URI.create(template.replace(FUNCTION_NAME_HOLDER, functionType.name()));
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/binders/v1/HttpEndpointBinderV1.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/binders/v1/HttpEndpointBinderV1.java
new file mode 100644
index 0000000..59b373e
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/binders/v1/HttpEndpointBinderV1.java
@@ -0,0 +1,129 @@
+/*
+ * 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.httpfn.binders.v1;
+
+import java.util.OptionalInt;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.statefun.extensions.ComponentBinder;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.common.json.Selectors;
+import org.apache.flink.statefun.flink.core.httpfn.DefaultHttpRequestReplyClientFactory;
+import org.apache.flink.statefun.flink.core.httpfn.HttpFunctionEndpointSpec;
+import org.apache.flink.statefun.flink.core.httpfn.HttpFunctionProvider;
+import org.apache.flink.statefun.flink.core.httpfn.TargetFunctions;
+import org.apache.flink.statefun.flink.core.httpfn.TransportClientConstants;
+import org.apache.flink.statefun.flink.core.httpfn.TransportClientSpec;
+import org.apache.flink.statefun.flink.core.httpfn.UrlPathTemplate;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+
+/**
+ * Version 1 {@link ComponentBinder} for binding a {@link HttpFunctionProvider}. Corresponding
+ * {@link TypeName} is {@code io.statefun.endpoints.v1/http}.
+ *
+ * <p>Below is an example YAML document of the {@link ComponentJsonObject} recognized by this
+ * binder, with the expected types of each field:
+ *
+ * <pre>
+ * kind: io.statefun.endpoints.v1/http                                (typename)
+ * spec:                                                              (object)
+ *   functions: com.foo.bar/*                                         (typename)
+ *   urlPathTemplate: https://bar.foo.com:8080/{function.name}        (string)
+ *   maxNumBatchRequests: 10000                                       (int, optional)
+ *   timeouts:                                                        (object, optional)
+ *     call: 1minute                                                  (duration, optional)
+ *     connect: 20seconds                                             (duration, optional)
+ *     read: 30seconds                                                (duration, optional)
+ *     write: 3seconds                                                (duration, optional)
+ * </pre>
+ */
+public final class HttpEndpointBinderV1 implements ComponentBinder {
+  static final HttpEndpointBinderV1 INSTANCE = new HttpEndpointBinderV1();
+
+  public static final TypeName KIND_TYPE = TypeName.parseFrom("io.statefun.endpoints.v1/http");
+
+  // =====================================================================
+  //  Json pointers for backwards compatibility
+  // =====================================================================
+
+  private static final JsonPointer TARGET_FUNCTIONS = JsonPointer.compile("/functions");
+  private static final JsonPointer URL_PATH_TEMPLATE = JsonPointer.compile("/urlPathTemplate");
+  private static final JsonPointer MAX_NUM_BATCH_REQUESTS =
+      JsonPointer.compile("/maxNumBatchRequests");
+
+  private HttpEndpointBinderV1() {}
+
+  @Override
+  public void bind(ComponentJsonObject component, StatefulFunctionModule.Binder binder) {
+    validateComponent(component);
+
+    final HttpFunctionEndpointSpec spec = parseSpec(component);
+    final HttpFunctionProvider provider =
+        new HttpFunctionProvider(spec, DefaultHttpRequestReplyClientFactory.INSTANCE);
+
+    final TargetFunctions target = spec.targetFunctions();
+    if (target.isSpecificFunctionType()) {
+      binder.bindFunctionProvider(target.asSpecificFunctionType(), provider);
+    } else {
+      binder.bindFunctionProvider(target.asNamespace(), provider);
+    }
+  }
+
+  private static void validateComponent(ComponentJsonObject componentJsonObject) {
+    final TypeName targetBinderType = componentJsonObject.binderTypename();
+    if (!targetBinderType.equals(KIND_TYPE)) {
+      throw new IllegalStateException(
+          "Received unexpected ModuleComponent to bind: " + componentJsonObject);
+    }
+  }
+
+  private static HttpFunctionEndpointSpec parseSpec(ComponentJsonObject component) {
+    final JsonNode httpEndpointSpecNode = component.specJsonNode();
+
+    final HttpFunctionEndpointSpec.Builder specBuilder =
+        HttpFunctionEndpointSpec.builder(
+            target(httpEndpointSpecNode), urlPathTemplate(httpEndpointSpecNode));
+
+    optionalMaxNumBatchRequests(httpEndpointSpecNode)
+        .ifPresent(specBuilder::withMaxNumBatchRequests);
+
+    final TransportClientSpec transportClientSpec =
+        new TransportClientSpec(
+            TransportClientConstants.OKHTTP_CLIENT_FACTORY_TYPE, (ObjectNode) httpEndpointSpecNode);
+    specBuilder.withTransport(transportClientSpec);
+
+    return specBuilder.build();
+  }
+
+  private static TargetFunctions target(JsonNode functionEndpointSpecNode) {
+    String targetPatternString = Selectors.textAt(functionEndpointSpecNode, TARGET_FUNCTIONS);
+    return TargetFunctions.fromPatternString(targetPatternString);
+  }
+
+  private static UrlPathTemplate urlPathTemplate(JsonNode functionEndpointSpecNode) {
+    String template = Selectors.textAt(functionEndpointSpecNode, URL_PATH_TEMPLATE);
+    return new UrlPathTemplate(template);
+  }
+
+  private static OptionalInt optionalMaxNumBatchRequests(JsonNode functionNode) {
+    return Selectors.optionalIntegerAt(functionNode, MAX_NUM_BATCH_REQUESTS);
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/binders/v1/Module.java
similarity index 63%
copy from statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
copy to statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/binders/v1/Module.java
index 097c3cf..011de2f 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/binders/v1/Module.java
@@ -15,18 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.statefun.flink.core.jsonmodule;
 
-import org.apache.flink.statefun.sdk.FunctionType;
+package org.apache.flink.statefun.flink.core.httpfn.binders.v1;
 
-public interface FunctionSpec {
+import com.google.auto.service.AutoService;
+import java.util.Map;
+import org.apache.flink.statefun.extensions.ExtensionModule;
 
-  FunctionType functionType();
-
-  Kind kind();
-
-  enum Kind {
-    HTTP,
-    GRPC
+@AutoService(ExtensionModule.class)
+public final class Module implements ExtensionModule {
+  @Override
+  public void configure(Map<String, String> globalConfigurations, Binder binder) {
+    binder.bindExtension(HttpEndpointBinderV1.KIND_TYPE, HttpEndpointBinderV1.INSTANCE);
   }
 }
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/binders/v2/HttpEndpointBinderV2.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/binders/v2/HttpEndpointBinderV2.java
new file mode 100644
index 0000000..f94404c
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/binders/v2/HttpEndpointBinderV2.java
@@ -0,0 +1,102 @@
+/*
+ * 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.httpfn.binders.v2;
+
+import static org.apache.flink.statefun.flink.core.spi.ExtensionResolverAccessor.getExtensionResolver;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.statefun.extensions.ComponentBinder;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.common.json.StateFunObjectMapper;
+import org.apache.flink.statefun.flink.core.httpfn.HttpFunctionEndpointSpec;
+import org.apache.flink.statefun.flink.core.httpfn.HttpFunctionProvider;
+import org.apache.flink.statefun.flink.core.httpfn.TargetFunctions;
+import org.apache.flink.statefun.flink.core.reqreply.RequestReplyClientFactory;
+import org.apache.flink.statefun.flink.core.spi.ExtensionResolver;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+
+/**
+ * Version 2 {@link ComponentBinder} for binding a {@link HttpFunctionProvider}. Corresponding
+ * {@link TypeName} is {@code io.statefun.endpoints.v2/http}.
+ *
+ * <p>Below is an example YAML document of the {@link ComponentJsonObject} recognized by this
+ * binder, with the expected types of each field:
+ *
+ * <pre>
+ * kind: io.statefun.endpoints.v2/http                                (typename)
+ * spec:                                                              (object)
+ *   functions: com.foo.bar/*                                         (typename)
+ *   urlPathTemplate: https://bar.foo.com:8080/{function.name}        (string)
+ *   maxNumBatchRequests: 10000                                       (int, optional)
+ *   transports:                                                      (object, optional)
+ *     type: io.statefun.transports/okhttp                            (typename, optional)
+ *     ...                                                            (remaining fields treated directly as properties)
+ * </pre>
+ */
+final class HttpEndpointBinderV2 implements ComponentBinder {
+
+  private static final ObjectMapper SPEC_OBJ_MAPPER = StateFunObjectMapper.create();
+
+  static final HttpEndpointBinderV2 INSTANCE = new HttpEndpointBinderV2();
+
+  static final TypeName KIND_TYPE = TypeName.parseFrom("io.statefun.endpoints.v2/http");
+
+  private HttpEndpointBinderV2() {}
+
+  @Override
+  public void bind(ComponentJsonObject component, StatefulFunctionModule.Binder binder) {
+    validateComponent(component);
+
+    final HttpFunctionEndpointSpec spec = parseSpec(component);
+    final HttpFunctionProvider provider = functionProvider(spec, getExtensionResolver(binder));
+
+    final TargetFunctions target = spec.targetFunctions();
+    if (target.isSpecificFunctionType()) {
+      binder.bindFunctionProvider(target.asSpecificFunctionType(), provider);
+    } else {
+      binder.bindFunctionProvider(target.asNamespace(), provider);
+    }
+  }
+
+  private static void validateComponent(ComponentJsonObject componentJsonObject) {
+    final TypeName targetBinderType = componentJsonObject.binderTypename();
+    if (!targetBinderType.equals(KIND_TYPE)) {
+      throw new IllegalStateException(
+          "Received unexpected ModuleComponent to bind: " + componentJsonObject);
+    }
+  }
+
+  private static HttpFunctionEndpointSpec parseSpec(ComponentJsonObject component) {
+    try {
+      return SPEC_OBJ_MAPPER.treeToValue(component.specJsonNode(), HttpFunctionEndpointSpec.class);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException("Error parsing a HttpFunctionEndpointSpec.", e);
+    }
+  }
+
+  private static HttpFunctionProvider functionProvider(
+      HttpFunctionEndpointSpec spec, ExtensionResolver extensionResolver) {
+    final RequestReplyClientFactory transportClientFactory =
+        extensionResolver.resolveExtension(
+            spec.transportClientFactoryType(), RequestReplyClientFactory.class);
+    return new HttpFunctionProvider(spec, transportClientFactory);
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/binders/v2/Module.java
similarity index 63%
copy from statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
copy to statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/binders/v2/Module.java
index 097c3cf..78c4a7b 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/binders/v2/Module.java
@@ -15,18 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.statefun.flink.core.jsonmodule;
 
-import org.apache.flink.statefun.sdk.FunctionType;
+package org.apache.flink.statefun.flink.core.httpfn.binders.v2;
 
-public interface FunctionSpec {
+import com.google.auto.service.AutoService;
+import java.util.Map;
+import org.apache.flink.statefun.extensions.ExtensionModule;
 
-  FunctionType functionType();
-
-  Kind kind();
-
-  enum Kind {
-    HTTP,
-    GRPC
+@AutoService(ExtensionModule.class)
+public final class Module implements ExtensionModule {
+  @Override
+  public void configure(Map<String, String> globalConfigurations, Binder binder) {
+    binder.bindExtension(HttpEndpointBinderV2.KIND_TYPE, HttpEndpointBinderV2.INSTANCE);
   }
 }
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/jsonutils/TargetFunctionsJsonDeserializer.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/jsonutils/TargetFunctionsJsonDeserializer.java
new file mode 100644
index 0000000..e883412
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/jsonutils/TargetFunctionsJsonDeserializer.java
@@ -0,0 +1,33 @@
+/*
+ * 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.httpfn.jsonutils;
+
+import java.io.IOException;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonDeserializer;
+import org.apache.flink.statefun.flink.core.httpfn.TargetFunctions;
+
+public final class TargetFunctionsJsonDeserializer extends JsonDeserializer<TargetFunctions> {
+  @Override
+  public TargetFunctions deserialize(
+      JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+    return TargetFunctions.fromPatternString(jsonParser.getText());
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/jsonutils/UrlPathTemplateJsonDeserializer.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/jsonutils/UrlPathTemplateJsonDeserializer.java
new file mode 100644
index 0000000..0d11e9b
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/httpfn/jsonutils/UrlPathTemplateJsonDeserializer.java
@@ -0,0 +1,33 @@
+/*
+ * 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.httpfn.jsonutils;
+
+import java.io.IOException;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonDeserializer;
+import org.apache.flink.statefun.flink.core.httpfn.UrlPathTemplate;
+
+public final class UrlPathTemplateJsonDeserializer extends JsonDeserializer<UrlPathTemplate> {
+  @Override
+  public UrlPathTemplate deserialize(
+      JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+    return new UrlPathTemplate(jsonParser.getText());
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/EgressJsonEntity.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/EgressJsonEntity.java
deleted file mode 100644
index e9b8ef7..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/EgressJsonEntity.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.jsonmodule;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.common.json.NamespaceNamePair;
-import org.apache.flink.statefun.flink.common.json.Selectors;
-import org.apache.flink.statefun.flink.core.spi.ExtensionResolver;
-import org.apache.flink.statefun.flink.io.spi.JsonEgressSpec;
-import org.apache.flink.statefun.sdk.EgressType;
-import org.apache.flink.statefun.sdk.io.EgressIdentifier;
-import org.apache.flink.statefun.sdk.reqreply.generated.TypedValue;
-import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule.Binder;
-
-final class EgressJsonEntity implements JsonEntity {
-
-  private static final JsonPointer EGRESS_SPECS_POINTER = JsonPointer.compile("/egresses");
-
-  private static final class MetaPointers {
-    private static final JsonPointer ID = JsonPointer.compile("/egress/meta/id");
-    private static final JsonPointer TYPE = JsonPointer.compile("/egress/meta/type");
-  }
-
-  @Override
-  public void bind(
-      Binder binder,
-      ExtensionResolver extensionResolver,
-      JsonNode moduleSpecRootNode,
-      FormatVersion formatVersion) {
-    final Iterable<? extends JsonNode> egressNodes =
-        Selectors.listAt(moduleSpecRootNode, EGRESS_SPECS_POINTER);
-
-    egressNodes.forEach(
-        egressNode -> {
-          binder.bindEgress(
-              new JsonEgressSpec<>(egressType(egressNode), egressId(egressNode), egressNode));
-        });
-  }
-
-  private static EgressType egressType(JsonNode spec) {
-    String typeString = Selectors.textAt(spec, MetaPointers.TYPE);
-    NamespaceNamePair nn = NamespaceNamePair.from(typeString);
-    return new EgressType(nn.namespace(), nn.name());
-  }
-
-  private static EgressIdentifier<TypedValue> egressId(JsonNode spec) {
-    String egressId = Selectors.textAt(spec, MetaPointers.ID);
-    NamespaceNamePair nn = NamespaceNamePair.from(egressId);
-    return new EgressIdentifier<>(nn.namespace(), nn.name(), TypedValue.class);
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionEndpointJsonEntity.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionEndpointJsonEntity.java
deleted file mode 100644
index 337aa4e..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionEndpointJsonEntity.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/*
- * 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.jsonmodule;
-
-import static java.util.stream.Collectors.groupingBy;
-import static java.util.stream.Collectors.toList;
-
-import java.time.Duration;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Optional;
-import java.util.OptionalInt;
-import java.util.stream.StreamSupport;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.flink.statefun.flink.common.json.Selectors;
-import org.apache.flink.statefun.flink.core.httpfn.HttpFunctionEndpointSpec;
-import org.apache.flink.statefun.flink.core.httpfn.HttpFunctionProvider;
-import org.apache.flink.statefun.flink.core.spi.ExtensionResolver;
-import org.apache.flink.statefun.sdk.FunctionType;
-import org.apache.flink.statefun.sdk.FunctionTypeNamespaceMatcher;
-import org.apache.flink.statefun.sdk.StatefulFunctionProvider;
-import org.apache.flink.statefun.sdk.TypeName;
-import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
-import org.apache.flink.util.TimeUtils;
-
-public final class FunctionEndpointJsonEntity implements JsonEntity {
-
-  private static final JsonPointer FUNCTION_ENDPOINTS_POINTER = JsonPointer.compile("/endpoints");
-
-  private static final class MetaPointers {
-    private static final JsonPointer KIND = JsonPointer.compile("/endpoint/meta/kind");
-  }
-
-  private static final class SpecPointers {
-    private static final JsonPointer TARGET_FUNCTIONS =
-        JsonPointer.compile("/endpoint/spec/functions");
-    private static final JsonPointer URL_PATH_TEMPLATE =
-        JsonPointer.compile("/endpoint/spec/urlPathTemplate");
-    private static final JsonPointer MAX_NUM_BATCH_REQUESTS =
-        JsonPointer.compile("/endpoint/spec/maxNumBatchRequests");
-    private static final JsonPointer TRANSPORT = JsonPointer.compile("/endpoint/spec/transport");
-
-    @Deprecated
-    private static final JsonPointer TIMEOUTS = JsonPointer.compile("/endpoint/spec/timeouts");
-  }
-
-  private static final class TransportPointers {
-    private static final JsonPointer CLIENT_FACTORY_TYPE = JsonPointer.compile("/type");
-  }
-
-  @Override
-  public void bind(
-      StatefulFunctionModule.Binder binder,
-      ExtensionResolver extensionResolver,
-      JsonNode moduleSpecNode,
-      FormatVersion formatVersion) {
-    if (formatVersion.compareTo(FormatVersion.v3_0) < 0) {
-      throw new IllegalArgumentException("endpoints is only supported with format version 3.0.");
-    }
-
-    final Iterable<? extends JsonNode> functionEndpointsSpecNodes =
-        functionEndpointSpecNodes(moduleSpecNode);
-
-    for (Map.Entry<FunctionEndpointSpec.Kind, List<FunctionEndpointSpec>> entry :
-        parseFunctionEndpointSpecs(functionEndpointsSpecNodes, formatVersion).entrySet()) {
-      final Map<FunctionType, FunctionEndpointSpec> specificTypeEndpointSpecs = new HashMap<>();
-      final Map<FunctionTypeNamespaceMatcher, FunctionEndpointSpec> perNamespaceEndpointSpecs =
-          new HashMap<>();
-
-      entry
-          .getValue()
-          .forEach(
-              spec -> {
-                FunctionEndpointSpec.Target target = spec.target();
-                if (target.isSpecificFunctionType()) {
-                  specificTypeEndpointSpecs.put(target.asSpecificFunctionType(), spec);
-                } else {
-                  perNamespaceEndpointSpecs.put(target.asNamespace(), spec);
-                }
-              });
-
-      StatefulFunctionProvider provider =
-          functionProvider(
-              entry.getKey(),
-              specificTypeEndpointSpecs,
-              perNamespaceEndpointSpecs,
-              extensionResolver);
-      specificTypeEndpointSpecs
-          .keySet()
-          .forEach(specificType -> binder.bindFunctionProvider(specificType, provider));
-      perNamespaceEndpointSpecs
-          .keySet()
-          .forEach(namespace -> binder.bindFunctionProvider(namespace, provider));
-    }
-  }
-
-  private static Iterable<? extends JsonNode> functionEndpointSpecNodes(
-      JsonNode moduleSpecRootNode) {
-    return Selectors.listAt(moduleSpecRootNode, FUNCTION_ENDPOINTS_POINTER);
-  }
-
-  private static Map<FunctionEndpointSpec.Kind, List<FunctionEndpointSpec>>
-      parseFunctionEndpointSpecs(
-          Iterable<? extends JsonNode> functionEndpointsSpecNodes, FormatVersion version) {
-    return StreamSupport.stream(functionEndpointsSpecNodes.spliterator(), false)
-        .map(node -> FunctionEndpointJsonEntity.parseFunctionEndpointsSpec(node, version))
-        .collect(groupingBy(FunctionEndpointSpec::kind, toList()));
-  }
-
-  private static FunctionEndpointSpec parseFunctionEndpointsSpec(
-      JsonNode functionEndpointSpecNode, FormatVersion version) {
-    FunctionEndpointSpec.Kind kind = endpointKind(functionEndpointSpecNode);
-
-    switch (kind) {
-      case HTTP:
-        final HttpFunctionEndpointSpec.Builder specBuilder =
-            HttpFunctionEndpointSpec.builder(
-                target(functionEndpointSpecNode), urlPathTemplate(functionEndpointSpecNode));
-
-        optionalMaxNumBatchRequests(functionEndpointSpecNode)
-            .ifPresent(specBuilder::withMaxNumBatchRequests);
-
-        switch (version) {
-          case v3_1:
-            final Optional<ObjectNode> transportSpec =
-                Selectors.optionalObjectAt(functionEndpointSpecNode, SpecPointers.TRANSPORT);
-            transportSpec.ifPresent(spec -> configureHttpTransport(specBuilder, spec));
-            break;
-          case v3_0:
-            final Optional<ObjectNode> deprecatedTimeoutsSpec =
-                Selectors.optionalObjectAt(functionEndpointSpecNode, SpecPointers.TIMEOUTS);
-            deprecatedTimeoutsSpec.ifPresent(
-                spec -> configureDeprecatedHttpTimeoutsSpec(specBuilder, spec));
-            break;
-          default:
-            throw new IllegalStateException("Unsupported format version: " + version);
-        }
-
-        return specBuilder.build();
-      case GRPC:
-        throw new UnsupportedOperationException("GRPC endpoints are not supported yet.");
-      default:
-        throw new IllegalArgumentException("Unrecognized function endpoint kind " + kind);
-    }
-  }
-
-  private static void configureHttpTransport(
-      HttpFunctionEndpointSpec.Builder endpointSpecBuilder, ObjectNode transportSpecNode) {
-    final Optional<TypeName> transportClientFactoryType =
-        Selectors.optionalTextAt(transportSpecNode, TransportPointers.CLIENT_FACTORY_TYPE)
-            .map(TypeName::parseFrom);
-    transportClientFactoryType.ifPresent(endpointSpecBuilder::withTransportClientFactoryType);
-
-    // pass the transport spec node as is as the client properties
-    endpointSpecBuilder.withTransportClientProperties(transportSpecNode);
-  }
-
-  private static void configureDeprecatedHttpTimeoutsSpec(
-      HttpFunctionEndpointSpec.Builder endpointSpecBuilder, ObjectNode deprecatedHttpTimeoutsSpec) {
-    final ObjectNode reconstructedTimeoutsSpec =
-        reconstructTimeoutsSpecNode(deprecatedHttpTimeoutsSpec);
-    endpointSpecBuilder.withTransportClientProperties(reconstructedTimeoutsSpec);
-  }
-
-  private static ObjectNode reconstructTimeoutsSpecNode(ObjectNode deprecatedHttpTimeoutsSpec) {
-    try {
-      return (ObjectNode)
-          new ObjectMapper()
-              .readTree("{\"timeouts\":" + deprecatedHttpTimeoutsSpec.toString() + "}");
-    } catch (Exception e) {
-      throw new RuntimeException("Unable to reconstruct deprecated timeouts spec node.");
-    }
-  }
-
-  private static FunctionEndpointSpec.Kind endpointKind(JsonNode functionEndpointSpecNode) {
-    String endpointKind = Selectors.textAt(functionEndpointSpecNode, MetaPointers.KIND);
-    return FunctionEndpointSpec.Kind.valueOf(endpointKind.toUpperCase(Locale.getDefault()));
-  }
-
-  private static FunctionEndpointSpec.Target target(JsonNode functionEndpointSpecNode) {
-    String targetTypeNameStr =
-        Selectors.textAt(functionEndpointSpecNode, SpecPointers.TARGET_FUNCTIONS);
-    TypeName targetTypeName = TypeName.parseFrom(targetTypeNameStr);
-    if (targetTypeName.namespace().contains("*")) {
-      throw new IllegalArgumentException(
-          "Invalid syntax for "
-              + SpecPointers.TARGET_FUNCTIONS
-              + ". Only <namespace>/<name> or <namespace>/* are supported.");
-    }
-    if (targetTypeName.name().equals("*")) {
-      return FunctionEndpointSpec.Target.namespace(targetTypeName.namespace());
-    }
-    if (targetTypeName.name().contains("*")) {
-      throw new IllegalArgumentException(
-          "Invalid syntax for "
-              + SpecPointers.TARGET_FUNCTIONS
-              + ". Only <namespace>/<name> or <namespace>/* are supported.");
-    }
-    FunctionType functionType = new FunctionType(targetTypeName.namespace(), targetTypeName.name());
-    return FunctionEndpointSpec.Target.functionType(functionType);
-  }
-
-  private static FunctionEndpointSpec.UrlPathTemplate urlPathTemplate(
-      JsonNode functionEndpointSpecNode) {
-    String template = Selectors.textAt(functionEndpointSpecNode, SpecPointers.URL_PATH_TEMPLATE);
-    return new FunctionEndpointSpec.UrlPathTemplate(template);
-  }
-
-  private static OptionalInt optionalMaxNumBatchRequests(JsonNode functionNode) {
-    return Selectors.optionalIntegerAt(functionNode, SpecPointers.MAX_NUM_BATCH_REQUESTS);
-  }
-
-  private static Optional<Duration> optionalTimeoutDuration(
-      JsonNode node, JsonPointer timeoutPointer) {
-    return Selectors.optionalTextAt(node, timeoutPointer).map(TimeUtils::parseDuration);
-  }
-
-  private static StatefulFunctionProvider functionProvider(
-      FunctionEndpointSpec.Kind kind,
-      Map<FunctionType, FunctionEndpointSpec> specificTypeEndpointSpecs,
-      Map<FunctionTypeNamespaceMatcher, FunctionEndpointSpec> perNamespaceEndpointSpecs,
-      ExtensionResolver extensionResolver) {
-    switch (kind) {
-      case HTTP:
-        return new HttpFunctionProvider(
-            castValues(specificTypeEndpointSpecs),
-            castValues(namespaceAsKey(perNamespaceEndpointSpecs)),
-            extensionResolver);
-      case GRPC:
-        throw new UnsupportedOperationException("GRPC endpoints are not supported yet.");
-      default:
-        throw new IllegalStateException("Unexpected kind: " + kind);
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  private static <K, NV extends FunctionEndpointSpec> Map<K, NV> castValues(
-      Map<K, FunctionEndpointSpec> toCast) {
-    return (Map<K, NV>) new HashMap<>(toCast);
-  }
-
-  private static Map<String, FunctionEndpointSpec> namespaceAsKey(
-      Map<FunctionTypeNamespaceMatcher, FunctionEndpointSpec> perNamespaceEndpointSpecs) {
-    final Map<String, FunctionEndpointSpec> converted =
-        new HashMap<>(perNamespaceEndpointSpecs.size());
-    perNamespaceEndpointSpecs.forEach(
-        (namespaceMatcher, spec) -> converted.put(namespaceMatcher.targetNamespace(), spec));
-    return converted;
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionEndpointSpec.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionEndpointSpec.java
deleted file mode 100644
index 46257ee..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionEndpointSpec.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * 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.jsonmodule;
-
-import java.io.Serializable;
-import java.net.URI;
-import java.util.Objects;
-import org.apache.flink.statefun.sdk.FunctionType;
-import org.apache.flink.statefun.sdk.FunctionTypeNamespaceMatcher;
-
-public interface FunctionEndpointSpec {
-
-  Target target();
-
-  Kind kind();
-
-  UrlPathTemplate urlPathTemplate();
-
-  enum Kind {
-    HTTP,
-    GRPC
-  }
-
-  abstract class Target implements Serializable {
-
-    public static Target namespace(String namespace) {
-      return new NamespaceTarget(FunctionTypeNamespaceMatcher.targetNamespace(namespace));
-    }
-
-    public static Target functionType(FunctionType functionType) {
-      return new FunctionTypeTarget(functionType);
-    }
-
-    public boolean isSpecificFunctionType() {
-      return this.getClass() == FunctionTypeTarget.class;
-    }
-
-    public boolean isNamespace() {
-      return this.getClass() == NamespaceTarget.class;
-    }
-
-    public abstract FunctionTypeNamespaceMatcher asNamespace();
-
-    public abstract FunctionType asSpecificFunctionType();
-
-    private static class NamespaceTarget extends Target {
-      private static final long serialVersionUID = 1;
-
-      private final FunctionTypeNamespaceMatcher namespaceMatcher;
-
-      private NamespaceTarget(FunctionTypeNamespaceMatcher namespaceMatcher) {
-        this.namespaceMatcher = Objects.requireNonNull(namespaceMatcher);
-      }
-
-      @Override
-      public FunctionTypeNamespaceMatcher asNamespace() {
-        return namespaceMatcher;
-      }
-
-      @Override
-      public FunctionType asSpecificFunctionType() {
-        throw new IllegalStateException("This target is not a specific function type");
-      }
-    }
-
-    private static class FunctionTypeTarget extends Target {
-      private static final long serialVersionUID = 1;
-
-      private final FunctionType functionType;
-
-      private FunctionTypeTarget(FunctionType functionType) {
-        this.functionType = Objects.requireNonNull(functionType);
-      }
-
-      @Override
-      public FunctionTypeNamespaceMatcher asNamespace() {
-        throw new IllegalStateException("This target is not a namespace.");
-      }
-
-      @Override
-      public FunctionType asSpecificFunctionType() {
-        return functionType;
-      }
-    }
-  }
-
-  class UrlPathTemplate implements Serializable {
-    private static final long serialVersionUID = 1;
-
-    private static final String FUNCTION_NAME_HOLDER = "{function.name}";
-
-    private final String template;
-
-    public UrlPathTemplate(String template) {
-      this.template = Objects.requireNonNull(template);
-    }
-
-    public URI apply(FunctionType functionType) {
-      return URI.create(template.replace(FUNCTION_NAME_HOLDER, functionType.name()));
-    }
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/IngressJsonEntity.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/IngressJsonEntity.java
deleted file mode 100644
index ad1fb35..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/IngressJsonEntity.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * 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.jsonmodule;
-
-import com.google.protobuf.Message;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.common.json.NamespaceNamePair;
-import org.apache.flink.statefun.flink.common.json.Selectors;
-import org.apache.flink.statefun.flink.core.protorouter.AutoRoutableProtobufRouter;
-import org.apache.flink.statefun.flink.core.spi.ExtensionResolver;
-import org.apache.flink.statefun.flink.io.kafka.ProtobufKafkaIngressTypes;
-import org.apache.flink.statefun.flink.io.kinesis.PolyglotKinesisIOTypes;
-import org.apache.flink.statefun.flink.io.spi.JsonIngressSpec;
-import org.apache.flink.statefun.sdk.IngressType;
-import org.apache.flink.statefun.sdk.io.IngressIdentifier;
-import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule.Binder;
-
-final class IngressJsonEntity implements JsonEntity {
-
-  private static final JsonPointer INGRESS_SPECS_POINTER = JsonPointer.compile("/ingresses");
-
-  private static final class MetaPointers {
-    private static final JsonPointer ID = JsonPointer.compile("/ingress/meta/id");
-    private static final JsonPointer TYPE = JsonPointer.compile("/ingress/meta/type");
-  }
-
-  @Override
-  public void bind(
-      Binder binder,
-      ExtensionResolver extensionResolver,
-      JsonNode moduleSpecRootNode,
-      FormatVersion formatVersion) {
-    final Iterable<? extends JsonNode> ingressNodes =
-        Selectors.listAt(moduleSpecRootNode, INGRESS_SPECS_POINTER);
-
-    ingressNodes.forEach(
-        ingressNode -> {
-          final IngressIdentifier<Message> id = ingressId(ingressNode);
-          final IngressType type = ingressType(ingressNode);
-
-          binder.bindIngress(new JsonIngressSpec<>(type, id, ingressNode));
-          if (isAutoRoutableIngress(type)) {
-            binder.bindIngressRouter(id, new AutoRoutableProtobufRouter());
-          }
-        });
-  }
-
-  private static IngressType ingressType(JsonNode spec) {
-    String typeString = Selectors.textAt(spec, MetaPointers.TYPE);
-    NamespaceNamePair nn = NamespaceNamePair.from(typeString);
-    return new IngressType(nn.namespace(), nn.name());
-  }
-
-  private static IngressIdentifier<Message> ingressId(JsonNode ingress) {
-    String ingressId = Selectors.textAt(ingress, MetaPointers.ID);
-    NamespaceNamePair nn = NamespaceNamePair.from(ingressId);
-    return new IngressIdentifier<>(Message.class, nn.namespace(), nn.name());
-  }
-
-  private static boolean isAutoRoutableIngress(IngressType ingressType) {
-    return ingressType.equals(ProtobufKafkaIngressTypes.ROUTABLE_PROTOBUF_KAFKA_INGRESS_TYPE)
-        || ingressType.equals(PolyglotKinesisIOTypes.ROUTABLE_PROTOBUF_KINESIS_INGRESS_TYPE);
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/JsonEntity.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/JsonEntity.java
deleted file mode 100644
index f1afd79..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/JsonEntity.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.jsonmodule;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.core.spi.ExtensionResolver;
-import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule.Binder;
-
-/**
- * A {@link JsonEntity} represents a section within a {@link JsonModule} that should be parsed into
- * application entity specs (of functions, routers, ingresses, egresses, etc.) and bind to the
- * module.
- */
-interface JsonEntity {
-
-  /**
-   * Parse the module spec node, and bind result specs to the module.
-   *
-   * @param binder used to bind specs to the module.
-   * @param moduleSpecNode the root module spec node.
-   * @param formatVersion the format version of the module spec.
-   */
-  void bind(
-      Binder binder,
-      ExtensionResolver extensionResolver,
-      JsonNode moduleSpecNode,
-      FormatVersion formatVersion);
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/JsonModule.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/JsonModule.java
deleted file mode 100644
index d38ab6d..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/JsonModule.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.jsonmodule;
-
-import static java.lang.String.format;
-
-import java.net.URL;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.core.spi.ExtensionResolver;
-import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
-
-final class JsonModule implements StatefulFunctionModule {
-
-  /** Entities in the JSON moduleSpecNode that should be parsed and bound to the module. */
-  private static final List<JsonEntity> ENTITIES =
-      Arrays.asList(
-          new FunctionEndpointJsonEntity(),
-          new IngressJsonEntity(),
-          new RouterJsonEntity(),
-          new EgressJsonEntity());
-
-  private final JsonNode moduleSpecNode;
-  private final FormatVersion formatVersion;
-  private final URL moduleUrl;
-
-  public JsonModule(JsonNode moduleSpecNode, FormatVersion formatVersion, URL moduleUrl) {
-    this.moduleSpecNode = Objects.requireNonNull(moduleSpecNode);
-    this.formatVersion = Objects.requireNonNull(formatVersion);
-    this.moduleUrl = Objects.requireNonNull(moduleUrl);
-  }
-
-  public void configure(Map<String, String> conf, Binder binder) {
-    try {
-      ENTITIES.forEach(
-          jsonEntity ->
-              jsonEntity.bind(binder, getExtensionResolver(binder), moduleSpecNode, formatVersion));
-    } catch (Throwable t) {
-      throw new ModuleConfigurationException(
-          format("Error while parsing module at %s", moduleUrl), t);
-    }
-  }
-
-  // TODO expose ExtensionResolver properly once we have more usages
-  private static ExtensionResolver getExtensionResolver(Binder binder) {
-    return (ExtensionResolver) binder;
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/JsonServiceLoader.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/JsonServiceLoader.java
index d5c7e77..0364cee 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/JsonServiceLoader.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/JsonServiceLoader.java
@@ -15,10 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.statefun.flink.core.jsonmodule;
 
 import java.io.IOException;
 import java.net.URL;
+import java.util.List;
 import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 import org.apache.flink.annotation.VisibleForTesting;
@@ -26,6 +28,7 @@
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLParser;
 import org.apache.flink.statefun.flink.common.ResourceLocator;
 import org.apache.flink.statefun.flink.common.json.Selectors;
 import org.apache.flink.statefun.flink.core.spi.Constants;
@@ -33,9 +36,12 @@
 
 public final class JsonServiceLoader {
 
+  // =======================================================================
+  //  Json pointers for backwards compatibility with legacy format v3.0
+  // =======================================================================
   private static final JsonPointer FORMAT_VERSION = JsonPointer.compile("/version");
-  private static final JsonPointer MODULE_META_TYPE = JsonPointer.compile("/module/meta/type");
   private static final JsonPointer MODULE_SPEC = JsonPointer.compile("/module/spec");
+  private static final JsonPointer MODULE_META_TYPE = JsonPointer.compile("/module/meta/type");
 
   public static Iterable<StatefulFunctionModule> load() {
     ObjectMapper mapper = mapper();
@@ -51,9 +57,12 @@
   @VisibleForTesting
   static StatefulFunctionModule fromUrl(ObjectMapper mapper, URL moduleUrl) {
     try {
-      final JsonNode root = readAndValidateModuleTree(mapper, moduleUrl);
-      return new JsonModule(
-          requireValidModuleSpecNode(moduleUrl, root), requireValidFormatVersion(root), moduleUrl);
+      final List<JsonNode> allComponentNodes = readAllComponentNodes(mapper, moduleUrl);
+
+      if (isLegacySingleRootFormat(allComponentNodes)) {
+        return createLegacyRemoteModule(allComponentNodes.get(0), moduleUrl);
+      }
+      return new RemoteModule(allComponentNodes);
     } catch (Throwable t) {
       throw new RuntimeException("Failed loading a module at " + moduleUrl, t);
     }
@@ -65,11 +74,12 @@
    * <p>A valid resource module definition has to contain the metadata associated with this module,
    * such as its type.
    */
-  private static JsonNode readAndValidateModuleTree(ObjectMapper mapper, URL moduleYamlFile)
+  private static List<JsonNode> readAllComponentNodes(ObjectMapper mapper, URL moduleYamlFile)
       throws IOException {
-    JsonNode root = mapper.readTree(moduleYamlFile);
-    validateMeta(moduleYamlFile, root);
-    return root;
+    YAMLFactory yaml = new YAMLFactory();
+
+    YAMLParser yamlParser = yaml.createParser(moduleYamlFile);
+    return mapper.readValues(yamlParser, JsonNode.class).readAll();
   }
 
   private static void validateMeta(URL moduleYamlFile, JsonNode root) {
@@ -96,6 +106,30 @@
     return moduleSpecNode;
   }
 
+  private static boolean isLegacySingleRootFormat(List<JsonNode> allComponentNodes) {
+    if (allComponentNodes.size() == 1) {
+      final JsonNode singleRootNode = allComponentNodes.get(0);
+      return hasLegacyFormatVersionField(singleRootNode);
+    }
+    return false;
+  }
+
+  private static boolean hasLegacyFormatVersionField(JsonNode singleRootNode) {
+    return Selectors.optionalTextAt(singleRootNode, FORMAT_VERSION).isPresent();
+  }
+
+  private static StatefulFunctionModule createLegacyRemoteModule(
+      JsonNode singleRootNode, URL moduleUrl) {
+    validateMeta(moduleUrl, singleRootNode);
+    final FormatVersion version = requireValidFormatVersion(singleRootNode);
+    switch (version) {
+      case v3_0:
+        return new LegacyRemoteModuleV30(requireValidModuleSpecNode(moduleUrl, singleRootNode));
+      default:
+        throw new IllegalStateException("Unrecognized format version: " + version);
+    }
+  }
+
   private static FormatVersion requireValidFormatVersion(JsonNode root) {
     final String formatVersionStr = Selectors.textAt(root, FORMAT_VERSION);
     final FormatVersion formatVersion = FormatVersion.fromString(formatVersionStr);
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/LegacyRemoteModuleV30.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/LegacyRemoteModuleV30.java
new file mode 100644
index 0000000..529fbac
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/LegacyRemoteModuleV30.java
@@ -0,0 +1,175 @@
+/*
+ * 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.jsonmodule;
+
+import static org.apache.flink.statefun.flink.core.spi.ExtensionResolverAccessor.getExtensionResolver;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.statefun.extensions.ComponentBinder;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.common.json.Selectors;
+import org.apache.flink.statefun.flink.core.spi.ExtensionResolver;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+
+@Deprecated
+public final class LegacyRemoteModuleV30 implements StatefulFunctionModule {
+
+  private final JsonNode moduleSpecNode;
+
+  // =====================================================================
+  //  Json pointers for backwards compatibility
+  // =====================================================================
+
+  private static final JsonPointer ENDPOINTS = JsonPointer.compile("/endpoints");
+  private static final JsonPointer INGRESSES = JsonPointer.compile("/ingresses");
+  private static final JsonPointer EGRESSES = JsonPointer.compile("/egresses");
+
+  private static final JsonPointer ENDPOINT_KIND = JsonPointer.compile("/endpoint/meta/kind");
+  private static final JsonPointer ENDPOINT_SPEC = JsonPointer.compile("/endpoint/spec");
+  private static final JsonPointer INGRESS_KIND = JsonPointer.compile("/ingress/meta/type");
+  private static final JsonPointer INGRESS_ID = JsonPointer.compile("/ingress/meta/id");
+  private static final JsonPointer INGRESS_SPEC = JsonPointer.compile("/ingress/spec");
+  private static final JsonPointer EGRESS_KIND = JsonPointer.compile("/egress/meta/type");
+  private static final JsonPointer EGRESS_ID = JsonPointer.compile("/egress/meta/id");
+  private static final JsonPointer EGRESS_SPEC = JsonPointer.compile("/egress/spec");
+
+  private static final Map<String, TypeName> LEGACY_KIND_CONVERSIONS = new HashMap<>();
+
+  static {
+    LEGACY_KIND_CONVERSIONS.put("http", TypeName.parseFrom("io.statefun.endpoints.v1/http"));
+    LEGACY_KIND_CONVERSIONS.put(
+        "io.statefun.kafka/ingress", TypeName.parseFrom("io.statefun.kafka.v1/ingress"));
+    LEGACY_KIND_CONVERSIONS.put(
+        "io.statefun.kafka/egress", TypeName.parseFrom("io.statefun.kafka.v1/egress"));
+    LEGACY_KIND_CONVERSIONS.put(
+        "io.statefun.kinesis/ingress", TypeName.parseFrom("io.statefun.kinesis.v1/ingress"));
+    LEGACY_KIND_CONVERSIONS.put(
+        "io.statefun.kinesis/egress", TypeName.parseFrom("io.statefun.kinesis.v1/egress"));
+  }
+
+  LegacyRemoteModuleV30(JsonNode moduleSpecNode) {
+    this.moduleSpecNode = Objects.requireNonNull(moduleSpecNode);
+  }
+
+  @Override
+  public void configure(Map<String, String> globalConfiguration, Binder moduleBinder) {
+    components(moduleSpecNode).forEach(component -> bindComponent(component, moduleBinder));
+  }
+
+  private static Iterable<ComponentJsonObject> components(JsonNode moduleRootNode) {
+    final List<ComponentJsonObject> components = new ArrayList<>();
+    components.addAll(endpointComponents(moduleRootNode));
+    components.addAll(ingressComponents(moduleRootNode));
+    components.addAll(egressComponents(moduleRootNode));
+    return components;
+  }
+
+  private static List<ComponentJsonObject> endpointComponents(JsonNode moduleRootNode) {
+    final Iterable<? extends JsonNode> endpointComponentNodes =
+        Selectors.listAt(moduleRootNode, ENDPOINTS);
+    return StreamSupport.stream(endpointComponentNodes.spliterator(), false)
+        .map(LegacyRemoteModuleV30::parseEndpointComponentNode)
+        .collect(Collectors.toList());
+  }
+
+  private static List<ComponentJsonObject> ingressComponents(JsonNode moduleRootNode) {
+    final Iterable<? extends JsonNode> ingressComponentNodes =
+        Selectors.listAt(moduleRootNode, INGRESSES);
+    return StreamSupport.stream(ingressComponentNodes.spliterator(), false)
+        .map(LegacyRemoteModuleV30::parseIngressComponentNode)
+        .collect(Collectors.toList());
+  }
+
+  private static List<ComponentJsonObject> egressComponents(JsonNode moduleRootNode) {
+    final Iterable<? extends JsonNode> egressComponentNodes =
+        Selectors.listAt(moduleRootNode, EGRESSES);
+    return StreamSupport.stream(egressComponentNodes.spliterator(), false)
+        .map(LegacyRemoteModuleV30::parseEgressComponentNode)
+        .collect(Collectors.toList());
+  }
+
+  private static ComponentJsonObject parseEndpointComponentNode(JsonNode node) {
+    final TypeName binderKind =
+        tryConvertLegacyBinderKindTypeName(Selectors.textAt(node, ENDPOINT_KIND));
+
+    // backwards compatibility path
+    return reconstructComponentJsonObject(binderKind, node.at(ENDPOINT_SPEC));
+  }
+
+  private static ComponentJsonObject parseIngressComponentNode(JsonNode node) {
+    final TypeName binderKind =
+        tryConvertLegacyBinderKindTypeName(Selectors.textAt(node, INGRESS_KIND));
+
+    // backwards compatibility path
+    final JsonNode specNode = node.at(INGRESS_SPEC);
+    final String idString = Selectors.textAt(node, INGRESS_ID);
+    ((ObjectNode) specNode).put("id", idString);
+
+    return reconstructComponentJsonObject(binderKind, specNode);
+  }
+
+  private static ComponentJsonObject parseEgressComponentNode(JsonNode node) {
+    final TypeName binderKind =
+        tryConvertLegacyBinderKindTypeName(Selectors.textAt(node, EGRESS_KIND));
+
+    // backwards compatibility path
+    final JsonNode specNode = node.at(EGRESS_SPEC);
+    final String idString = Selectors.textAt(node, EGRESS_ID);
+    ((ObjectNode) specNode).put("id", idString);
+
+    return reconstructComponentJsonObject(binderKind, specNode);
+  }
+
+  private static TypeName tryConvertLegacyBinderKindTypeName(String binderKindString) {
+    final TypeName binderKind = LEGACY_KIND_CONVERSIONS.get(binderKindString);
+    if (binderKind != null) {
+      return binderKind;
+    }
+    // if it isn't one of the recognized legacy kinds, it could be something custom added by the
+    // user
+    return TypeName.parseFrom(binderKindString);
+  }
+
+  private static ComponentJsonObject reconstructComponentJsonObject(
+      TypeName binderTypename, JsonNode specJsonNode) {
+    final ObjectNode reconstructedNode = new ObjectMapper().createObjectNode();
+    reconstructedNode.put(
+        ComponentJsonObject.BINDER_KIND_FIELD, binderTypename.canonicalTypenameString());
+    reconstructedNode.set(ComponentJsonObject.SPEC_FIELD, specJsonNode);
+    return new ComponentJsonObject(reconstructedNode);
+  }
+
+  private static void bindComponent(ComponentJsonObject component, Binder moduleBinder) {
+    final ExtensionResolver extensionResolver = getExtensionResolver(moduleBinder);
+    final ComponentBinder componentBinder =
+        extensionResolver.resolveExtension(component.binderTypename(), ComponentBinder.class);
+    componentBinder.bind(component, moduleBinder);
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/ModuleConfigurationException.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/ModuleConfigurationException.java
index f1b0346..28bc23f 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/ModuleConfigurationException.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/ModuleConfigurationException.java
@@ -23,4 +23,8 @@
   public ModuleConfigurationException(String message, Throwable cause) {
     super(message, cause);
   }
+
+  public ModuleConfigurationException(String message) {
+    super(message);
+  }
 }
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/RemoteModule.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/RemoteModule.java
new file mode 100644
index 0000000..db1b07e
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/RemoteModule.java
@@ -0,0 +1,61 @@
+/*
+ * 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.jsonmodule;
+
+import static org.apache.flink.statefun.flink.core.spi.ExtensionResolverAccessor.getExtensionResolver;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.statefun.extensions.ComponentBinder;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.core.spi.ExtensionResolver;
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+
+public final class RemoteModule implements StatefulFunctionModule {
+
+  private final List<JsonNode> componentNodes;
+
+  RemoteModule(List<JsonNode> componentNodes) {
+    this.componentNodes = Objects.requireNonNull(componentNodes);
+  }
+
+  @Override
+  public void configure(Map<String, String> globalConfiguration, Binder moduleBinder) {
+    parseComponentNodes(componentNodes)
+        .forEach(component -> bindComponent(component, moduleBinder));
+  }
+
+  private static List<ComponentJsonObject> parseComponentNodes(
+      Iterable<? extends JsonNode> componentNodes) {
+    return StreamSupport.stream(componentNodes.spliterator(), false)
+        .map(ComponentJsonObject::new)
+        .collect(Collectors.toList());
+  }
+
+  private static void bindComponent(ComponentJsonObject component, Binder moduleBinder) {
+    final ExtensionResolver extensionResolver = getExtensionResolver(moduleBinder);
+    final ComponentBinder componentBinder =
+        extensionResolver.resolveExtension(component.binderTypename(), ComponentBinder.class);
+    componentBinder.bind(component, moduleBinder);
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/RouterJsonEntity.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/RouterJsonEntity.java
deleted file mode 100644
index 66d99a2..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/RouterJsonEntity.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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.jsonmodule;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import java.io.IOException;
-import java.net.URL;
-import java.util.Optional;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.common.ResourceLocator;
-import org.apache.flink.statefun.flink.common.json.NamespaceNamePair;
-import org.apache.flink.statefun.flink.common.json.Selectors;
-import org.apache.flink.statefun.flink.common.protobuf.ProtobufDescriptorMap;
-import org.apache.flink.statefun.flink.core.protorouter.ProtobufRouter;
-import org.apache.flink.statefun.flink.core.spi.ExtensionResolver;
-import org.apache.flink.statefun.sdk.io.IngressIdentifier;
-import org.apache.flink.statefun.sdk.io.Router;
-import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule.Binder;
-
-final class RouterJsonEntity implements JsonEntity {
-
-  private static final JsonPointer ROUTER_SPECS_POINTER = JsonPointer.compile("/routers");
-
-  private static final class MetaPointers {
-    private static final JsonPointer TYPE = JsonPointer.compile("/router/meta/type");
-  }
-
-  private static final class SpecPointers {
-    private static final JsonPointer INGRESS = JsonPointer.compile("/router/spec/ingress");
-    private static final JsonPointer TARGET = JsonPointer.compile("/router/spec/target");
-    private static final JsonPointer DESCRIPTOR = JsonPointer.compile("/router/spec/descriptorSet");
-    private static final JsonPointer MESSAGE_TYPE = JsonPointer.compile("/router/spec/messageType");
-  }
-
-  @Override
-  public void bind(
-      Binder binder,
-      ExtensionResolver extensionResolver,
-      JsonNode moduleSpecRootNode,
-      FormatVersion formatVersion) {
-    final Iterable<? extends JsonNode> routerNodes =
-        Selectors.listAt(moduleSpecRootNode, ROUTER_SPECS_POINTER);
-
-    routerNodes.forEach(
-        routerNode -> {
-          // currently the only type of router supported in a module.yaml, is a protobuf
-          // dynamicMessage
-          // router once we will introduce further router types we should refactor this to be more
-          // dynamic.
-          requireProtobufRouterType(routerNode);
-
-          binder.bindIngressRouter(targetRouterIngress(routerNode), dynamicRouter(routerNode));
-        });
-  }
-
-  // ----------------------------------------------------------------------------------------------------------
-  // Routers
-  // ----------------------------------------------------------------------------------------------------------
-
-  private static Router<Message> dynamicRouter(JsonNode router) {
-    String addressTemplate = Selectors.textAt(router, SpecPointers.TARGET);
-    String descriptorSetPath = Selectors.textAt(router, SpecPointers.DESCRIPTOR);
-    String messageType = Selectors.textAt(router, SpecPointers.MESSAGE_TYPE);
-
-    ProtobufDescriptorMap descriptorPath = protobufDescriptorMap(descriptorSetPath);
-    Optional<Descriptors.GenericDescriptor> maybeDescriptor =
-        descriptorPath.getDescriptorByName(messageType);
-    if (!maybeDescriptor.isPresent()) {
-      throw new IllegalStateException(
-          "Error while processing a router definition. Unable to locate a message "
-              + messageType
-              + " in a descriptor set "
-              + descriptorSetPath);
-    }
-    return ProtobufRouter.forAddressTemplate(
-        (Descriptors.Descriptor) maybeDescriptor.get(), addressTemplate);
-  }
-
-  private static ProtobufDescriptorMap protobufDescriptorMap(String descriptorSetPath) {
-    try {
-      URL url = ResourceLocator.findNamedResource(descriptorSetPath);
-      if (url == null) {
-        throw new IllegalArgumentException(
-            "Unable to locate a Protobuf descriptor set at " + descriptorSetPath);
-      }
-      return ProtobufDescriptorMap.from(url);
-    } catch (IOException e) {
-      throw new IllegalStateException(
-          "Error while processing a router definition. Unable to read the descriptor set at  "
-              + descriptorSetPath,
-          e);
-    }
-  }
-
-  private static IngressIdentifier<Message> targetRouterIngress(JsonNode routerNode) {
-    String targetIngress = Selectors.textAt(routerNode, SpecPointers.INGRESS);
-    NamespaceNamePair nn = NamespaceNamePair.from(targetIngress);
-    return new IngressIdentifier<>(Message.class, nn.namespace(), nn.name());
-  }
-
-  private static void requireProtobufRouterType(JsonNode routerNode) {
-    String routerType = Selectors.textAt(routerNode, MetaPointers.TYPE);
-    if (!routerType.equalsIgnoreCase("org.apache.flink.statefun.sdk/protobuf-router")) {
-      throw new IllegalStateException("Invalid router type " + routerType);
-    }
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/AddressResolver.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/AddressResolver.java
deleted file mode 100644
index c8476a7..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/AddressResolver.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.protorouter;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import java.util.List;
-import java.util.Objects;
-import org.apache.flink.statefun.flink.common.protopath.ProtobufPath;
-import org.apache.flink.statefun.sdk.Address;
-import org.apache.flink.statefun.sdk.FunctionType;
-
-final class AddressResolver {
-
-  /**
-   * Creates an address resolver that is able to produce {@link Address} from an address template
-   * and a Protocol Buffers message.
-   *
-   * <p>An address template is an address of the form function-namespace/function-type/function-id.
-   * where each component can contain multiple {@link ProtobufPath} expressions.
-   *
-   * @param messageDescriptor the protocol buffers message descriptor that would be used to extract
-   *     the target address from.
-   * @param addressTemplate the template that would be used to extract the target address by.
-   * @return an instance of an address evaluator, that is able to produce an {@link Address} given a
-   *     Protocol Buffers message.
-   */
-  static AddressResolver fromAddressTemplate(
-      Descriptors.Descriptor messageDescriptor, String addressTemplate) {
-    Objects.requireNonNull(messageDescriptor);
-    Objects.requireNonNull(addressTemplate);
-
-    int lastSlash = addressTemplate.lastIndexOf("/");
-    if (lastSlash <= 0) {
-      throw new IllegalArgumentException(
-          "The address template is not of the form <function type>/<id>");
-    }
-    String functionTypeTemplate = addressTemplate.substring(0, lastSlash);
-    String idTemplate = addressTemplate.substring(lastSlash + 1);
-    if (idTemplate.isEmpty()) {
-      throw new IllegalArgumentException(
-          "The address template is not of the form <function type>/<id>");
-    }
-    lastSlash = functionTypeTemplate.lastIndexOf("/");
-    if (lastSlash <= 0) {
-      throw new IllegalArgumentException(
-          "The function type template is not of the form <function namespace>/<function name>");
-    }
-    String functionNamespaceTemplate = functionTypeTemplate.substring(0, lastSlash);
-    String functionNameIdTemplate = functionTypeTemplate.substring(lastSlash + 1);
-    if (functionNameIdTemplate.isEmpty()) {
-      throw new IllegalArgumentException(
-          "The address template is not of the form <function type>/<id>");
-    }
-    return new AddressResolver(
-        evaluator(messageDescriptor, functionNamespaceTemplate),
-        evaluator(messageDescriptor, functionNameIdTemplate),
-        evaluator(messageDescriptor, idTemplate));
-  }
-
-  static TemplateEvaluator evaluator(Descriptors.Descriptor descriptor, String template) {
-    List<TemplateParser.TextFragment> fragments = TemplateParser.parseTemplateString(template);
-    return new TemplateEvaluator(descriptor, fragments);
-  }
-
-  private final TemplateEvaluator functionNamespace;
-  private final TemplateEvaluator functionName;
-  private final TemplateEvaluator functionId;
-
-  private AddressResolver(
-      TemplateEvaluator functionNamespace,
-      TemplateEvaluator functionName,
-      TemplateEvaluator functionId) {
-    this.functionNamespace = Objects.requireNonNull(functionNamespace);
-    this.functionName = Objects.requireNonNull(functionName);
-    this.functionId = Objects.requireNonNull(functionId);
-  }
-
-  Address evaluate(Message message) {
-    FunctionType functionType =
-        new FunctionType(functionNamespace.evaluate(message), functionName.evaluate(message));
-
-    return new Address(functionType, functionId.evaluate(message));
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/ProtobufRouter.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/ProtobufRouter.java
deleted file mode 100644
index 970836e..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/ProtobufRouter.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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.protorouter;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.DynamicMessage;
-import com.google.protobuf.Message;
-import java.util.Objects;
-import org.apache.flink.statefun.flink.common.protopath.ProtobufPath;
-import org.apache.flink.statefun.sdk.Address;
-import org.apache.flink.statefun.sdk.io.Router;
-
-/**
- * Routes {@code Protocol Buffers} {@link DynamicMessage}s based on an address template string.
- *
- * <p>Route messages downstream to an address determined by an address template of the form
- * target-function-namespace/target-function-type/target-function-id. Each address template
- * component can reference a section of the input {@link DynamicMessage} by using {@link
- * ProtobufPath} expressions.
- *
- * <p>For example consider the following {@code Protocol Buffers} message type
- *
- * <pre>{@code
- * message MyInput {
- *     string name = 1;
- * }
- * }</pre>
- *
- * And an instance of the message {@code { "name" : "bob" }}, and the following template string:
- * "org.apache.flink/python-function/{{$.name}}".
- *
- * <p>This message would be routed to the address: {@code Address(FunctionType(org.apache.flink,
- * python-function), bob)}.
- */
-public final class ProtobufRouter implements Router<Message> {
-
-  public static ProtobufRouter forAddressTemplate(
-      Descriptors.Descriptor descriptor, String addressTemplate) {
-    AddressResolver evaluator = AddressResolver.fromAddressTemplate(descriptor, addressTemplate);
-    return new ProtobufRouter(evaluator);
-  }
-
-  private final AddressResolver addressResolver;
-
-  private ProtobufRouter(AddressResolver addressResolver) {
-    this.addressResolver = Objects.requireNonNull(addressResolver);
-  }
-
-  @Override
-  public void route(Message message, Downstream<Message> downstream) {
-    Address targetAddress = addressResolver.evaluate(message);
-    downstream.forward(targetAddress, message);
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/TemplateEvaluator.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/TemplateEvaluator.java
deleted file mode 100644
index 5f7b05a..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/TemplateEvaluator.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.protorouter;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import java.util.List;
-import java.util.function.Function;
-import org.apache.flink.statefun.flink.common.protopath.ProtobufPath;
-
-final class TemplateEvaluator {
-
-  private interface FragmentEvaluator {
-    void eval(StringBuilder builder, Message message);
-  }
-
-  private final FragmentEvaluator[] fragmentEvaluators;
-  private final StringBuilder builder = new StringBuilder();
-
-  TemplateEvaluator(
-      Descriptors.Descriptor descriptor, List<TemplateParser.TextFragment> fragments) {
-    this.fragmentEvaluators = fragmentEvaluators(descriptor, fragments);
-  }
-
-  public String evaluate(Message message) {
-    for (FragmentEvaluator e : fragmentEvaluators) {
-      e.eval(builder, message);
-    }
-    final String result = builder.toString();
-    builder.delete(0, builder.length());
-    return result;
-  }
-
-  private static FragmentEvaluator[] fragmentEvaluators(
-      Descriptors.Descriptor descriptor, List<TemplateParser.TextFragment> fragments) {
-    return fragments.stream()
-        .map(
-            fragment ->
-                fragment.dynamic()
-                    ? dynamicEvaluator(descriptor, fragment)
-                    : staticEvaluator(fragment))
-        .toArray(FragmentEvaluator[]::new);
-  }
-
-  private static FragmentEvaluator staticEvaluator(TemplateParser.TextFragment fragment) {
-    final String text = fragment.fragment();
-    return (builder, unused) -> builder.append(text);
-  }
-
-  private static FragmentEvaluator dynamicEvaluator(
-      Descriptors.Descriptor descriptor, TemplateParser.TextFragment fragment) {
-    final Function<Message, ?> protopathEvaluator =
-        ProtobufPath.protobufPath(descriptor, fragment.fragment());
-    return (builder, message) -> {
-      Object result = protopathEvaluator.apply(message);
-      builder.append(result);
-    };
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/TemplateParser.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/TemplateParser.java
deleted file mode 100644
index d5f30da..0000000
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/TemplateParser.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * 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.protorouter;
-
-import static org.apache.flink.statefun.flink.core.protorouter.TemplateParser.TextFragment.dynamicFragment;
-import static org.apache.flink.statefun.flink.core.protorouter.TemplateParser.TextFragment.staticFragment;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-final class TemplateParser {
-  private static final Pattern DYNAMIC_FRAGMENT_PATTERN = Pattern.compile("\\{\\{([^}]*)}}");
-
-  private TemplateParser() {}
-
-  static List<TextFragment> parseTemplateString(String template) {
-    ArrayList<TextFragment> fragments = new ArrayList<>();
-    Matcher fragmentMatcher = DYNAMIC_FRAGMENT_PATTERN.matcher(template);
-    int position = 0;
-    while (position < template.length()) {
-      if (!fragmentMatcher.find(position)) {
-        // no more dynamic parts to the pattern. We take whatever we have left from position to the
-        // end
-        // as a static pattern, and terminate the loop.
-        String prefix = template.substring(position);
-        fragments.add(staticFragment(prefix));
-        break;
-      }
-      // A dynamic text fragment has been found. It is of the form: X{{Y}}
-      // where X is a static prefix that spans from (position, matchStart - 2)
-      // and Y is the dynamic part that spans from (dynamicStart, dynamicEnd)
-      final int prefixStart = position;
-      final int prefixEnd = fragmentMatcher.start(1) - "{{".length();
-      final int dynamicStart = fragmentMatcher.start(1);
-      final int dynamicEnd = fragmentMatcher.end(1);
-      if (prefixEnd - prefixStart > 0) {
-        // we have a static prefix
-        String prefixText = template.substring(prefixStart, prefixEnd);
-        fragments.add(staticFragment(prefixText));
-      }
-      String dynamicFragmentText = template.substring(dynamicStart, dynamicEnd);
-      fragments.add(dynamicFragment(dynamicFragmentText));
-      position = dynamicEnd + "}}".length();
-    }
-    return fragments;
-  }
-
-  public static final class TextFragment {
-    private final String fragment;
-    private final boolean dynamic;
-
-    public static TextFragment dynamicFragment(String text) {
-      return new TextFragment(text, true);
-    }
-
-    public static TextFragment staticFragment(String text) {
-      return new TextFragment(text, false);
-    }
-
-    private TextFragment(String fragment, boolean dynamic) {
-      this.fragment = Objects.requireNonNull(fragment);
-      this.dynamic = dynamic;
-    }
-
-    public String fragment() {
-      return fragment;
-    }
-
-    public boolean dynamic() {
-      return dynamic;
-    }
-
-    @Override
-    public String toString() {
-      return "Fragment{" + "text='" + fragment + '\'' + ", dynamic=" + dynamic + '}';
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) return true;
-      if (o == null || getClass() != o.getClass()) return false;
-      TextFragment fragment = (TextFragment) o;
-      return dynamic == fragment.dynamic && this.fragment.equals(fragment.fragment);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(fragment, dynamic);
-    }
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/spi/ExtensionResolver.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/spi/ExtensionResolver.java
index c7a44f9..5b06b3f 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/spi/ExtensionResolver.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/spi/ExtensionResolver.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.statefun.flink.core.spi;
 
+import org.apache.flink.statefun.extensions.ExtensionModule;
 import org.apache.flink.statefun.sdk.TypeName;
-import org.apache.flink.statefun.sdk.spi.ExtensionModule;
 
 /**
  * Resolves a bound extension (bound by {@link ExtensionModule}s) given specified {@link TypeName}s.
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/spi/ExtensionResolverAccessor.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/spi/ExtensionResolverAccessor.java
new file mode 100644
index 0000000..de1e220
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/spi/ExtensionResolverAccessor.java
@@ -0,0 +1,34 @@
+/*
+ * 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.spi;
+
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+
+/**
+ * TODO This is a temporary workaround for accessing the {@link ExtensionResolver}. TODO We should
+ * expose the resolver properly once we have more usages.
+ */
+public final class ExtensionResolverAccessor {
+  private ExtensionResolverAccessor() {}
+
+  public static ExtensionResolver getExtensionResolver(StatefulFunctionModule.Binder moduleBinder) {
+    // the binder is always the StatefulFunctionsUniverse, which implements ExtensionResolver
+    return (ExtensionResolver) moduleBinder;
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/spi/Modules.java b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/spi/Modules.java
index 26bd524..bcf937d 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/spi/Modules.java
+++ b/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/spi/Modules.java
@@ -18,13 +18,13 @@
 package org.apache.flink.statefun.flink.core.spi;
 
 import java.util.*;
+import org.apache.flink.statefun.extensions.ExtensionModule;
 import org.apache.flink.statefun.flink.common.SetContextClassLoader;
 import org.apache.flink.statefun.flink.core.StatefulFunctionsConfig;
 import org.apache.flink.statefun.flink.core.StatefulFunctionsUniverse;
 import org.apache.flink.statefun.flink.core.jsonmodule.JsonServiceLoader;
 import org.apache.flink.statefun.flink.core.message.MessageFactoryKey;
 import org.apache.flink.statefun.flink.io.spi.FlinkIoModule;
-import org.apache.flink.statefun.sdk.spi.ExtensionModule;
 import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
 
 public final class Modules {
diff --git a/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/httpfn/binders/v1/HttpEndpointBinderV1Test.java b/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/httpfn/binders/v1/HttpEndpointBinderV1Test.java
new file mode 100644
index 0000000..c149b86
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/httpfn/binders/v1/HttpEndpointBinderV1Test.java
@@ -0,0 +1,66 @@
+/*
+ * 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.httpfn.binders.v1;
+
+import static org.hamcrest.Matchers.hasKey;
+import static org.junit.Assert.assertThat;
+
+import java.net.URL;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.core.StatefulFunctionsUniverse;
+import org.apache.flink.statefun.flink.core.httpfn.DefaultHttpRequestReplyClientFactory;
+import org.apache.flink.statefun.flink.core.httpfn.TransportClientConstants;
+import org.apache.flink.statefun.flink.core.message.MessageFactoryKey;
+import org.apache.flink.statefun.flink.core.message.MessageFactoryType;
+import org.junit.Test;
+
+public final class HttpEndpointBinderV1Test {
+  private static final ObjectMapper OBJ_MAPPER = new ObjectMapper(new YAMLFactory());
+
+  private static final String SPEC_YAML_PATH = "http-endpoint-binders/v1.yaml";
+
+  @Test
+  public void exampleUsage() throws Exception {
+    final ComponentJsonObject component = loadComponentJsonObject(SPEC_YAML_PATH);
+    final StatefulFunctionsUniverse universe = testUniverse();
+
+    HttpEndpointBinderV1.INSTANCE.bind(component, universe);
+
+    assertThat(universe.namespaceFunctions(), hasKey("com.foo.bar"));
+  }
+
+  private static ComponentJsonObject loadComponentJsonObject(String yamlPath) throws Exception {
+    final URL url = HttpEndpointBinderV1Test.class.getClassLoader().getResource(yamlPath);
+    final ObjectNode componentObject = OBJ_MAPPER.readValue(url, ObjectNode.class);
+    return new ComponentJsonObject(componentObject);
+  }
+
+  private static StatefulFunctionsUniverse testUniverse() {
+    final StatefulFunctionsUniverse universe =
+        new StatefulFunctionsUniverse(
+            MessageFactoryKey.forType(MessageFactoryType.WITH_PROTOBUF_PAYLOADS, null));
+    universe.bindExtension(
+        TransportClientConstants.OKHTTP_CLIENT_FACTORY_TYPE,
+        DefaultHttpRequestReplyClientFactory.INSTANCE);
+    return universe;
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/httpfn/binders/v2/HttpEndpointBinderV2Test.java b/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/httpfn/binders/v2/HttpEndpointBinderV2Test.java
new file mode 100644
index 0000000..f41be1f
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/httpfn/binders/v2/HttpEndpointBinderV2Test.java
@@ -0,0 +1,66 @@
+/*
+ * 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.httpfn.binders.v2;
+
+import static org.hamcrest.Matchers.hasKey;
+import static org.junit.Assert.assertThat;
+
+import java.net.URL;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.core.StatefulFunctionsUniverse;
+import org.apache.flink.statefun.flink.core.httpfn.DefaultHttpRequestReplyClientFactory;
+import org.apache.flink.statefun.flink.core.httpfn.TransportClientConstants;
+import org.apache.flink.statefun.flink.core.message.MessageFactoryKey;
+import org.apache.flink.statefun.flink.core.message.MessageFactoryType;
+import org.junit.Test;
+
+public final class HttpEndpointBinderV2Test {
+  private static final ObjectMapper OBJ_MAPPER = new ObjectMapper(new YAMLFactory());
+
+  private static final String SPEC_YAML_PATH = "http-endpoint-binders/v2.yaml";
+
+  @Test
+  public void exampleUsage() throws Exception {
+    final ComponentJsonObject component = loadComponentJsonObject(SPEC_YAML_PATH);
+    final StatefulFunctionsUniverse universe = testUniverse();
+
+    HttpEndpointBinderV2.INSTANCE.bind(component, universe);
+
+    assertThat(universe.namespaceFunctions(), hasKey("com.foo.bar"));
+  }
+
+  private static ComponentJsonObject loadComponentJsonObject(String yamlPath) throws Exception {
+    final URL url = HttpEndpointBinderV2Test.class.getClassLoader().getResource(yamlPath);
+    final ObjectNode componentObject = OBJ_MAPPER.readValue(url, ObjectNode.class);
+    return new ComponentJsonObject(componentObject);
+  }
+
+  private static StatefulFunctionsUniverse testUniverse() {
+    final StatefulFunctionsUniverse universe =
+        new StatefulFunctionsUniverse(
+            MessageFactoryKey.forType(MessageFactoryType.WITH_PROTOBUF_PAYLOADS, null));
+    universe.bindExtension(
+        TransportClientConstants.OKHTTP_CLIENT_FACTORY_TYPE,
+        DefaultHttpRequestReplyClientFactory.INSTANCE);
+    return universe;
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/jsonmodule/JsonModuleTest.java b/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/jsonmodule/JsonModuleTest.java
deleted file mode 100644
index cba075f..0000000
--- a/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/jsonmodule/JsonModuleTest.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * 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.jsonmodule;
-
-import static org.hamcrest.Matchers.allOf;
-import static org.hamcrest.Matchers.hasKey;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.notNullValue;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-
-import com.google.protobuf.Message;
-import java.net.URI;
-import java.net.URL;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.flink.statefun.flink.core.StatefulFunctionsUniverse;
-import org.apache.flink.statefun.flink.core.message.MessageFactoryKey;
-import org.apache.flink.statefun.flink.core.message.MessageFactoryType;
-import org.apache.flink.statefun.flink.core.reqreply.RequestReplyClient;
-import org.apache.flink.statefun.flink.core.reqreply.RequestReplyClientFactory;
-import org.apache.flink.statefun.sdk.FunctionType;
-import org.apache.flink.statefun.sdk.TypeName;
-import org.apache.flink.statefun.sdk.io.EgressIdentifier;
-import org.apache.flink.statefun.sdk.io.IngressIdentifier;
-import org.apache.flink.statefun.sdk.reqreply.generated.TypedValue;
-import org.apache.flink.statefun.sdk.spi.ExtensionModule;
-import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class JsonModuleTest {
-
-  private final String modulePath;
-
-  @Parameterized.Parameters(name = "{0}")
-  public static Collection<String> modulePaths() {
-    return Arrays.asList("module-v3_0/module.yaml", "module-v3_1/module.yaml");
-  }
-
-  public JsonModuleTest(String modulePath) {
-    this.modulePath = modulePath;
-  }
-
-  @Test
-  public void exampleUsage() {
-    StatefulFunctionModule module = fromPath(modulePath);
-
-    assertThat(module, notNullValue());
-  }
-
-  @Test
-  public void testFunctions() {
-    StatefulFunctionModule module = fromPath(modulePath);
-    ExtensionModule extensionModule =
-        transportClientExtensions(TypeName.parseFrom("my.custom/http.transport.type"));
-
-    StatefulFunctionsUniverse universe = emptyUniverse();
-    setupUniverse(universe, module, extensionModule);
-
-    assertThat(
-        universe.functions(),
-        allOf(
-            hasKey(new FunctionType("com.foo.bar", "specific_function")),
-            hasKey(new FunctionType("com.other.namespace", "hello"))));
-
-    assertThat(universe.namespaceFunctions(), hasKey("com.foo.bar"));
-  }
-
-  @Test
-  public void testRouters() {
-    StatefulFunctionModule module = fromPath(modulePath);
-    ExtensionModule extensionModule =
-        transportClientExtensions(TypeName.parseFrom("my.custom/http.transport.type"));
-
-    StatefulFunctionsUniverse universe = emptyUniverse();
-    setupUniverse(universe, module, extensionModule);
-
-    assertThat(
-        universe.routers(),
-        hasKey(new IngressIdentifier<>(Message.class, "com.mycomp.igal", "names")));
-  }
-
-  @Test
-  public void testIngresses() {
-    StatefulFunctionModule module = fromPath(modulePath);
-    ExtensionModule extensionModule =
-        transportClientExtensions(TypeName.parseFrom("my.custom/http.transport.type"));
-
-    StatefulFunctionsUniverse universe = emptyUniverse();
-    setupUniverse(universe, module, extensionModule);
-
-    assertThat(
-        universe.ingress(),
-        hasKey(new IngressIdentifier<>(Message.class, "com.mycomp.igal", "names")));
-  }
-
-  @Test
-  public void testEgresses() {
-    StatefulFunctionModule module = fromPath(modulePath);
-    ExtensionModule extensionModule =
-        transportClientExtensions(TypeName.parseFrom("my.custom/http.transport.type"));
-
-    StatefulFunctionsUniverse universe = emptyUniverse();
-    setupUniverse(universe, module, extensionModule);
-
-    assertThat(
-        universe.egress(),
-        hasKey(new EgressIdentifier<>("com.mycomp.foo", "bar", TypedValue.class)));
-  }
-
-  private static StatefulFunctionModule fromPath(String path) {
-    URL moduleUrl = JsonModuleTest.class.getClassLoader().getResource(path);
-    assertThat(moduleUrl, not(nullValue()));
-    ObjectMapper mapper = JsonServiceLoader.mapper();
-    return JsonServiceLoader.fromUrl(mapper, moduleUrl);
-  }
-
-  private static ExtensionModule transportClientExtensions(TypeName type) {
-    return new TransportClientBindingModule(type);
-  }
-
-  private static StatefulFunctionsUniverse emptyUniverse() {
-    return new StatefulFunctionsUniverse(
-        MessageFactoryKey.forType(MessageFactoryType.WITH_PROTOBUF_PAYLOADS, null));
-  }
-
-  private static void setupUniverse(
-      StatefulFunctionsUniverse universe,
-      StatefulFunctionModule functionModule,
-      ExtensionModule extensionModule) {
-    final Map<String, String> globalConfig = new HashMap<>();
-    extensionModule.configure(globalConfig, universe);
-    functionModule.configure(globalConfig, universe);
-  }
-
-  private static class TransportClientBindingModule implements ExtensionModule {
-
-    private final TypeName transportClientType;
-
-    TransportClientBindingModule(TypeName transportClientType) {
-      this.transportClientType = transportClientType;
-    }
-
-    @Override
-    public void configure(Map<String, String> globalConfigurations, Binder binder) {
-      binder.bindExtension(transportClientType, new TestRequestReplyClientFactory());
-    }
-  }
-
-  private static class TestRequestReplyClientFactory implements RequestReplyClientFactory {
-    @Override
-    public RequestReplyClient createTransportClient(
-        ObjectNode transportProperties, URI endpointUrl) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void cleanup() {
-      throw new UnsupportedOperationException();
-    }
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/jsonmodule/RemoteModuleTest.java b/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/jsonmodule/RemoteModuleTest.java
new file mode 100644
index 0000000..fa3efa1
--- /dev/null
+++ b/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/jsonmodule/RemoteModuleTest.java
@@ -0,0 +1,168 @@
+/*
+ * 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.jsonmodule;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.hasKey;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.hamcrest.Matchers.nullValue;
+
+import java.net.URL;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.statefun.extensions.ComponentBinder;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.extensions.ExtensionModule;
+import org.apache.flink.statefun.flink.core.StatefulFunctionsUniverse;
+import org.apache.flink.statefun.flink.core.message.MessageFactoryKey;
+import org.apache.flink.statefun.flink.core.message.MessageFactoryType;
+import org.apache.flink.statefun.sdk.EgressType;
+import org.apache.flink.statefun.sdk.FunctionType;
+import org.apache.flink.statefun.sdk.IngressType;
+import org.apache.flink.statefun.sdk.StatefulFunction;
+import org.apache.flink.statefun.sdk.StatefulFunctionProvider;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.statefun.sdk.io.EgressIdentifier;
+import org.apache.flink.statefun.sdk.io.EgressSpec;
+import org.apache.flink.statefun.sdk.io.IngressIdentifier;
+import org.apache.flink.statefun.sdk.io.IngressSpec;
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+import org.junit.Test;
+
+public final class RemoteModuleTest {
+
+  private final String modulePath = "remote-module/module.yaml";
+
+  @Test
+  public void exampleUsage() {
+    StatefulFunctionModule module = fromPath(modulePath);
+
+    assertThat(module, notNullValue());
+  }
+
+  @Test
+  public void testComponents() {
+    StatefulFunctionModule module = fromPath(modulePath);
+
+    StatefulFunctionsUniverse universe = emptyUniverse();
+    setupUniverse(universe, module, new TestComponentBindersModule());
+
+    assertThat(universe.functions(), hasKey(TestComponentBinder1.TEST_FUNCTION_TYPE));
+    assertThat(universe.ingress(), hasKey(TestComponentBinder2.TEST_INGRESS.id()));
+    assertThat(universe.egress(), hasKey(TestComponentBinder3.TEST_EGRESS.id()));
+  }
+
+  private static StatefulFunctionModule fromPath(String path) {
+    URL moduleUrl = RemoteModuleTest.class.getClassLoader().getResource(path);
+    assertThat(moduleUrl, not(nullValue()));
+    ObjectMapper mapper = JsonServiceLoader.mapper();
+    return JsonServiceLoader.fromUrl(mapper, moduleUrl);
+  }
+
+  private static StatefulFunctionsUniverse emptyUniverse() {
+    return new StatefulFunctionsUniverse(
+        MessageFactoryKey.forType(MessageFactoryType.WITH_PROTOBUF_PAYLOADS, null));
+  }
+
+  private static void setupUniverse(
+      StatefulFunctionsUniverse universe,
+      StatefulFunctionModule functionModule,
+      ExtensionModule extensionModule) {
+    final Map<String, String> globalConfig = new HashMap<>();
+    extensionModule.configure(globalConfig, universe);
+    functionModule.configure(globalConfig, universe);
+  }
+
+  private static class TestComponentBindersModule implements ExtensionModule {
+    @Override
+    public void configure(Map<String, String> globalConfigurations, Binder binder) {
+      binder.bindExtension(
+          TypeName.parseFrom("com.foo.bar/test.component.1"), new TestComponentBinder1());
+      binder.bindExtension(
+          TypeName.parseFrom("com.foo.bar/test.component.2"), new TestComponentBinder2());
+      binder.bindExtension(
+          TypeName.parseFrom("com.foo.bar/test.component.3"), new TestComponentBinder3());
+    }
+  }
+
+  private static class TestComponentBinder1 implements ComponentBinder {
+
+    private static final FunctionType TEST_FUNCTION_TYPE =
+        new FunctionType("test", "function.type");
+
+    @Override
+    public void bind(
+        ComponentJsonObject component, StatefulFunctionModule.Binder remoteModuleBinder) {
+      remoteModuleBinder.bindFunctionProvider(TEST_FUNCTION_TYPE, new TestFunctionProvider());
+    }
+  }
+
+  private static class TestComponentBinder2 implements ComponentBinder {
+    private static final TestIngressSpec TEST_INGRESS = new TestIngressSpec();
+
+    @Override
+    public void bind(
+        ComponentJsonObject component, StatefulFunctionModule.Binder remoteModuleBinder) {
+      remoteModuleBinder.bindIngress(TEST_INGRESS);
+    }
+  }
+
+  private static class TestComponentBinder3 implements ComponentBinder {
+    private static final TestEgressSpec TEST_EGRESS = new TestEgressSpec();
+
+    @Override
+    public void bind(
+        ComponentJsonObject component, StatefulFunctionModule.Binder remoteModuleBinder) {
+      remoteModuleBinder.bindEgress(TEST_EGRESS);
+    }
+  }
+
+  private static class TestFunctionProvider implements StatefulFunctionProvider {
+    @Override
+    public StatefulFunction functionOfType(FunctionType type) {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  private static class TestIngressSpec implements IngressSpec<String> {
+    @Override
+    public IngressIdentifier<String> id() {
+      return new IngressIdentifier<>(String.class, "test-namespace", "test-ingress");
+    }
+
+    @Override
+    public IngressType type() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  private static class TestEgressSpec implements EgressSpec<String> {
+    @Override
+    public EgressIdentifier<String> id() {
+      return new EgressIdentifier<>("test-namespace", "test-egress", String.class);
+    }
+
+    @Override
+    public EgressType type() {
+      throw new UnsupportedOperationException();
+    }
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/protorouter/AddressResolverTest.java b/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/protorouter/AddressResolverTest.java
deleted file mode 100644
index 2fa0713..0000000
--- a/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/protorouter/AddressResolverTest.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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.protorouter;
-
-import static org.apache.flink.statefun.flink.core.protorouter.AddressResolver.fromAddressTemplate;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-
-import com.google.protobuf.DynamicMessage;
-import com.google.protobuf.InvalidProtocolBufferException;
-import com.google.protobuf.Message;
-import org.apache.flink.statefun.flink.core.protorouter.generated.TestProtos.SimpleMessage;
-import org.apache.flink.statefun.sdk.Address;
-import org.apache.flink.statefun.sdk.FunctionType;
-import org.junit.Test;
-
-public class AddressResolverTest {
-
-  @Test
-  public void exampleUsage() {
-    Message originalMessage = SimpleMessage.newBuilder().setName("bob").build();
-    DynamicMessage message = dynamic(originalMessage);
-
-    AddressResolver addressResolver =
-        fromAddressTemplate(
-            originalMessage.getDescriptorForType(), "org.apache.flink/python-function/{{$.name}}");
-
-    assertThat(
-        addressResolver.evaluate(message),
-        is(address("org.apache.flink", "python-function", "bob")));
-  }
-
-  @Test
-  public void multipleReplacements() {
-    Message originalMessage = SimpleMessage.newBuilder().setName("bob").build();
-    DynamicMessage message = dynamic(originalMessage);
-
-    AddressResolver addressResolver =
-        fromAddressTemplate(
-            originalMessage.getDescriptorForType(), "com.{{$.name}}/python-{{$.name}}/{{$.name}}");
-
-    assertThat(addressResolver.evaluate(message), is(address("com.bob", "python-bob", "bob")));
-  }
-
-  @Test
-  public void namespaceWithMultipleSlashes() {
-    Message originalMessage = SimpleMessage.newBuilder().setName("cat").build();
-    DynamicMessage message = dynamic(originalMessage);
-
-    AddressResolver addressResolver =
-        fromAddressTemplate(
-            originalMessage.getDescriptorForType(), "a/b/c/apache/python-function/{{$.name}}");
-
-    assertThat(
-        addressResolver.evaluate(message), is(address("a/b/c/apache", "python-function", "cat")));
-  }
-
-  private static Address address(String ns, String type, String id) {
-    return new Address(new FunctionType(ns, type), id);
-  }
-
-  private static DynamicMessage dynamic(Message message) {
-    try {
-      return DynamicMessage.parseFrom(message.getDescriptorForType(), message.toByteString());
-    } catch (InvalidProtocolBufferException e) {
-      throw new AssertionError(e);
-    }
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/protorouter/TemplateEvaluatorTest.java b/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/protorouter/TemplateEvaluatorTest.java
deleted file mode 100644
index 538d0ec..0000000
--- a/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/protorouter/TemplateEvaluatorTest.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.protorouter;
-
-import static org.apache.flink.statefun.flink.core.protorouter.TemplateParser.TextFragment.dynamicFragment;
-import static org.apache.flink.statefun.flink.core.protorouter.TemplateParser.TextFragment.staticFragment;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-
-import com.google.protobuf.DynamicMessage;
-import com.google.protobuf.InvalidProtocolBufferException;
-import com.google.protobuf.Message;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.flink.statefun.flink.core.protorouter.generated.TestProtos.SimpleMessage;
-import org.junit.Test;
-
-public class TemplateEvaluatorTest {
-
-  @Test
-  public void exampleUsage() {
-    Message originalMessage = SimpleMessage.newBuilder().setName("bob").build();
-    DynamicMessage message = dynamic(originalMessage);
-
-    TemplateEvaluator evaluator =
-        new TemplateEvaluator(
-            originalMessage.getDescriptorForType(),
-            fragments(staticFragment("foo.bar/"), dynamicFragment("$.name")));
-
-    assertThat(evaluator.evaluate(message), is("foo.bar/bob"));
-  }
-
-  private static List<TemplateParser.TextFragment> fragments(
-      TemplateParser.TextFragment... fragments) {
-    return Arrays.asList(fragments);
-  }
-
-  private static DynamicMessage dynamic(Message message) {
-    try {
-      return DynamicMessage.parseFrom(message.getDescriptorForType(), message.toByteString());
-    } catch (InvalidProtocolBufferException e) {
-      throw new AssertionError(e);
-    }
-  }
-}
diff --git a/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/protorouter/TemplateParserTest.java b/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/protorouter/TemplateParserTest.java
deleted file mode 100644
index 2b3bd29..0000000
--- a/statefun-flink/statefun-flink-core/src/test/java/org/apache/flink/statefun/flink/core/protorouter/TemplateParserTest.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * 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.protorouter;
-
-import static org.apache.flink.statefun.flink.core.protorouter.TemplateParser.TextFragment.staticFragment;
-import static org.hamcrest.Matchers.contains;
-import static org.junit.Assert.assertThat;
-
-import java.util.List;
-import org.apache.flink.statefun.flink.core.protorouter.TemplateParser.TextFragment;
-import org.junit.Test;
-
-public class TemplateParserTest {
-
-  @Test
-  public void exampleUsage() {
-    List<TextFragment> fragments = TemplateParser.parseTemplateString("hello-{{world}}");
-
-    assertThat(
-        fragments, contains(staticFragment("hello-"), TextFragment.dynamicFragment("world")));
-  }
-
-  @Test
-  public void anotherExample() {
-    List<TextFragment> fragments =
-        TemplateParser.parseTemplateString("io.example/greet-python/{{$.who[0].what[5].name}}");
-
-    assertThat(
-        fragments,
-        contains(
-            staticFragment("io.example/greet-python/"),
-            TextFragment.dynamicFragment("$.who[0].what[5].name")));
-  }
-
-  @Test
-  public void longDynamicText() {
-    List<TextFragment> fragments =
-        TemplateParser.parseTemplateString("{{this text should be dynamic}}");
-
-    assertThat(fragments, contains(TextFragment.dynamicFragment("this text should be dynamic")));
-  }
-
-  @Test
-  public void twoDynamicFragmentsWithSeparator() {
-    List<TextFragment> fragments = TemplateParser.parseTemplateString("{{hello}}/{{world}}");
-
-    assertThat(
-        fragments,
-        contains(
-            TextFragment.dynamicFragment("hello"),
-            staticFragment("/"),
-            TextFragment.dynamicFragment("world")));
-  }
-
-  @Test
-  public void twoDynamicFragmentsWithoutSeparator() {
-    List<TextFragment> fragments = TemplateParser.parseTemplateString("{{hello}}{{world}}");
-
-    assertThat(
-        fragments,
-        contains(TextFragment.dynamicFragment("hello"), TextFragment.dynamicFragment("world")));
-  }
-
-  @Test
-  public void noDynamicText() {
-    List<TextFragment> fragments = TemplateParser.parseTemplateString("hello world");
-
-    assertThat(fragments, contains(staticFragment("hello world")));
-  }
-
-  @Test
-  public void dynamicFragmentToTheLeft() {
-    List<TextFragment> fragments = TemplateParser.parseTemplateString("{{hello}}-world");
-
-    assertThat(
-        fragments, contains(TextFragment.dynamicFragment("hello"), staticFragment("-world")));
-  }
-
-  @Test
-  public void unclosedInterpolatedTextConsideredASticFragment() {
-    assertThat(TemplateParser.parseTemplateString("{{"), contains(staticFragment("{{")));
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml b/statefun-flink/statefun-flink-core/src/test/resources/http-endpoint-binders/v1.yaml
similarity index 74%
copy from statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml
copy to statefun-flink/statefun-flink-core/src/test/resources/http-endpoint-binders/v1.yaml
index 2014cb3..a888ac0 100644
--- a/statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml
+++ b/statefun-flink/statefun-flink-core/src/test/resources/http-endpoint-binders/v1.yaml
@@ -13,14 +13,13 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-egress:
-  meta:
-    type: io.statefun.kafka/egress
-    id: com.mycomp.foo/bar
-  spec:
-    address: kafka-broker:9092
-    deliverySemantic:
-      type: exactly-once
-      transactionTimeout: 15min
-    properties:
-      - foo.config: bar
+kind: io.statefun.endpoints.v1/http
+spec:
+  functions: com.foo.bar/*
+  urlPathTemplate: http://bar.foo.com:8080/functions/{function.name}
+  maxNumBatchRequests: 10000
+  timeouts:
+    call: 1minute
+    connect: 30seconds
+    read: 20seconds
+    write: 10seconds
\ No newline at end of file
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml b/statefun-flink/statefun-flink-core/src/test/resources/http-endpoint-binders/v2.yaml
similarity index 70%
copy from statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml
copy to statefun-flink/statefun-flink-core/src/test/resources/http-endpoint-binders/v2.yaml
index 2014cb3..3243a5b 100644
--- a/statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml
+++ b/statefun-flink/statefun-flink-core/src/test/resources/http-endpoint-binders/v2.yaml
@@ -13,14 +13,15 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-egress:
-  meta:
-    type: io.statefun.kafka/egress
-    id: com.mycomp.foo/bar
-  spec:
-    address: kafka-broker:9092
-    deliverySemantic:
-      type: exactly-once
-      transactionTimeout: 15min
-    properties:
-      - foo.config: bar
+kind: io.statefun.endpoints.v2/http
+spec:
+  functions: com.foo.bar/*
+  urlPathTemplate: http://bar.foo.com:8080/functions/{function.name}
+  maxNumBatchRequests: 10000
+  transport:
+    type: io.statefun.transports/okhttp
+    timeouts:
+      call: 1minute
+      connect: 30seconds
+      read: 20seconds
+      write: 10seconds
\ No newline at end of file
diff --git a/statefun-flink/statefun-flink-core/src/test/resources/module-v1_0/module.yaml b/statefun-flink/statefun-flink-core/src/test/resources/module-v1_0/module.yaml
deleted file mode 100644
index dfba425..0000000
--- a/statefun-flink/statefun-flink-core/src/test/resources/module-v1_0/module.yaml
+++ /dev/null
@@ -1,81 +0,0 @@
-# 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.
-
-version: "1.0"
-
-module:
-  meta:
-    type: remote
-  spec:
-    functions:
-      - function:
-          meta:
-            kind: grpc
-            type: com.example/hello
-          spec:
-            host: localhost
-            port: 5000
-      - function:
-          meta:
-            kind: http
-            type: com.foo/world
-          spec:
-            endpoint: http://localhost:5959/statefun
-            states:
-              - seen_count
-            maxNumBatchRequests: 10000
-      - function:
-          meta:
-            kind: http
-            type: com.bar/world
-          spec:
-            endpoint: http+unix:///hello/world.sock/statefun
-            states:
-              - seen_count
-            maxNumBatchRequests: 10000
-    routers:
-      - router:
-          meta:
-            type: org.apache.flink.statefun.sdk/protobuf-router
-          spec:
-            ingress: com.mycomp.igal/names
-            target: "com.example/hello/{{$.name}}"
-            messageType: org.apache.flink.test.SimpleMessage
-            descriptorSet: classpath:test.desc
-    ingresses:
-      - ingress:
-          meta:
-            type: statefun.kafka.io/protobuf-ingress
-            id: com.mycomp.igal/names
-          spec:
-            address: kafka-broker:9092
-            topics:
-              - names
-            properties:
-              - consumer.group: greeter
-            messageType: org.apache.flink.test.SimpleMessage
-            descriptorSet: classpath:test.desc
-    egresses:
-      - egress:
-          meta:
-            type: io.statefun.kafka/egress
-            id: com.mycomp.foo/bar
-          spec:
-            address: kafka-broker:9092
-            deliverySemantic:
-              type: exactly-once
-              transactionTimeoutMillis: 100000
-            properties:
-              - foo.config: bar
diff --git a/statefun-flink/statefun-flink-core/src/test/resources/module-v2_0/module.yaml b/statefun-flink/statefun-flink-core/src/test/resources/module-v2_0/module.yaml
deleted file mode 100644
index 11ea59e..0000000
--- a/statefun-flink/statefun-flink-core/src/test/resources/module-v2_0/module.yaml
+++ /dev/null
@@ -1,87 +0,0 @@
-# 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.
-
-version: "2.0"
-
-module:
-  meta:
-    type: remote
-  spec:
-    functions:
-      - function:
-          meta:
-            kind: grpc
-            type: com.example/hello
-          spec:
-            host: localhost
-            port: 5000
-      - function:
-          meta:
-            kind: http
-            type: com.foo/world
-          spec:
-            endpoint: http://localhost:5959/statefun
-            states:
-              - name: seen_count
-                expireAfter: 60000millisecond
-                expireMode: after-invoke
-            timeout: 1minutes
-            connectTimeout: 10seconds
-            readTimeout: 10second
-            writeTimeout: 10seconds
-            maxNumBatchRequests: 10000
-      - function:
-          meta:
-            kind: http
-            type: com.bar/world
-          spec:
-            endpoint: http+unix:///hello/world.sock/statefun
-            states:
-              - name: seen_count
-            maxNumBatchRequests: 10000
-    routers:
-      - router:
-          meta:
-            type: org.apache.flink.statefun.sdk/protobuf-router
-          spec:
-            ingress: com.mycomp.igal/names
-            target: "com.example/hello/{{$.name}}"
-            messageType: org.apache.flink.test.SimpleMessage
-            descriptorSet: classpath:test.desc
-    ingresses:
-      - ingress:
-          meta:
-            type: statefun.kafka.io/protobuf-ingress
-            id: com.mycomp.igal/names
-          spec:
-            address: kafka-broker:9092
-            topics:
-              - names
-            properties:
-              - consumer.group: greeter
-            messageType: org.apache.flink.test.SimpleMessage
-            descriptorSet: classpath:test.desc
-    egresses:
-      - egress:
-          meta:
-            type: io.statefun.kafka/egress
-            id: com.mycomp.foo/bar
-          spec:
-            address: kafka-broker:9092
-            deliverySemantic:
-              type: exactly-once
-              transactionTimeoutMillis: 100000
-            properties:
-              - foo.config: bar
diff --git a/statefun-flink/statefun-flink-core/src/test/resources/module-v3_0/module.yaml b/statefun-flink/statefun-flink-core/src/test/resources/module-v3_0/module.yaml
index 2447c06..e39fd07 100644
--- a/statefun-flink/statefun-flink-core/src/test/resources/module-v3_0/module.yaml
+++ b/statefun-flink/statefun-flink-core/src/test/resources/module-v3_0/module.yaml
@@ -32,27 +32,6 @@
               read: 10second
               write: 10seconds
             maxNumBatchRequests: 10000
-      - endpoint:
-          meta:
-            kind: http
-          spec:
-            functions: com.foo.bar/specific_function
-            urlPathTemplate: http://bar.foo.com:8080/functions/abc
-      - endpoint:
-          meta:
-            kind: http
-          spec:
-            functions: com.other.namespace/hello
-            urlPathTemplate: http://namespace.other.com:8080/hello
-    routers:
-      - router:
-          meta:
-            type: org.apache.flink.statefun.sdk/protobuf-router
-          spec:
-            ingress: com.mycomp.igal/names
-            target: "com.example/hello/{{$.name}}"
-            messageType: org.apache.flink.test.SimpleMessage
-            descriptorSet: classpath:test.desc
     ingresses:
       - ingress:
           meta:
diff --git a/statefun-flink/statefun-flink-core/src/test/resources/module-v3_1/module.yaml b/statefun-flink/statefun-flink-core/src/test/resources/module-v3_1/module.yaml
deleted file mode 100644
index 8b595d4..0000000
--- a/statefun-flink/statefun-flink-core/src/test/resources/module-v3_1/module.yaml
+++ /dev/null
@@ -1,93 +0,0 @@
-# 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.
-
-version: "3.1"
-
-module:
-  meta:
-    type: remote
-  spec:
-    endpoints:
-      - endpoint:
-          meta:
-            kind: http
-          spec:
-            functions: com.foo.bar/*
-            urlPathTemplate: http://bar.foo.com:8080/functions/{function.name}
-            maxNumBatchRequests: 10000
-            transport:
-              timeouts:
-                call: 1minutes
-                connect: 10seconds
-                read: 10second
-                write: 10seconds
-      - endpoint:
-          meta:
-            kind: http
-          spec:
-            functions: com.foo.bar.2/*
-            urlPathTemplate: http://2.bar.foo.com:8080/functions/{function.name}
-            transport:
-              type: my.custom/http.transport.type
-              property1: value1
-              property2:
-                - k1: v1
-                - k2: v2
-      - endpoint:
-          meta:
-            kind: http
-          spec:
-            functions: com.foo.bar/specific_function
-            urlPathTemplate: http://bar.foo.com:8080/functions/abc
-      - endpoint:
-          meta:
-            kind: http
-          spec:
-            functions: com.other.namespace/hello
-            urlPathTemplate: http://namespace.other.com:8080/hello
-    routers:
-      - router:
-          meta:
-            type: org.apache.flink.statefun.sdk/protobuf-router
-          spec:
-            ingress: com.mycomp.igal/names
-            target: "com.example/hello/{{$.name}}"
-            messageType: org.apache.flink.test.SimpleMessage
-            descriptorSet: classpath:test.desc
-    ingresses:
-      - ingress:
-          meta:
-            type: statefun.kafka.io/protobuf-ingress
-            id: com.mycomp.igal/names
-          spec:
-            address: kafka-broker:9092
-            topics:
-              - names
-            properties:
-              - consumer.group: greeter
-            messageType: org.apache.flink.test.SimpleMessage
-            descriptorSet: classpath:test.desc
-    egresses:
-      - egress:
-          meta:
-            type: io.statefun.kafka/egress
-            id: com.mycomp.foo/bar
-          spec:
-            address: kafka-broker:9092
-            deliverySemantic:
-              type: exactly-once
-              transactionTimeoutMillis: 100000
-            properties:
-              - foo.config: bar
\ No newline at end of file
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml b/statefun-flink/statefun-flink-core/src/test/resources/remote-module/module.yaml
similarity index 76%
rename from statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml
rename to statefun-flink/statefun-flink-core/src/test/resources/remote-module/module.yaml
index 2014cb3..b1e7142 100644
--- a/statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml
+++ b/statefun-flink/statefun-flink-core/src/test/resources/remote-module/module.yaml
@@ -13,14 +13,14 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-egress:
-  meta:
-    type: io.statefun.kafka/egress
-    id: com.mycomp.foo/bar
-  spec:
-    address: kafka-broker:9092
-    deliverySemantic:
-      type: exactly-once
-      transactionTimeout: 15min
-    properties:
-      - foo.config: bar
+kind: com.foo.bar/test.component.1
+spec:
+  foo: bar
+---
+kind: com.foo.bar/test.component.2
+spec:
+  hello: world
+---
+kind: com.foo.bar/test.component.3
+spec:
+  nonsense: ignored
diff --git a/statefun-flink/statefun-flink-datastream/src/main/java/org/apache/flink/statefun/flink/datastream/RequestReplyFunctionBuilder.java b/statefun-flink/statefun-flink-datastream/src/main/java/org/apache/flink/statefun/flink/datastream/RequestReplyFunctionBuilder.java
index 8cc2ec0..9f479eb 100644
--- a/statefun-flink/statefun-flink-datastream/src/main/java/org/apache/flink/statefun/flink/datastream/RequestReplyFunctionBuilder.java
+++ b/statefun-flink/statefun-flink-datastream/src/main/java/org/apache/flink/statefun/flink/datastream/RequestReplyFunctionBuilder.java
@@ -25,9 +25,10 @@
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.flink.statefun.flink.core.httpfn.DefaultHttpRequestReplyClientSpec;
 import org.apache.flink.statefun.flink.core.httpfn.HttpFunctionEndpointSpec;
+import org.apache.flink.statefun.flink.core.httpfn.TargetFunctions;
 import org.apache.flink.statefun.flink.core.httpfn.TransportClientConstants;
-import org.apache.flink.statefun.flink.core.jsonmodule.FunctionEndpointSpec.Target;
-import org.apache.flink.statefun.flink.core.jsonmodule.FunctionEndpointSpec.UrlPathTemplate;
+import org.apache.flink.statefun.flink.core.httpfn.TransportClientSpec;
+import org.apache.flink.statefun.flink.core.httpfn.UrlPathTemplate;
 import org.apache.flink.statefun.sdk.FunctionType;
 
 /** A Builder for RequestReply remote function type. */
@@ -53,7 +54,8 @@
   private RequestReplyFunctionBuilder(FunctionType functionType, URI endpoint) {
     this.builder =
         HttpFunctionEndpointSpec.builder(
-            Target.functionType(functionType), new UrlPathTemplate(endpoint.toASCIIString()));
+            TargetFunctions.functionType(functionType),
+            new UrlPathTemplate(endpoint.toASCIIString()));
   }
 
   /**
@@ -114,9 +116,11 @@
 
   @Internal
   HttpFunctionEndpointSpec spec() {
-    builder.withTransportClientFactoryType(TransportClientConstants.OKHTTP_CLIENT_FACTORY_TYPE);
-    builder.withTransportClientProperties(
-        transportClientPropertiesAsObjectNode(transportClientTimeoutsSpec));
+    final TransportClientSpec transportClientSpec =
+        new TransportClientSpec(
+            TransportClientConstants.OKHTTP_CLIENT_FACTORY_TYPE,
+            transportClientPropertiesAsObjectNode(transportClientTimeoutsSpec));
+    builder.withTransport(transportClientSpec);
     return builder.build();
   }
 
diff --git a/statefun-flink/statefun-flink-datastream/src/main/java/org/apache/flink/statefun/flink/datastream/SerializableHttpFunctionProvider.java b/statefun-flink/statefun-flink-datastream/src/main/java/org/apache/flink/statefun/flink/datastream/SerializableHttpFunctionProvider.java
index 38477d5..56dc641 100644
--- a/statefun-flink/statefun-flink-datastream/src/main/java/org/apache/flink/statefun/flink/datastream/SerializableHttpFunctionProvider.java
+++ b/statefun-flink/statefun-flink-datastream/src/main/java/org/apache/flink/statefun/flink/datastream/SerializableHttpFunctionProvider.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.statefun.flink.datastream;
 
-import java.util.Collections;
-import java.util.Map;
 import java.util.Objects;
 import javax.annotation.Nullable;
 import javax.annotation.concurrent.NotThreadSafe;
@@ -27,12 +25,8 @@
 import org.apache.flink.statefun.flink.core.httpfn.DefaultHttpRequestReplyClientFactory;
 import org.apache.flink.statefun.flink.core.httpfn.HttpFunctionEndpointSpec;
 import org.apache.flink.statefun.flink.core.httpfn.HttpFunctionProvider;
-import org.apache.flink.statefun.flink.core.httpfn.TransportClientConstants;
-import org.apache.flink.statefun.flink.core.reqreply.RequestReplyClientFactory;
-import org.apache.flink.statefun.flink.core.spi.ExtensionResolver;
 import org.apache.flink.statefun.sdk.FunctionType;
 import org.apache.flink.statefun.sdk.StatefulFunction;
-import org.apache.flink.statefun.sdk.TypeName;
 
 @NotThreadSafe
 @Internal
@@ -40,36 +34,18 @@
 
   private static final long serialVersionUID = 1;
 
-  private final Map<FunctionType, HttpFunctionEndpointSpec> supportedTypes;
+  private final HttpFunctionEndpointSpec spec;
   private transient @Nullable HttpFunctionProvider delegate;
 
-  SerializableHttpFunctionProvider(Map<FunctionType, HttpFunctionEndpointSpec> supportedTypes) {
-    this.supportedTypes = Objects.requireNonNull(supportedTypes);
+  SerializableHttpFunctionProvider(HttpFunctionEndpointSpec spec) {
+    this.spec = Objects.requireNonNull(spec);
   }
 
   @Override
   public StatefulFunction functionOfType(FunctionType type) {
     if (delegate == null) {
-      delegate =
-          new HttpFunctionProvider(
-              supportedTypes, Collections.emptyMap(), new OkHttpTransportClientExtensionResolver());
+      delegate = new HttpFunctionProvider(spec, DefaultHttpRequestReplyClientFactory.INSTANCE);
     }
     return delegate.functionOfType(type);
   }
-
-  private static class OkHttpTransportClientExtensionResolver implements ExtensionResolver {
-
-    private final DefaultHttpRequestReplyClientFactory defaultTransportClientFactory =
-        new DefaultHttpRequestReplyClientFactory();
-
-    @Override
-    public <T> T resolveExtension(TypeName typeName, Class<T> extensionClass) {
-      // the DataStream bridge SDK only supports using the default OkHttp request-reply client
-      if (!typeName.equals(TransportClientConstants.OKHTTP_CLIENT_FACTORY_TYPE)
-          || extensionClass != RequestReplyClientFactory.class) {
-        throw new IllegalStateException("The DataStream SDK does not support extensions.");
-      }
-      return (T) defaultTransportClientFactory;
-    }
-  }
 }
diff --git a/statefun-flink/statefun-flink-datastream/src/main/java/org/apache/flink/statefun/flink/datastream/StatefulFunctionDataStreamBuilder.java b/statefun-flink/statefun-flink-datastream/src/main/java/org/apache/flink/statefun/flink/datastream/StatefulFunctionDataStreamBuilder.java
index 60d3ab9..b61a89d 100644
--- a/statefun-flink/statefun-flink-datastream/src/main/java/org/apache/flink/statefun/flink/datastream/StatefulFunctionDataStreamBuilder.java
+++ b/statefun-flink/statefun-flink-datastream/src/main/java/org/apache/flink/statefun/flink/datastream/StatefulFunctionDataStreamBuilder.java
@@ -105,7 +105,8 @@
       RequestReplyFunctionBuilder builder) {
     Objects.requireNonNull(builder);
     HttpFunctionEndpointSpec spec = builder.spec();
-    putAndThrowIfPresent(requestReplyFunctions, spec.target().asSpecificFunctionType(), spec);
+    putAndThrowIfPresent(
+        requestReplyFunctions, spec.targetFunctions().asSpecificFunctionType(), spec);
     return this;
   }
 
@@ -145,10 +146,8 @@
     final StatefulFunctionsConfig config =
         Optional.fromNullable(this.config).or(() -> StatefulFunctionsConfig.fromEnvironment(env));
 
-    SerializableHttpFunctionProvider httpFunctionProvider =
-        new SerializableHttpFunctionProvider(requestReplyFunctions);
     requestReplyFunctions.forEach(
-        (type, unused) -> functionProviders.put(type, httpFunctionProvider));
+        (type, spec) -> functionProviders.put(type, new SerializableHttpFunctionProvider(spec)));
 
     FeedbackKey<Message> key =
         new FeedbackKey<>(pipelineName, FEEDBACK_INVOCATION_ID_SEQ.incrementAndGet());
diff --git a/statefun-flink/statefun-flink-extensions/pom.xml b/statefun-flink/statefun-flink-extensions/pom.xml
new file mode 100644
index 0000000..5ebc618
--- /dev/null
+++ b/statefun-flink/statefun-flink-extensions/pom.xml
@@ -0,0 +1,52 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.flink</groupId>
+        <artifactId>statefun-flink</artifactId>
+        <version>3.1-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+
+    <artifactId>statefun-flink-extensions</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>statefun-sdk-embedded</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-shaded-jackson</artifactId>
+            <version>2.12.1-13.0</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-annotations</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+    </dependencies>
+
+</project>
diff --git a/statefun-flink/statefun-flink-extensions/src/main/java/org/apache/flink/statefun/extensions/ComponentBinder.java b/statefun-flink/statefun-flink-extensions/src/main/java/org/apache/flink/statefun/extensions/ComponentBinder.java
new file mode 100644
index 0000000..0e92fc7
--- /dev/null
+++ b/statefun-flink/statefun-flink-extensions/src/main/java/org/apache/flink/statefun/extensions/ComponentBinder.java
@@ -0,0 +1,41 @@
+/*
+ * 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.extensions;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+
+/**
+ * A {@link ComponentBinder} binds {@link ComponentJsonObject}s to a remote module. It parses the
+ * specifications of a given component, resolves them into application entities, such as function
+ * providers, ingresses, or egresses, and then binds the entities to the module.
+ */
+@PublicEvolving
+public interface ComponentBinder {
+
+  /**
+   * Bind a {@link ComponentJsonObject} to an underlying remote module through the provided module
+   * binder.
+   *
+   * @param component the component to parse and bind.
+   * @param remoteModuleBinder the binder to use to bind application entities to the underlying
+   *     remote module.
+   */
+  void bind(ComponentJsonObject component, StatefulFunctionModule.Binder remoteModuleBinder);
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java b/statefun-flink/statefun-flink-extensions/src/main/java/org/apache/flink/statefun/extensions/ComponentJsonFormatException.java
similarity index 68%
rename from statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
rename to statefun-flink/statefun-flink-extensions/src/main/java/org/apache/flink/statefun/extensions/ComponentJsonFormatException.java
index 097c3cf..1c1aa41 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
+++ b/statefun-flink/statefun-flink-extensions/src/main/java/org/apache/flink/statefun/extensions/ComponentJsonFormatException.java
@@ -15,18 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.statefun.flink.core.jsonmodule;
 
-import org.apache.flink.statefun.sdk.FunctionType;
+package org.apache.flink.statefun.extensions;
 
-public interface FunctionSpec {
+public final class ComponentJsonFormatException extends IllegalArgumentException {
+  private static final long serialVersionUID = 1L;
 
-  FunctionType functionType();
+  public ComponentJsonFormatException(String message) {
+    super(message);
+  }
 
-  Kind kind();
-
-  enum Kind {
-    HTTP,
-    GRPC
+  public ComponentJsonFormatException(String message, Throwable cause) {
+    super(message, cause);
   }
 }
diff --git a/statefun-flink/statefun-flink-extensions/src/main/java/org/apache/flink/statefun/extensions/ComponentJsonObject.java b/statefun-flink/statefun-flink-extensions/src/main/java/org/apache/flink/statefun/extensions/ComponentJsonObject.java
new file mode 100644
index 0000000..fa53957
--- /dev/null
+++ b/statefun-flink/statefun-flink-extensions/src/main/java/org/apache/flink/statefun/extensions/ComponentJsonObject.java
@@ -0,0 +1,131 @@
+/*
+ * 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.extensions;
+
+import java.util.Objects;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+
+/**
+ * A {@link ComponentJsonObject} consists one or more application entities (i.e. function providers,
+ * ingresses, routers, or egresses) that should be bound to a remote {@link StatefulFunctionModule}.
+ *
+ * <p>Each component is represented in remote module YAML specification files as a single YAML
+ * document of the following format:
+ *
+ * <pre>
+ * kind: com.foo.bar.v5/some.component         (typename)
+ * spec:
+ *   ...                                       (specification document of the component)
+ * </pre>
+ *
+ * <p>The {@code kind} is a {@link TypeName} that should be universally unique within the
+ * application. It identifies which {@link ComponentBinder} recognizes this component and knows how
+ * to parse it to resolve application entities to be bound to the module.
+ *
+ * @see ComponentBinder
+ */
+@PublicEvolving
+public final class ComponentJsonObject {
+
+  public static final String BINDER_KIND_FIELD = "kind";
+  public static final String SPEC_FIELD = "spec";
+
+  private final ObjectNode rawObjectNode;
+
+  private final TypeName binderTypename;
+  private final JsonNode specJsonNode;
+
+  public ComponentJsonObject(JsonNode jsonNode) {
+    Objects.requireNonNull(jsonNode);
+
+    checkIsObject(jsonNode);
+    this.rawObjectNode = (ObjectNode) jsonNode;
+
+    this.binderTypename = parseBinderTypename(rawObjectNode);
+    this.specJsonNode = extractSpecJsonNode(rawObjectNode);
+  }
+
+  /**
+   * Returns the complete component JSON object.
+   *
+   * @return the complete component JSON object.
+   */
+  public ObjectNode get() {
+    return rawObjectNode;
+  }
+
+  /**
+   * Returns the {@link TypeName} of the binder for this component.
+   *
+   * @return the {@link TypeName} of the binder for this component.
+   */
+  public TypeName binderTypename() {
+    return binderTypename;
+  }
+
+  /**
+   * Returns the specification JSON node for this component.
+   *
+   * @return the specification JSON node for this component.
+   */
+  public JsonNode specJsonNode() {
+    return specJsonNode;
+  }
+
+  @Override
+  public String toString() {
+    return rawObjectNode.toString();
+  }
+
+  private static void checkIsObject(JsonNode jsonNode) {
+    if (!jsonNode.isObject()) {
+      throwExceptionWithFormatHint();
+    }
+  }
+
+  private static TypeName parseBinderTypename(ObjectNode componentObject) {
+    final JsonNode binderKindObject = componentObject.get(BINDER_KIND_FIELD);
+    if (binderKindObject == null) {
+      throwExceptionWithFormatHint();
+    }
+
+    try {
+      return TypeName.parseFrom(binderKindObject.asText());
+    } catch (Exception e) {
+      throw new ComponentJsonFormatException("Invalid binder kind format.", e);
+    }
+  }
+
+  private static JsonNode extractSpecJsonNode(ObjectNode componentObject) {
+    final JsonNode specJsonNode = componentObject.get(SPEC_FIELD);
+    if (specJsonNode == null) {
+      throwExceptionWithFormatHint();
+    }
+    return specJsonNode;
+  }
+
+  private static void throwExceptionWithFormatHint() {
+    throw new ComponentJsonFormatException(
+        "Invalid ComponentJsonObject; components should be a JSON object with the required fields [kind] and [spec].");
+  }
+}
diff --git a/statefun-sdk-embedded/src/main/java/org/apache/flink/statefun/sdk/spi/ExtensionModule.java b/statefun-flink/statefun-flink-extensions/src/main/java/org/apache/flink/statefun/extensions/ExtensionModule.java
similarity index 96%
rename from statefun-sdk-embedded/src/main/java/org/apache/flink/statefun/sdk/spi/ExtensionModule.java
rename to statefun-flink/statefun-flink-extensions/src/main/java/org/apache/flink/statefun/extensions/ExtensionModule.java
index 5f84ffd..254268e 100644
--- a/statefun-sdk-embedded/src/main/java/org/apache/flink/statefun/sdk/spi/ExtensionModule.java
+++ b/statefun-flink/statefun-flink-extensions/src/main/java/org/apache/flink/statefun/extensions/ExtensionModule.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.statefun.sdk.spi;
+package org.apache.flink.statefun.extensions;
 
 import java.util.Map;
 import org.apache.flink.statefun.sdk.TypeName;
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/AutoRoutableProtobufRouter.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/common/AutoRoutableProtobufRouter.java
similarity index 97%
rename from statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/AutoRoutableProtobufRouter.java
rename to statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/common/AutoRoutableProtobufRouter.java
index bb05564..5d9d901 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/protorouter/AutoRoutableProtobufRouter.java
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/common/AutoRoutableProtobufRouter.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.statefun.flink.core.protorouter;
+package org.apache.flink.statefun.flink.io.common;
 
 import com.google.protobuf.ByteString;
 import com.google.protobuf.Message;
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/common/json/EgressIdentifierJsonDeserializer.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/common/json/EgressIdentifierJsonDeserializer.java
new file mode 100644
index 0000000..841d1fd
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/common/json/EgressIdentifierJsonDeserializer.java
@@ -0,0 +1,37 @@
+/*
+ * 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.io.common.json;
+
+import java.io.IOException;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonDeserializer;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.statefun.sdk.io.EgressIdentifier;
+import org.apache.flink.statefun.sdk.reqreply.generated.TypedValue;
+
+public final class EgressIdentifierJsonDeserializer
+    extends JsonDeserializer<EgressIdentifier<TypedValue>> {
+  @Override
+  public EgressIdentifier<TypedValue> deserialize(
+      JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+    final TypeName id = TypeName.parseFrom(jsonParser.getText());
+    return new EgressIdentifier<>(id.namespace(), id.name(), TypedValue.class);
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/common/json/IngressIdentifierJsonDeserializer.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/common/json/IngressIdentifierJsonDeserializer.java
new file mode 100644
index 0000000..81a85d4
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/common/json/IngressIdentifierJsonDeserializer.java
@@ -0,0 +1,37 @@
+/*
+ * 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.io.common.json;
+
+import com.google.protobuf.Message;
+import java.io.IOException;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonDeserializer;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.statefun.sdk.io.IngressIdentifier;
+
+public final class IngressIdentifierJsonDeserializer
+    extends JsonDeserializer<IngressIdentifier<Message>> {
+  @Override
+  public IngressIdentifier<Message> deserialize(
+      JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+    final TypeName id = TypeName.parseFrom(jsonParser.getText());
+    return new IngressIdentifier<>(Message.class, id.namespace(), id.name());
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/common/json/PropertiesJsonDeserializer.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/common/json/PropertiesJsonDeserializer.java
new file mode 100644
index 0000000..4578aa6
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/common/json/PropertiesJsonDeserializer.java
@@ -0,0 +1,42 @@
+/*
+ * 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.io.common.json;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonDeserializer;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+
+public final class PropertiesJsonDeserializer extends JsonDeserializer<Properties> {
+  @Override
+  public Properties deserialize(
+      JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+    final Iterable<JsonNode> propertyNodes = jsonParser.readValueAs(JsonNode.class);
+    final Properties properties = new Properties();
+    propertyNodes.forEach(
+        jsonNode -> {
+          Map.Entry<String, JsonNode> offsetNode = jsonNode.fields().next();
+          properties.setProperty(offsetNode.getKey(), offsetNode.getValue().asText());
+        });
+    return properties;
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/GenericKafkaSinkProvider.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/GenericKafkaSinkProvider.java
deleted file mode 100644
index fd87a69..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/GenericKafkaSinkProvider.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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.io.kafka;
-
-import static org.apache.flink.statefun.flink.io.kafka.KafkaEgressSpecJsonParser.exactlyOnceDeliveryTxnTimeout;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaEgressSpecJsonParser.kafkaAddress;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaEgressSpecJsonParser.kafkaClientProperties;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaEgressSpecJsonParser.optionalDeliverySemantic;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.io.spi.JsonEgressSpec;
-import org.apache.flink.statefun.flink.io.spi.SinkProvider;
-import org.apache.flink.statefun.sdk.io.EgressIdentifier;
-import org.apache.flink.statefun.sdk.io.EgressSpec;
-import org.apache.flink.statefun.sdk.kafka.KafkaEgressBuilder;
-import org.apache.flink.statefun.sdk.kafka.KafkaEgressSpec;
-import org.apache.flink.statefun.sdk.reqreply.generated.TypedValue;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-
-final class GenericKafkaSinkProvider implements SinkProvider {
-
-  private final KafkaSinkProvider delegateProvider = new KafkaSinkProvider();
-
-  @Override
-  public <T> SinkFunction<T> forSpec(EgressSpec<T> spec) {
-    KafkaEgressSpec<T> kafkaEgressSpec = asKafkaEgressSpec(spec);
-    return delegateProvider.forSpec(kafkaEgressSpec);
-  }
-
-  private static <T> KafkaEgressSpec<T> asKafkaEgressSpec(EgressSpec<T> spec) {
-    if (!(spec instanceof JsonEgressSpec)) {
-      throw new IllegalArgumentException("Wrong type " + spec.type());
-    }
-    JsonEgressSpec<T> casted = (JsonEgressSpec<T>) spec;
-
-    EgressIdentifier<T> id = casted.id();
-    validateConsumedType(id);
-
-    JsonNode json = casted.json();
-
-    KafkaEgressBuilder<T> kafkaEgressBuilder = KafkaEgressBuilder.forIdentifier(id);
-    kafkaEgressBuilder
-        .withKafkaAddress(kafkaAddress(json))
-        .withProperties(kafkaClientProperties(json))
-        .withSerializer(serializerClass());
-
-    optionalDeliverySemantic(json)
-        .ifPresent(
-            semantic -> {
-              switch (semantic) {
-                case AT_LEAST_ONCE:
-                  kafkaEgressBuilder.withAtLeastOnceProducerSemantics();
-                  break;
-                case EXACTLY_ONCE:
-                  kafkaEgressBuilder.withExactlyOnceProducerSemantics(
-                      exactlyOnceDeliveryTxnTimeout(json));
-                  break;
-                case NONE:
-                  kafkaEgressBuilder.withNoProducerSemantics();
-                  break;
-                default:
-                  throw new IllegalStateException("Unrecognized producer semantic: " + semantic);
-              }
-            });
-
-    return kafkaEgressBuilder.build();
-  }
-
-  private static void validateConsumedType(EgressIdentifier<?> id) {
-    Class<?> consumedType = id.consumedType();
-    if (TypedValue.class != consumedType) {
-      throw new IllegalArgumentException(
-          "Generic Kafka egress is only able to consume messages types of "
-              + TypedValue.class.getName()
-              + " but "
-              + consumedType.getName()
-              + " is provided.");
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  private static <T> Class<T> serializerClass() {
-    // this cast is safe, because we've already validated that the consumed type is Any.
-    return (Class<T>) GenericKafkaEgressSerializer.class;
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaEgressSpecJsonParser.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaEgressSpecJsonParser.java
deleted file mode 100644
index b357fe9..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaEgressSpecJsonParser.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * 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.io.kafka;
-
-import java.time.Duration;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Optional;
-import java.util.OptionalLong;
-import java.util.Properties;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.common.json.Selectors;
-import org.apache.flink.statefun.sdk.kafka.KafkaProducerSemantic;
-
-final class KafkaEgressSpecJsonParser {
-
-  private KafkaEgressSpecJsonParser() {}
-
-  private static final JsonPointer PROPERTIES_POINTER =
-      JsonPointer.compile("/egress/spec/properties");
-  private static final JsonPointer ADDRESS_POINTER = JsonPointer.compile("/egress/spec/address");
-
-  private static final JsonPointer DELIVERY_SEMANTICS_POINTER =
-      JsonPointer.compile("/egress/spec/deliverySemantic");
-  private static final JsonPointer DELIVERY_SEMANTICS_TYPE_POINTER =
-      JsonPointer.compile("/egress/spec/deliverySemantic/type");
-
-  /** @deprecated see {@link #DELIVERY_EXACTLY_ONCE_DURATION_TXN_TIMEOUT_POINTER}. */
-  private static final JsonPointer DELIVERY_EXACTLY_ONCE_TXN_TIMEOUT_POINTER =
-      JsonPointer.compile("/egress/spec/deliverySemantic/transactionTimeoutMillis");
-
-  private static final JsonPointer DELIVERY_EXACTLY_ONCE_DURATION_TXN_TIMEOUT_POINTER =
-      JsonPointer.compile("/egress/spec/deliverySemantic/transactionTimeout");
-
-  static String kafkaAddress(JsonNode json) {
-    return Selectors.textAt(json, ADDRESS_POINTER);
-  }
-
-  static Properties kafkaClientProperties(JsonNode json) {
-    Map<String, String> kvs = Selectors.propertiesAt(json, PROPERTIES_POINTER);
-    Properties properties = new Properties();
-    kvs.forEach(properties::setProperty);
-    return properties;
-  }
-
-  static Optional<KafkaProducerSemantic> optionalDeliverySemantic(JsonNode json) {
-    if (json.at(DELIVERY_SEMANTICS_POINTER).isMissingNode()) {
-      return Optional.empty();
-    }
-
-    String deliverySemanticType =
-        Selectors.textAt(json, DELIVERY_SEMANTICS_TYPE_POINTER).toLowerCase(Locale.ENGLISH);
-    switch (deliverySemanticType) {
-      case "at-least-once":
-        return Optional.of(KafkaProducerSemantic.AT_LEAST_ONCE);
-      case "exactly-once":
-        return Optional.of(KafkaProducerSemantic.EXACTLY_ONCE);
-      case "none":
-        return Optional.of(KafkaProducerSemantic.NONE);
-      default:
-        throw new IllegalArgumentException(
-            "Invalid delivery semantic type: "
-                + deliverySemanticType
-                + "; valid types are [at-least-once, exactly-once, none]");
-    }
-  }
-
-  static Duration exactlyOnceDeliveryTxnTimeout(JsonNode json) {
-    // Prefer deprecated millis based timeout for backwards compatibility
-    // then fallback to duration based configuration.
-    OptionalLong transactionTimeoutMilli =
-        Selectors.optionalLongAt(json, DELIVERY_EXACTLY_ONCE_TXN_TIMEOUT_POINTER);
-    if (transactionTimeoutMilli.isPresent()) {
-      return Duration.ofMillis(transactionTimeoutMilli.getAsLong());
-    }
-
-    return Selectors.durationAt(json, DELIVERY_EXACTLY_ONCE_DURATION_TXN_TIMEOUT_POINTER);
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaFlinkIoModule.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaFlinkIoModule.java
index cefbc84..f7465d3 100644
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaFlinkIoModule.java
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaFlinkIoModule.java
@@ -28,13 +28,6 @@
   @Override
   public void configure(Map<String, String> globalConfiguration, Binder binder) {
     binder.bindSourceProvider(Constants.KAFKA_INGRESS_TYPE, new KafkaSourceProvider());
-    binder.bindSourceProvider(
-        ProtobufKafkaIngressTypes.PROTOBUF_KAFKA_INGRESS_TYPE, new ProtobufKafkaSourceProvider());
-    binder.bindSourceProvider(
-        ProtobufKafkaIngressTypes.ROUTABLE_PROTOBUF_KAFKA_INGRESS_TYPE,
-        new RoutableProtobufKafkaSourceProvider());
     binder.bindSinkProvider(Constants.KAFKA_EGRESS_TYPE, new KafkaSinkProvider());
-    binder.bindSinkProvider(
-        KafkaEgressTypes.GENERIC_KAFKA_EGRESS_TYPE, new GenericKafkaSinkProvider());
   }
 }
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaIngressSpecJsonParser.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaIngressSpecJsonParser.java
deleted file mode 100644
index 43dfe86..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaIngressSpecJsonParser.java
+++ /dev/null
@@ -1,221 +0,0 @@
-/*
- * 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.io.kafka;
-
-import java.time.ZonedDateTime;
-import java.time.format.DateTimeFormatter;
-import java.time.format.DateTimeParseException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.common.json.NamespaceNamePair;
-import org.apache.flink.statefun.flink.common.json.Selectors;
-import org.apache.flink.statefun.flink.io.generated.RoutingConfig;
-import org.apache.flink.statefun.flink.io.generated.TargetFunctionType;
-import org.apache.flink.statefun.sdk.kafka.KafkaIngressAutoResetPosition;
-import org.apache.flink.statefun.sdk.kafka.KafkaIngressDeserializer;
-import org.apache.flink.statefun.sdk.kafka.KafkaIngressStartupPosition;
-import org.apache.flink.statefun.sdk.kafka.KafkaTopicPartition;
-
-final class KafkaIngressSpecJsonParser {
-
-  private KafkaIngressSpecJsonParser() {}
-
-  private static final JsonPointer DESCRIPTOR_SET_POINTER =
-      JsonPointer.compile("/ingress/spec/descriptorSet");
-  private static final JsonPointer TOPICS_POINTER = JsonPointer.compile("/ingress/spec/topics");
-  private static final JsonPointer MESSAGE_TYPE_POINTER =
-      JsonPointer.compile("/ingress/spec/messageType");
-  private static final JsonPointer PROPERTIES_POINTER =
-      JsonPointer.compile("/ingress/spec/properties");
-  private static final JsonPointer ADDRESS_POINTER = JsonPointer.compile("/ingress/spec/address");
-  private static final JsonPointer GROUP_ID_POINTER =
-      JsonPointer.compile("/ingress/spec/consumerGroupId");
-  private static final JsonPointer AUTO_RESET_POS_POINTER =
-      JsonPointer.compile("/ingress/spec/autoOffsetResetPosition");
-
-  private static final JsonPointer STARTUP_POS_POINTER =
-      JsonPointer.compile("/ingress/spec/startupPosition");
-  private static final JsonPointer STARTUP_POS_TYPE_POINTER =
-      JsonPointer.compile("/ingress/spec/startupPosition/type");
-  private static final JsonPointer STARTUP_SPECIFIC_OFFSETS_POINTER =
-      JsonPointer.compile("/ingress/spec/startupPosition/offsets");
-  private static final JsonPointer STARTUP_DATE_POINTER =
-      JsonPointer.compile("/ingress/spec/startupPosition/date");
-
-  private static final JsonPointer ROUTABLE_TOPIC_NAME_POINTER = JsonPointer.compile("/topic");
-  private static final JsonPointer ROUTABLE_TOPIC_VALUE_TYPE_POINTER =
-      JsonPointer.compile("/valueType");
-  private static final JsonPointer ROUTABLE_TOPIC_TARGETS_POINTER = JsonPointer.compile("/targets");
-
-  private static final String STARTUP_DATE_PATTERN = "yyyy-MM-dd HH:mm:ss.SSS Z";
-  private static final DateTimeFormatter STARTUP_DATE_FORMATTER =
-      DateTimeFormatter.ofPattern(STARTUP_DATE_PATTERN);
-
-  static List<String> topics(JsonNode json) {
-    return Selectors.textListAt(json, TOPICS_POINTER);
-  }
-
-  static Map<String, RoutingConfig> routableTopics(JsonNode json) {
-    Map<String, RoutingConfig> routableTopics = new HashMap<>();
-    for (JsonNode routableTopicNode : Selectors.listAt(json, TOPICS_POINTER)) {
-      final String topic = Selectors.textAt(routableTopicNode, ROUTABLE_TOPIC_NAME_POINTER);
-      final String typeUrl = Selectors.textAt(routableTopicNode, ROUTABLE_TOPIC_VALUE_TYPE_POINTER);
-      final List<TargetFunctionType> targets = parseRoutableTargetFunctionTypes(routableTopicNode);
-
-      routableTopics.put(
-          topic,
-          RoutingConfig.newBuilder()
-              .setTypeUrl(typeUrl)
-              .addAllTargetFunctionTypes(targets)
-              .build());
-    }
-    return routableTopics;
-  }
-
-  static Properties kafkaClientProperties(JsonNode json) {
-    Map<String, String> kvs = Selectors.propertiesAt(json, PROPERTIES_POINTER);
-    Properties properties = new Properties();
-    kvs.forEach(properties::setProperty);
-    return properties;
-  }
-
-  static String kafkaAddress(JsonNode json) {
-    return Selectors.textAt(json, ADDRESS_POINTER);
-  }
-
-  @SuppressWarnings("unchecked")
-  static <T> KafkaIngressDeserializer<T> deserializer(JsonNode json) {
-    String descriptorSetPath = Selectors.textAt(json, DESCRIPTOR_SET_POINTER);
-    String messageType = Selectors.textAt(json, MESSAGE_TYPE_POINTER);
-    // this cast is safe since we validate that the produced message type (T) is assignable to a
-    // Message.
-    // see asJsonIngressSpec()
-    return (KafkaIngressDeserializer<T>)
-        new ProtobufKafkaIngressDeserializer(descriptorSetPath, messageType);
-  }
-
-  static Optional<String> optionalConsumerGroupId(JsonNode json) {
-    return Selectors.optionalTextAt(json, GROUP_ID_POINTER);
-  }
-
-  static Optional<KafkaIngressAutoResetPosition> optionalAutoOffsetResetPosition(JsonNode json) {
-    Optional<String> conf = Selectors.optionalTextAt(json, AUTO_RESET_POS_POINTER);
-    if (!conf.isPresent()) {
-      return Optional.empty();
-    }
-
-    String autoOffsetResetConfig = conf.get().toUpperCase(Locale.ENGLISH);
-
-    try {
-      return Optional.of(KafkaIngressAutoResetPosition.valueOf(autoOffsetResetConfig));
-    } catch (IllegalArgumentException e) {
-      throw new IllegalArgumentException(
-          "Invalid autoOffsetResetPosition: "
-              + autoOffsetResetConfig
-              + "; valid values are "
-              + Arrays.toString(KafkaIngressAutoResetPosition.values()),
-          e);
-    }
-  }
-
-  static Optional<KafkaIngressStartupPosition> optionalStartupPosition(JsonNode json) {
-    if (json.at(STARTUP_POS_POINTER).isMissingNode()) {
-      return Optional.empty();
-    }
-
-    String startupType =
-        Selectors.textAt(json, STARTUP_POS_TYPE_POINTER).toLowerCase(Locale.ENGLISH);
-    switch (startupType) {
-      case "group-offsets":
-        return Optional.of(KafkaIngressStartupPosition.fromGroupOffsets());
-      case "earliest":
-        return Optional.of(KafkaIngressStartupPosition.fromEarliest());
-      case "latest":
-        return Optional.of(KafkaIngressStartupPosition.fromLatest());
-      case "specific-offsets":
-        return Optional.of(
-            KafkaIngressStartupPosition.fromSpecificOffsets(specificOffsetsStartupMap(json)));
-      case "date":
-        return Optional.of(KafkaIngressStartupPosition.fromDate(startupDate(json)));
-      default:
-        throw new IllegalArgumentException(
-            "Invalid startup position type: "
-                + startupType
-                + "; valid values are [group-offsets, earliest, latest, specific-offsets, date]");
-    }
-  }
-
-  private static Map<KafkaTopicPartition, Long> specificOffsetsStartupMap(JsonNode json) {
-    Map<String, Long> kvs = Selectors.longPropertiesAt(json, STARTUP_SPECIFIC_OFFSETS_POINTER);
-    Map<KafkaTopicPartition, Long> offsets = new HashMap<>(kvs.size());
-    kvs.forEach(
-        (partition, offset) ->
-            offsets.put(KafkaTopicPartition.fromString(partition), validateOffsetLong(offset)));
-    return offsets;
-  }
-
-  private static ZonedDateTime startupDate(JsonNode json) {
-    String dateStr = Selectors.textAt(json, STARTUP_DATE_POINTER);
-    try {
-      return ZonedDateTime.parse(dateStr, STARTUP_DATE_FORMATTER);
-    } catch (DateTimeParseException e) {
-      throw new IllegalArgumentException(
-          "Unable to parse date string for startup position: "
-              + dateStr
-              + "; the date should conform to the pattern "
-              + STARTUP_DATE_PATTERN,
-          e);
-    }
-  }
-
-  private static Long validateOffsetLong(Long offset) {
-    if (offset < 0) {
-      throw new IllegalArgumentException(
-          "Invalid offset value: "
-              + offset
-              + "; must be a numeric integer with value between 0 and "
-              + Long.MAX_VALUE);
-    }
-
-    return offset;
-  }
-
-  private static List<TargetFunctionType> parseRoutableTargetFunctionTypes(
-      JsonNode routableTopicNode) {
-    final List<TargetFunctionType> targets = new ArrayList<>();
-    for (String namespaceAndName :
-        Selectors.textListAt(routableTopicNode, ROUTABLE_TOPIC_TARGETS_POINTER)) {
-      NamespaceNamePair namespaceNamePair = NamespaceNamePair.from(namespaceAndName);
-      targets.add(
-          TargetFunctionType.newBuilder()
-              .setNamespace(namespaceNamePair.namespace())
-              .setType(namespaceNamePair.name())
-              .build());
-    }
-    return targets;
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaSinkProvider.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaSinkProvider.java
index ba0f941..e6c585b 100644
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaSinkProvider.java
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaSinkProvider.java
@@ -26,6 +26,7 @@
 import org.apache.flink.statefun.sdk.io.EgressSpec;
 import org.apache.flink.statefun.sdk.kafka.KafkaEgressSerializer;
 import org.apache.flink.statefun.sdk.kafka.KafkaEgressSpec;
+import org.apache.flink.statefun.sdk.kafka.KafkaProducerSemantic;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer;
 import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.Semantic;
@@ -46,7 +47,7 @@
     if (producerSemantic == Semantic.EXACTLY_ONCE) {
       properties.setProperty(
           ProducerConfig.TRANSACTION_TIMEOUT_CONFIG,
-          String.valueOf(spec.transactionTimeoutDuration().toMillis()));
+          String.valueOf(spec.semantic().asExactlyOnceSemantic().transactionTimeout().toMillis()));
     }
 
     return new FlinkKafkaProducer<>(
@@ -63,15 +64,15 @@
   }
 
   private static <T> Semantic semanticFromSpec(KafkaEgressSpec<T> spec) {
-    switch (spec.semantic()) {
-      case EXACTLY_ONCE:
-        return Semantic.EXACTLY_ONCE;
-      case AT_LEAST_ONCE:
-        return Semantic.AT_LEAST_ONCE;
-      case NONE:
-        return Semantic.NONE;
-      default:
-        throw new IllegalArgumentException("Unknown producer semantic " + spec.semantic());
+    final KafkaProducerSemantic semantic = spec.semantic();
+    if (semantic.isExactlyOnceSemantic()) {
+      return Semantic.EXACTLY_ONCE;
+    } else if (semantic.isAtLeastOnceSemantic()) {
+      return Semantic.AT_LEAST_ONCE;
+    } else if (semantic.isNoSemantic()) {
+      return Semantic.NONE;
+    } else {
+      throw new IllegalArgumentException("Unknown producer semantic " + semantic.getClass());
     }
   }
 
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/ProtobufKafkaIngressDeserializer.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/ProtobufKafkaIngressDeserializer.java
deleted file mode 100644
index 0b0447a..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/ProtobufKafkaIngressDeserializer.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.io.kafka;
-
-import com.google.protobuf.*;
-import java.io.IOException;
-import java.net.URL;
-import java.util.Objects;
-import java.util.Optional;
-import org.apache.flink.statefun.flink.common.ResourceLocator;
-import org.apache.flink.statefun.flink.common.protobuf.ProtobufDescriptorMap;
-import org.apache.flink.statefun.sdk.kafka.KafkaIngressDeserializer;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-
-final class ProtobufKafkaIngressDeserializer implements KafkaIngressDeserializer<Message> {
-
-  private static final long serialVersionUID = 1;
-
-  private final String descriptorSetPath;
-  private final String messageType;
-
-  private transient Parser<? extends Message> parser;
-
-  ProtobufKafkaIngressDeserializer(String descriptorSetPath, String messageType) {
-    this.descriptorSetPath = Objects.requireNonNull(descriptorSetPath);
-    this.messageType = Objects.requireNonNull(messageType);
-  }
-
-  @Override
-  public Message deserialize(ConsumerRecord<byte[], byte[]> input) {
-    try {
-      return parser().parseFrom(input.value());
-    } catch (InvalidProtocolBufferException e) {
-      throw new IllegalStateException(e);
-    }
-  }
-
-  private Parser<? extends Message> parser() {
-    if (parser != null) {
-      return parser;
-    }
-    ProtobufDescriptorMap descriptorPath = protobufDescriptorMap(descriptorSetPath);
-    Optional<Descriptors.GenericDescriptor> maybeDescriptor =
-        descriptorPath.getDescriptorByName(messageType);
-    if (!maybeDescriptor.isPresent()) {
-      throw new IllegalStateException(
-          "Unable to read the descriptor set locate at  " + descriptorSetPath);
-    }
-    Descriptors.Descriptor descriptor = (Descriptors.Descriptor) maybeDescriptor.get();
-    DynamicMessage dynamicMessage = DynamicMessage.getDefaultInstance(descriptor);
-    Parser<? extends Message> parser = dynamicMessage.getParserForType();
-
-    this.parser = parser;
-    return parser;
-  }
-
-  private static ProtobufDescriptorMap protobufDescriptorMap(String descriptorSetPath) {
-    try {
-      URL url = ResourceLocator.findNamedResource(descriptorSetPath);
-      return ProtobufDescriptorMap.from(url);
-    } catch (IOException e) {
-      throw new IllegalStateException(
-          "Error while processing an ingress definition. Unable to read the descriptor set at  "
-              + descriptorSetPath,
-          e);
-    }
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/ProtobufKafkaSourceProvider.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/ProtobufKafkaSourceProvider.java
deleted file mode 100644
index f8dc7d1..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/ProtobufKafkaSourceProvider.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * 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.io.kafka;
-
-import static org.apache.flink.statefun.flink.io.kafka.KafkaIngressSpecJsonParser.deserializer;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaIngressSpecJsonParser.kafkaAddress;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaIngressSpecJsonParser.kafkaClientProperties;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaIngressSpecJsonParser.optionalAutoOffsetResetPosition;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaIngressSpecJsonParser.optionalConsumerGroupId;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaIngressSpecJsonParser.optionalStartupPosition;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaIngressSpecJsonParser.topics;
-
-import com.google.protobuf.Message;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.io.spi.JsonIngressSpec;
-import org.apache.flink.statefun.flink.io.spi.SourceProvider;
-import org.apache.flink.statefun.sdk.io.IngressIdentifier;
-import org.apache.flink.statefun.sdk.io.IngressSpec;
-import org.apache.flink.statefun.sdk.kafka.KafkaIngressBuilder;
-import org.apache.flink.statefun.sdk.kafka.KafkaIngressBuilderApiExtension;
-import org.apache.flink.statefun.sdk.kafka.KafkaIngressSpec;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-
-final class ProtobufKafkaSourceProvider implements SourceProvider {
-
-  private final KafkaSourceProvider delegateProvider = new KafkaSourceProvider();
-
-  @Override
-  public <T> SourceFunction<T> forSpec(IngressSpec<T> spec) {
-    KafkaIngressSpec<T> kafkaIngressSpec = asKafkaIngressSpec(spec);
-    return delegateProvider.forSpec(kafkaIngressSpec);
-  }
-
-  private static <T> KafkaIngressSpec<T> asKafkaIngressSpec(IngressSpec<T> spec) {
-    if (!(spec instanceof JsonIngressSpec)) {
-      throw new IllegalArgumentException("Wrong type " + spec.type());
-    }
-    JsonIngressSpec<T> casted = (JsonIngressSpec<T>) spec;
-
-    IngressIdentifier<T> id = casted.id();
-    Class<?> producedType = casted.id().producedType();
-    if (!Message.class.isAssignableFrom(producedType)) {
-      throw new IllegalArgumentException(
-          "ProtocolBuffer based ingress is only able to produce types that derive from "
-              + Message.class.getName()
-              + " but "
-              + producedType.getName()
-              + " is provided.");
-    }
-
-    JsonNode json = casted.json();
-
-    KafkaIngressBuilder<T> kafkaIngressBuilder = KafkaIngressBuilder.forIdentifier(id);
-    kafkaIngressBuilder
-        .withKafkaAddress(kafkaAddress(json))
-        .withProperties(kafkaClientProperties(json))
-        .addTopics(topics(json));
-
-    optionalConsumerGroupId(json).ifPresent(kafkaIngressBuilder::withConsumerGroupId);
-    optionalAutoOffsetResetPosition(json).ifPresent(kafkaIngressBuilder::withAutoResetPosition);
-    optionalStartupPosition(json).ifPresent(kafkaIngressBuilder::withStartupPosition);
-
-    KafkaIngressBuilderApiExtension.withDeserializer(kafkaIngressBuilder, deserializer(json));
-
-    return kafkaIngressBuilder.build();
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/RoutableProtobufKafkaSourceProvider.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/RoutableProtobufKafkaSourceProvider.java
deleted file mode 100644
index a892b39..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/RoutableProtobufKafkaSourceProvider.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * 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.io.kafka;
-
-import static org.apache.flink.statefun.flink.io.kafka.KafkaIngressSpecJsonParser.kafkaAddress;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaIngressSpecJsonParser.kafkaClientProperties;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaIngressSpecJsonParser.optionalAutoOffsetResetPosition;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaIngressSpecJsonParser.optionalConsumerGroupId;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaIngressSpecJsonParser.optionalStartupPosition;
-import static org.apache.flink.statefun.flink.io.kafka.KafkaIngressSpecJsonParser.routableTopics;
-
-import com.google.protobuf.Message;
-import java.util.ArrayList;
-import java.util.Map;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.io.generated.RoutingConfig;
-import org.apache.flink.statefun.flink.io.spi.JsonIngressSpec;
-import org.apache.flink.statefun.flink.io.spi.SourceProvider;
-import org.apache.flink.statefun.sdk.io.IngressIdentifier;
-import org.apache.flink.statefun.sdk.io.IngressSpec;
-import org.apache.flink.statefun.sdk.kafka.KafkaIngressBuilder;
-import org.apache.flink.statefun.sdk.kafka.KafkaIngressBuilderApiExtension;
-import org.apache.flink.statefun.sdk.kafka.KafkaIngressDeserializer;
-import org.apache.flink.statefun.sdk.kafka.KafkaIngressSpec;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-
-final class RoutableProtobufKafkaSourceProvider implements SourceProvider {
-
-  private final KafkaSourceProvider delegateProvider = new KafkaSourceProvider();
-
-  @Override
-  public <T> SourceFunction<T> forSpec(IngressSpec<T> spec) {
-    KafkaIngressSpec<T> kafkaIngressSpec = asKafkaIngressSpec(spec);
-    return delegateProvider.forSpec(kafkaIngressSpec);
-  }
-
-  private static <T> KafkaIngressSpec<T> asKafkaIngressSpec(IngressSpec<T> spec) {
-    if (!(spec instanceof JsonIngressSpec)) {
-      throw new IllegalArgumentException("Wrong type " + spec.type());
-    }
-    JsonIngressSpec<T> casted = (JsonIngressSpec<T>) spec;
-
-    IngressIdentifier<T> id = casted.id();
-    Class<T> producedType = casted.id().producedType();
-    if (!Message.class.isAssignableFrom(producedType)) {
-      throw new IllegalArgumentException(
-          "ProtocolBuffer based ingress is only able to produce types that derive from "
-              + Message.class.getName()
-              + " but "
-              + producedType.getName()
-              + " is provided.");
-    }
-
-    JsonNode json = casted.json();
-
-    Map<String, RoutingConfig> routableTopics = routableTopics(json);
-
-    KafkaIngressBuilder<T> kafkaIngressBuilder = KafkaIngressBuilder.forIdentifier(id);
-    kafkaIngressBuilder
-        .withKafkaAddress(kafkaAddress(json))
-        .withProperties(kafkaClientProperties(json))
-        .addTopics(new ArrayList<>(routableTopics.keySet()));
-
-    optionalConsumerGroupId(json).ifPresent(kafkaIngressBuilder::withConsumerGroupId);
-    optionalAutoOffsetResetPosition(json).ifPresent(kafkaIngressBuilder::withAutoResetPosition);
-    optionalStartupPosition(json).ifPresent(kafkaIngressBuilder::withStartupPosition);
-
-    KafkaIngressBuilderApiExtension.withDeserializer(
-        kafkaIngressBuilder, deserializer(routableTopics));
-
-    return kafkaIngressBuilder.build();
-  }
-
-  @SuppressWarnings("unchecked")
-  private static <T> KafkaIngressDeserializer<T> deserializer(
-      Map<String, RoutingConfig> routingConfig) {
-    // this cast is safe since we've already checked that T is a Message
-    return (KafkaIngressDeserializer<T>)
-        new RoutableProtobufKafkaIngressDeserializer(routingConfig);
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/GenericKafkaEgressBinderV1.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/GenericKafkaEgressBinderV1.java
new file mode 100644
index 0000000..916938e
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/GenericKafkaEgressBinderV1.java
@@ -0,0 +1,90 @@
+/*
+ * 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.io.kafka.binders.egress.v1;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.statefun.extensions.ComponentBinder;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.common.json.StateFunObjectMapper;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.statefun.sdk.egress.generated.KafkaProducerRecord;
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+
+/**
+ * Version 1 {@link ComponentBinder} for binding a Kafka egress which expects {@link
+ * KafkaProducerRecord} as input, and writes the wrapped value bytes to Kafka. Corresponding {@link
+ * TypeName} is {@code io.statefun.kafka.v1/egress}.
+ *
+ * <p>Below is an example YAML document of the {@link ComponentJsonObject} recognized by this
+ * binder, with the expected types of each field:
+ *
+ * <pre>
+ * kind: io.statefun.kafka.v1/egress                                  (typename)
+ * spec:                                                              (object)
+ *   id: com.foo.bar/my-ingress                                       (typename)
+ *   address: kafka-broker:9092                                       (string, optional)
+ *   deliverySemantic:                                                (object, optional)
+ *     type: exactly-once                                             (string)
+ *     transactionTimeout: 15min                                      (duration)
+ *   properties:                                                      (array)
+ *     - foo.config: bar                                              (string)
+ * </pre>
+ *
+ * <p>The {@code deliverySemantic} can be one of the following options: {@code exactly-once}, {@code
+ * at-least-once}, or {@code none}.
+ *
+ * <p>Please see {@link GenericKafkaEgressSpec} for further details.
+ */
+final class GenericKafkaEgressBinderV1 implements ComponentBinder {
+  private static final ObjectMapper SPEC_OBJ_MAPPER = StateFunObjectMapper.create();
+
+  static final GenericKafkaEgressBinderV1 INSTANCE = new GenericKafkaEgressBinderV1();
+
+  static final TypeName KIND_TYPE = TypeName.parseFrom("io.statefun.kafka.v1/egress");
+
+  private GenericKafkaEgressBinderV1() {}
+
+  @Override
+  public void bind(
+      ComponentJsonObject component, StatefulFunctionModule.Binder remoteModuleBinder) {
+    validateComponent(component);
+
+    final JsonNode specJsonNode = component.specJsonNode();
+    final GenericKafkaEgressSpec spec = parseSpec(specJsonNode);
+    remoteModuleBinder.bindEgress(spec.toUniversalKafkaEgressSpec());
+  }
+
+  private static void validateComponent(ComponentJsonObject componentJsonObject) {
+    final TypeName targetBinderType = componentJsonObject.binderTypename();
+    if (!targetBinderType.equals(KIND_TYPE)) {
+      throw new IllegalStateException(
+          "Received unexpected ModuleComponent to bind: " + componentJsonObject);
+    }
+  }
+
+  private static GenericKafkaEgressSpec parseSpec(JsonNode specJsonNode) {
+    try {
+      return SPEC_OBJ_MAPPER.treeToValue(specJsonNode, GenericKafkaEgressSpec.class);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException("Error parsing a GenericKafkaEgressSpec.", e);
+    }
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/GenericKafkaEgressSerializer.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/GenericKafkaEgressSerializer.java
similarity index 93%
rename from statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/GenericKafkaEgressSerializer.java
rename to statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/GenericKafkaEgressSerializer.java
index c232ba3..abdc029 100644
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/GenericKafkaEgressSerializer.java
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/GenericKafkaEgressSerializer.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.statefun.flink.io.kafka;
+package org.apache.flink.statefun.flink.io.kafka.binders.egress.v1;
 
 import com.google.protobuf.InvalidProtocolBufferException;
 import java.nio.charset.StandardCharsets;
@@ -26,8 +26,8 @@
 import org.apache.kafka.clients.producer.ProducerRecord;
 
 /**
- * A {@link KafkaEgressSerializer} used solely by sinks provided by the {@link
- * GenericKafkaSinkProvider}.
+ * A {@link KafkaEgressSerializer} used solely by Kafka egresses bound by {@link
+ * GenericKafkaEgressBinderV1}.
  *
  * <p>This serializer expects Protobuf messages of type {@link KafkaProducerRecord}, and simply
  * transforms those into Kafka's {@link ProducerRecord}.
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/GenericKafkaEgressSpec.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/GenericKafkaEgressSpec.java
new file mode 100644
index 0000000..3c68826
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/GenericKafkaEgressSpec.java
@@ -0,0 +1,147 @@
+/*
+ * 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.io.kafka.binders.egress.v1;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonDeserializer;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.statefun.flink.io.common.json.EgressIdentifierJsonDeserializer;
+import org.apache.flink.statefun.flink.io.common.json.PropertiesJsonDeserializer;
+import org.apache.flink.statefun.sdk.io.EgressIdentifier;
+import org.apache.flink.statefun.sdk.kafka.KafkaEgressBuilder;
+import org.apache.flink.statefun.sdk.kafka.KafkaEgressSpec;
+import org.apache.flink.statefun.sdk.kafka.KafkaProducerSemantic;
+import org.apache.flink.statefun.sdk.reqreply.generated.TypedValue;
+import org.apache.flink.util.TimeUtils;
+
+@JsonDeserialize(builder = GenericKafkaEgressSpec.Builder.class)
+final class GenericKafkaEgressSpec {
+
+  private final EgressIdentifier<TypedValue> id;
+  private final Optional<String> address;
+  private final KafkaProducerSemantic producerSemantic;
+  private final Properties properties;
+
+  private GenericKafkaEgressSpec(
+      EgressIdentifier<TypedValue> id,
+      Optional<String> address,
+      KafkaProducerSemantic producerSemantic,
+      Properties properties) {
+    this.id = Objects.requireNonNull(id);
+    this.address = Objects.requireNonNull(address);
+    this.producerSemantic = Objects.requireNonNull(producerSemantic);
+    this.properties = Objects.requireNonNull(properties);
+  }
+
+  public KafkaEgressSpec<TypedValue> toUniversalKafkaEgressSpec() {
+    final KafkaEgressBuilder<TypedValue> builder = KafkaEgressBuilder.forIdentifier(id);
+    address.ifPresent(builder::withKafkaAddress);
+    builder.withProducerSemantic(producerSemantic);
+    builder.withProperties(properties);
+    builder.withSerializer(GenericKafkaEgressSerializer.class);
+    return builder.build();
+  }
+
+  @JsonPOJOBuilder
+  public static class Builder {
+
+    private final EgressIdentifier<TypedValue> id;
+
+    private Optional<String> kafkaAddress = Optional.empty();
+    private KafkaProducerSemantic producerSemantic = KafkaProducerSemantic.atLeastOnce();
+    private Properties properties = new Properties();
+
+    @JsonCreator
+    private Builder(
+        @JsonProperty("id") @JsonDeserialize(using = EgressIdentifierJsonDeserializer.class)
+            EgressIdentifier<TypedValue> id) {
+      this.id = Objects.requireNonNull(id);
+    }
+
+    @JsonProperty("address")
+    public Builder withKafkaAddress(String address) {
+      Objects.requireNonNull(address);
+      this.kafkaAddress = Optional.of(address);
+      return this;
+    }
+
+    @JsonProperty("deliverySemantic")
+    @JsonDeserialize(using = ProducerSemanticJsonDeserializer.class)
+    public Builder withDeliverySemantic(KafkaProducerSemantic producerSemantic) {
+      this.producerSemantic = Objects.requireNonNull(producerSemantic);
+      return this;
+    }
+
+    @JsonProperty("properties")
+    @JsonDeserialize(using = PropertiesJsonDeserializer.class)
+    public Builder withProperties(Properties properties) {
+      this.properties = Objects.requireNonNull(properties);
+      return this;
+    }
+
+    public GenericKafkaEgressSpec build() {
+      return new GenericKafkaEgressSpec(id, kafkaAddress, producerSemantic, properties);
+    }
+  }
+
+  private static class ProducerSemanticJsonDeserializer
+      extends JsonDeserializer<KafkaProducerSemantic> {
+    @Override
+    public KafkaProducerSemantic deserialize(
+        JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+      final ObjectNode producerSemanticNode = jsonParser.readValueAs(ObjectNode.class);
+      final String semanticTypeString = producerSemanticNode.get("type").asText();
+      switch (semanticTypeString) {
+        case "at-least-once":
+          return KafkaProducerSemantic.atLeastOnce();
+        case "exactly-once":
+          return KafkaProducerSemantic.exactlyOnce(parseTransactionTimeout(producerSemanticNode));
+        case "none":
+          return KafkaProducerSemantic.none();
+        default:
+          throw new IllegalArgumentException(
+              "Invalid delivery semantic type: "
+                  + semanticTypeString
+                  + "; valid types are [at-least-once, exactly-once, none]");
+      }
+    }
+  }
+
+  private static Duration parseTransactionTimeout(ObjectNode producerSemanticNode) {
+    // Prefer deprecated millis based timeout for backwards compatibility
+    // then fallback to duration based configuration.
+    final JsonNode deprecatedTransactionTimeoutMillisNode =
+        producerSemanticNode.get("transactionTimeoutMillis");
+    if (deprecatedTransactionTimeoutMillisNode != null) {
+      return Duration.ofMillis(deprecatedTransactionTimeoutMillisNode.asLong());
+    }
+    return TimeUtils.parseDuration(producerSemanticNode.get("transactionTimeout").asText());
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/Module.java
similarity index 61%
copy from statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
copy to statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/Module.java
index 097c3cf..4baadff 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/Module.java
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.statefun.flink.core.jsonmodule;
 
-import org.apache.flink.statefun.sdk.FunctionType;
+package org.apache.flink.statefun.flink.io.kafka.binders.egress.v1;
 
-public interface FunctionSpec {
+import com.google.auto.service.AutoService;
+import java.util.Map;
+import org.apache.flink.statefun.extensions.ExtensionModule;
 
-  FunctionType functionType();
+@AutoService(ExtensionModule.class)
+public final class Module implements ExtensionModule {
 
-  Kind kind();
-
-  enum Kind {
-    HTTP,
-    GRPC
+  @Override
+  public void configure(Map<String, String> globalConfigurations, Binder universeBinder) {
+    universeBinder.bindExtension(
+        GenericKafkaEgressBinderV1.KIND_TYPE, GenericKafkaEgressBinderV1.INSTANCE);
   }
 }
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/Module.java
similarity index 61%
copy from statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
copy to statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/Module.java
index 097c3cf..e3064b6 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/Module.java
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.statefun.flink.core.jsonmodule;
 
-import org.apache.flink.statefun.sdk.FunctionType;
+package org.apache.flink.statefun.flink.io.kafka.binders.ingress.v1;
 
-public interface FunctionSpec {
+import com.google.auto.service.AutoService;
+import java.util.Map;
+import org.apache.flink.statefun.extensions.ExtensionModule;
 
-  FunctionType functionType();
+@AutoService(ExtensionModule.class)
+public final class Module implements ExtensionModule {
 
-  Kind kind();
-
-  enum Kind {
-    HTTP,
-    GRPC
+  @Override
+  public void configure(Map<String, String> globalConfigurations, Binder universeBinder) {
+    universeBinder.bindExtension(
+        RoutableKafkaIngressBinderV1.KIND_TYPE, RoutableKafkaIngressBinderV1.INSTANCE);
   }
 }
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/RoutableKafkaIngressBinderV1.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/RoutableKafkaIngressBinderV1.java
new file mode 100644
index 0000000..74737d6
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/RoutableKafkaIngressBinderV1.java
@@ -0,0 +1,103 @@
+/*
+ * 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.io.kafka.binders.ingress.v1;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.statefun.extensions.ComponentBinder;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.common.json.StateFunObjectMapper;
+import org.apache.flink.statefun.flink.io.common.AutoRoutableProtobufRouter;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+
+/**
+ * Version 1 {@link ComponentBinder} for binding a Kafka ingress which automatically routes records
+ * to target functions using the record key as the function id. Corresponding {@link TypeName} is
+ * {@code io.statefun.kafka.v1/ingress}.
+ *
+ * <p>Below is an example YAML document of the {@link ComponentJsonObject} recognized by this
+ * binder, with the expected types of each field:
+ *
+ * <pre>
+ * kind: io.statefun.kafka.v1/ingress                                 (typename)
+ * spec:                                                              (object)
+ *   id: com.foo.bar/my-ingress                                       (typename)
+ *   address: kafka-broker:9092                                       (string, optional)
+ *   consumerGroupId: my-group-id                                     (string, optional)
+ *   topics:                                                          (array)
+ *   - topic: topic-1                                                 (string)
+ *     valueType: com.foo.bar/my-type-1                               (typename)
+ *     targets:                                                       (array)
+ *       - com.mycomp.foo/function-1                                  (typename)
+ *       - ...
+ *   - ...
+ *   autoOffsetResetPosition: earliest                                (string, optional)
+ *   startupPosition:                                                 (object)
+ *     type: earliest                                                 (string)
+ *   properties:                                                      (array, optional)
+ *     - foo.config: bar                                              (string)
+ * </pre>
+ *
+ * <p>The {@code autoOffsetResetPosition} can be one of the following options: {@code earliest} or
+ * {@code latest}.
+ *
+ * <p>Furthermore, the {@code startupPosition} can be of one of the following options: {@code
+ * earliest}, {@code latest}, {@code group-offsets}, {@code specific-offsets}, or {@code date}.
+ * Please see {@link RoutableKafkaIngressSpec} for further details.
+ */
+final class RoutableKafkaIngressBinderV1 implements ComponentBinder {
+
+  private static final ObjectMapper SPEC_OBJ_MAPPER = StateFunObjectMapper.create();
+
+  static final RoutableKafkaIngressBinderV1 INSTANCE = new RoutableKafkaIngressBinderV1();
+
+  static final TypeName KIND_TYPE = TypeName.parseFrom("io.statefun.kafka.v1/ingress");
+
+  private RoutableKafkaIngressBinderV1() {}
+
+  @Override
+  public void bind(
+      ComponentJsonObject component, StatefulFunctionModule.Binder remoteModuleBinder) {
+    validateComponent(component);
+
+    final JsonNode specJsonNode = component.specJsonNode();
+    final RoutableKafkaIngressSpec spec = parseSpec(specJsonNode);
+
+    remoteModuleBinder.bindIngress(spec.toUniversalKafkaIngressSpec());
+    remoteModuleBinder.bindIngressRouter(spec.id(), new AutoRoutableProtobufRouter());
+  }
+
+  private static void validateComponent(ComponentJsonObject componentJsonObject) {
+    final TypeName targetBinderType = componentJsonObject.binderTypename();
+    if (!targetBinderType.equals(KIND_TYPE)) {
+      throw new IllegalStateException(
+          "Received unexpected ModuleComponent to bind: " + componentJsonObject);
+    }
+  }
+
+  private static RoutableKafkaIngressSpec parseSpec(JsonNode specJsonNode) {
+    try {
+      return SPEC_OBJ_MAPPER.treeToValue(specJsonNode, RoutableKafkaIngressSpec.class);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException("Error parsing an AutoRoutableKafkaIngressSpec.", e);
+    }
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/RoutableProtobufKafkaIngressDeserializer.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/RoutableKafkaIngressDeserializer.java
similarity index 83%
rename from statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/RoutableProtobufKafkaIngressDeserializer.java
rename to statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/RoutableKafkaIngressDeserializer.java
index 1e7fd7d..1689f46 100644
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/RoutableProtobufKafkaIngressDeserializer.java
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/RoutableKafkaIngressDeserializer.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.statefun.flink.io.kafka;
+package org.apache.flink.statefun.flink.io.kafka.binders.ingress.v1;
 
 import com.google.protobuf.Message;
 import com.google.protobuf.MoreByteStrings;
@@ -23,18 +23,17 @@
 import java.util.Map;
 import org.apache.flink.statefun.flink.io.generated.AutoRoutable;
 import org.apache.flink.statefun.flink.io.generated.RoutingConfig;
-import org.apache.flink.statefun.sdk.IngressType;
-import org.apache.flink.statefun.sdk.kafka.KafkaIngressDeserializer;
+import org.apache.flink.statefun.sdk.TypeName;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 
-public final class RoutableProtobufKafkaIngressDeserializer
-    implements KafkaIngressDeserializer<Message> {
+public final class RoutableKafkaIngressDeserializer
+    implements org.apache.flink.statefun.sdk.kafka.KafkaIngressDeserializer<Message> {
 
   private static final long serialVersionUID = 1L;
 
   private final Map<String, RoutingConfig> routingConfigs;
 
-  RoutableProtobufKafkaIngressDeserializer(Map<String, RoutingConfig> routingConfigs) {
+  public RoutableKafkaIngressDeserializer(Map<String, RoutingConfig> routingConfigs) {
     if (routingConfigs == null || routingConfigs.isEmpty()) {
       throw new IllegalArgumentException(
           "Routing config for routable Kafka ingress cannot be empty.");
@@ -63,12 +62,12 @@
 
   private byte[] requireNonNullKey(byte[] key) {
     if (key == null) {
-      IngressType tpe = ProtobufKafkaIngressTypes.ROUTABLE_PROTOBUF_KAFKA_INGRESS_TYPE;
+      TypeName tpe = RoutableKafkaIngressBinderV1.KIND_TYPE;
       throw new IllegalStateException(
           "The "
               + tpe.namespace()
               + "/"
-              + tpe.type()
+              + tpe.name()
               + " ingress requires a UTF-8 key set for each record.");
     }
     return key;
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/RoutableKafkaIngressSpec.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/RoutableKafkaIngressSpec.java
new file mode 100644
index 0000000..eece38e
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/RoutableKafkaIngressSpec.java
@@ -0,0 +1,292 @@
+/*
+ * 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.io.kafka.binders.ingress.v1;
+
+import com.google.protobuf.Message;
+import java.io.IOException;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeParseException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonDeserializer;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.statefun.flink.io.common.json.IngressIdentifierJsonDeserializer;
+import org.apache.flink.statefun.flink.io.common.json.PropertiesJsonDeserializer;
+import org.apache.flink.statefun.flink.io.generated.RoutingConfig;
+import org.apache.flink.statefun.flink.io.generated.TargetFunctionType;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.statefun.sdk.io.IngressIdentifier;
+import org.apache.flink.statefun.sdk.kafka.KafkaIngressAutoResetPosition;
+import org.apache.flink.statefun.sdk.kafka.KafkaIngressBuilder;
+import org.apache.flink.statefun.sdk.kafka.KafkaIngressBuilderApiExtension;
+import org.apache.flink.statefun.sdk.kafka.KafkaIngressStartupPosition;
+import org.apache.flink.statefun.sdk.kafka.KafkaTopicPartition;
+
+@JsonDeserialize(builder = RoutableKafkaIngressSpec.Builder.class)
+final class RoutableKafkaIngressSpec {
+
+  private final IngressIdentifier<Message> id;
+  private final Optional<String> kafkaAddress;
+  private final Optional<String> consumerGroupId;
+  private final Map<String, RoutingConfig> topicRoutings;
+  private final KafkaIngressAutoResetPosition autoOffsetResetPosition;
+  private final KafkaIngressStartupPosition startupPosition;
+  private final Properties properties;
+
+  private RoutableKafkaIngressSpec(
+      IngressIdentifier<Message> id,
+      Optional<String> kafkaAddress,
+      Optional<String> consumerGroupId,
+      Map<String, RoutingConfig> topicRoutings,
+      KafkaIngressAutoResetPosition autoOffsetResetPosition,
+      KafkaIngressStartupPosition startupPosition,
+      Properties properties) {
+    this.id = id;
+    this.kafkaAddress = kafkaAddress;
+    this.consumerGroupId = consumerGroupId;
+    this.topicRoutings = topicRoutings;
+    this.autoOffsetResetPosition = autoOffsetResetPosition;
+    this.startupPosition = startupPosition;
+    this.properties = properties;
+  }
+
+  public IngressIdentifier<Message> id() {
+    return id;
+  }
+
+  public org.apache.flink.statefun.sdk.kafka.KafkaIngressSpec toUniversalKafkaIngressSpec() {
+    final KafkaIngressBuilder<Message> builder = KafkaIngressBuilder.forIdentifier(id);
+    kafkaAddress.ifPresent(builder::withKafkaAddress);
+    consumerGroupId.ifPresent(builder::withConsumerGroupId);
+    topicRoutings.keySet().forEach(builder::withTopic);
+    builder.withAutoResetPosition(autoOffsetResetPosition);
+    builder.withStartupPosition(startupPosition);
+    builder.withProperties(properties);
+    KafkaIngressBuilderApiExtension.withDeserializer(
+        builder, new RoutableKafkaIngressDeserializer(topicRoutings));
+
+    return builder.build();
+  }
+
+  @JsonPOJOBuilder
+  public static class Builder {
+
+    private final IngressIdentifier<Message> id;
+
+    private Optional<String> kafkaAddress = Optional.empty();
+    private Optional<String> consumerGroupId = Optional.empty();
+    private Map<String, RoutingConfig> topicRoutings = new HashMap<>();
+    private KafkaIngressAutoResetPosition autoOffsetResetPosition =
+        KafkaIngressAutoResetPosition.LATEST;
+    private KafkaIngressStartupPosition startupPosition = KafkaIngressStartupPosition.fromLatest();
+    private Properties properties = new Properties();
+
+    @JsonCreator
+    private Builder(
+        @JsonProperty("id") @JsonDeserialize(using = IngressIdentifierJsonDeserializer.class)
+            IngressIdentifier<Message> id) {
+      this.id = Objects.requireNonNull(id);
+    }
+
+    @JsonProperty("address")
+    public Builder withKafkaAddress(String address) {
+      Objects.requireNonNull(address);
+      this.kafkaAddress = Optional.of(address);
+      return this;
+    }
+
+    @JsonProperty("consumerGroupId")
+    public Builder withConsumerGroupId(String consumerGroupId) {
+      Objects.requireNonNull(consumerGroupId);
+      this.consumerGroupId = Optional.of(consumerGroupId);
+      return this;
+    }
+
+    @JsonProperty("topics")
+    @JsonDeserialize(using = TopicRoutingsJsonDeserializer.class)
+    public Builder withTopicRoutings(Map<String, RoutingConfig> topicRoutings) {
+      this.topicRoutings = Objects.requireNonNull(topicRoutings);
+      return this;
+    }
+
+    @JsonProperty("autoOffsetResetPosition")
+    @JsonDeserialize(using = AutoOffsetResetPositionJsonDeserializer.class)
+    public Builder withAutoOffsetResetPosition(
+        KafkaIngressAutoResetPosition autoOffsetResetPosition) {
+      this.autoOffsetResetPosition = Objects.requireNonNull(autoOffsetResetPosition);
+      return this;
+    }
+
+    @JsonProperty("startupPosition")
+    @JsonDeserialize(using = StartupPositionJsonDeserializer.class)
+    public Builder withStartupPosition(KafkaIngressStartupPosition startupPosition) {
+      this.startupPosition = Objects.requireNonNull(startupPosition);
+      return this;
+    }
+
+    @JsonProperty("properties")
+    @JsonDeserialize(using = PropertiesJsonDeserializer.class)
+    public Builder withProperties(Properties properties) {
+      this.properties = Objects.requireNonNull(properties);
+      return this;
+    }
+
+    public RoutableKafkaIngressSpec build() {
+      return new RoutableKafkaIngressSpec(
+          id,
+          kafkaAddress,
+          consumerGroupId,
+          topicRoutings,
+          autoOffsetResetPosition,
+          startupPosition,
+          properties);
+    }
+  }
+
+  private static class TopicRoutingsJsonDeserializer
+      extends JsonDeserializer<Map<String, RoutingConfig>> {
+    @Override
+    public Map<String, RoutingConfig> deserialize(
+        JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+      final ObjectNode[] routingJsonNodes = jsonParser.readValueAs(ObjectNode[].class);
+
+      final Map<String, RoutingConfig> result = new HashMap<>(routingJsonNodes.length);
+      for (ObjectNode routingJsonNode : routingJsonNodes) {
+        final RoutingConfig routingConfig =
+            RoutingConfig.newBuilder()
+                .setTypeUrl(routingJsonNode.get("valueType").textValue())
+                .addAllTargetFunctionTypes(parseTargetFunctions(routingJsonNode))
+                .build();
+        result.put(routingJsonNode.get("topic").asText(), routingConfig);
+      }
+      return result;
+    }
+  }
+
+  private static class AutoOffsetResetPositionJsonDeserializer
+      extends JsonDeserializer<KafkaIngressAutoResetPosition> {
+    @Override
+    public KafkaIngressAutoResetPosition deserialize(
+        JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+      try {
+        return KafkaIngressAutoResetPosition.valueOf(
+            jsonParser.getText().toUpperCase(Locale.ENGLISH));
+      } catch (IllegalArgumentException e) {
+        throw new IllegalArgumentException(
+            "Invalid autoOffsetResetPosition: "
+                + jsonParser.getText()
+                + "; valid values are "
+                + Arrays.toString(KafkaIngressAutoResetPosition.values()),
+            e);
+      }
+    }
+  }
+
+  private static class StartupPositionJsonDeserializer
+      extends JsonDeserializer<KafkaIngressStartupPosition> {
+    private static final String STARTUP_DATE_PATTERN = "yyyy-MM-dd HH:mm:ss.SSS Z";
+    private static final DateTimeFormatter STARTUP_DATE_FORMATTER =
+        DateTimeFormatter.ofPattern(STARTUP_DATE_PATTERN);
+
+    @Override
+    public KafkaIngressStartupPosition deserialize(
+        JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+      final ObjectNode startupPositionNode = jsonParser.readValueAs(ObjectNode.class);
+      final String startupTypeString = startupPositionNode.get("type").asText();
+      switch (startupTypeString) {
+        case "group-offsets":
+          return KafkaIngressStartupPosition.fromGroupOffsets();
+        case "earliest":
+          return KafkaIngressStartupPosition.fromEarliest();
+        case "latest":
+          return KafkaIngressStartupPosition.fromLatest();
+        case "specific-offsets":
+          return KafkaIngressStartupPosition.fromSpecificOffsets(
+              parseSpecificStartupOffsetsMap(startupPositionNode));
+        case "date":
+          return KafkaIngressStartupPosition.fromDate(parseStartupDate(startupPositionNode));
+        default:
+          throw new IllegalArgumentException(
+              "Invalid startup position type: "
+                  + startupTypeString
+                  + "; valid values are [group-offsets, earliest, latest, specific-offsets, date]");
+      }
+    }
+  }
+
+  private static List<TargetFunctionType> parseTargetFunctions(JsonNode routingJsonNode) {
+    final Iterable<JsonNode> targetFunctionNodes = routingJsonNode.get("targets");
+    return StreamSupport.stream(targetFunctionNodes.spliterator(), false)
+        .map(RoutableKafkaIngressSpec::parseTargetFunctionType)
+        .collect(Collectors.toList());
+  }
+
+  private static TargetFunctionType parseTargetFunctionType(JsonNode targetFunctionNode) {
+    final TypeName targetType = TypeName.parseFrom(targetFunctionNode.asText());
+    return TargetFunctionType.newBuilder()
+        .setNamespace(targetType.namespace())
+        .setType(targetType.name())
+        .build();
+  }
+
+  private static Map<KafkaTopicPartition, Long> parseSpecificStartupOffsetsMap(
+      ObjectNode startupPositionNode) {
+    final Iterable<JsonNode> offsetNodes = startupPositionNode.get("offsets");
+    final Map<KafkaTopicPartition, Long> offsets = new HashMap<>();
+    offsetNodes.forEach(
+        jsonNode -> {
+          Map.Entry<String, JsonNode> offsetNode = jsonNode.fields().next();
+          offsets.put(
+              KafkaTopicPartition.fromString(offsetNode.getKey()), offsetNode.getValue().asLong());
+        });
+    return offsets;
+  }
+
+  private static ZonedDateTime parseStartupDate(ObjectNode startupPositionNode) {
+
+    final String dateString = startupPositionNode.get("date").asText();
+    try {
+      return ZonedDateTime.parse(
+          dateString, StartupPositionJsonDeserializer.STARTUP_DATE_FORMATTER);
+    } catch (DateTimeParseException e) {
+      throw new IllegalArgumentException(
+          "Unable to parse date string for startup position: "
+              + dateString
+              + "; the date should conform to the pattern "
+              + StartupPositionJsonDeserializer.STARTUP_DATE_PATTERN,
+          e);
+    }
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/KinesisFlinkIOModule.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/KinesisFlinkIOModule.java
index fa813fd..c3cbd0a 100644
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/KinesisFlinkIOModule.java
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/KinesisFlinkIOModule.java
@@ -19,8 +19,6 @@
 
 import com.google.auto.service.AutoService;
 import java.util.Map;
-import org.apache.flink.statefun.flink.io.kinesis.polyglot.GenericKinesisSinkProvider;
-import org.apache.flink.statefun.flink.io.kinesis.polyglot.RoutableProtobufKinesisSourceProvider;
 import org.apache.flink.statefun.flink.io.spi.FlinkIoModule;
 import org.apache.flink.statefun.sdk.kinesis.KinesisIOTypes;
 
@@ -31,10 +29,5 @@
   public void configure(Map<String, String> globalConfiguration, Binder binder) {
     binder.bindSourceProvider(KinesisIOTypes.UNIVERSAL_INGRESS_TYPE, new KinesisSourceProvider());
     binder.bindSinkProvider(KinesisIOTypes.UNIVERSAL_EGRESS_TYPE, new KinesisSinkProvider());
-    binder.bindSourceProvider(
-        PolyglotKinesisIOTypes.ROUTABLE_PROTOBUF_KINESIS_INGRESS_TYPE,
-        new RoutableProtobufKinesisSourceProvider());
-    binder.bindSinkProvider(
-        PolyglotKinesisIOTypes.GENERIC_KINESIS_EGRESS_TYPE, new GenericKinesisSinkProvider());
   }
 }
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/AwsCredentialsJsonDeserializer.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/AwsCredentialsJsonDeserializer.java
new file mode 100644
index 0000000..e3d002a
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/AwsCredentialsJsonDeserializer.java
@@ -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://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.io.kinesis.binders;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonDeserializer;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.statefun.sdk.kinesis.auth.AwsCredentials;
+
+public final class AwsCredentialsJsonDeserializer extends JsonDeserializer<AwsCredentials> {
+  private static final String DEFAULT_TYPE = "default";
+  private static final String BASIC_TYPE = "basic";
+  private static final String PROFILE_TYPE = "profile";
+
+  @Override
+  public AwsCredentials deserialize(
+      JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+    final ObjectNode awsCredentialsNode = jsonParser.readValueAs(ObjectNode.class);
+    final String typeString = awsCredentialsNode.get("type").asText();
+
+    switch (typeString) {
+      case DEFAULT_TYPE:
+        return AwsCredentials.fromDefaultProviderChain();
+      case BASIC_TYPE:
+        return AwsCredentials.basic(
+            awsCredentialsNode.get("accessKeyId").asText(),
+            awsCredentialsNode.get("secretAccessKey").asText());
+      case PROFILE_TYPE:
+        final JsonNode pathNode = awsCredentialsNode.get("profilePath");
+        if (pathNode != null) {
+          return AwsCredentials.profile(
+              awsCredentialsNode.get("profileName").asText(), pathNode.asText());
+        } else {
+          return AwsCredentials.profile(awsCredentialsNode.get("profileName").asText());
+        }
+      default:
+        final List<String> validValues = Arrays.asList(DEFAULT_TYPE, BASIC_TYPE, PROFILE_TYPE);
+        throw new IllegalArgumentException(
+            "Invalid AWS credential type: "
+                + typeString
+                + "; valid values are ["
+                + String.join(", ", validValues)
+                + "]");
+    }
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/AwsRegionJsonDeserializer.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/AwsRegionJsonDeserializer.java
new file mode 100644
index 0000000..11b07b4
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/AwsRegionJsonDeserializer.java
@@ -0,0 +1,60 @@
+/*
+ * 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.io.kinesis.binders;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonDeserializer;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.statefun.sdk.kinesis.auth.AwsRegion;
+
+public final class AwsRegionJsonDeserializer extends JsonDeserializer<AwsRegion> {
+  private static final String DEFAULT_TYPE = "default";
+  private static final String SPECIFIED_ID_TYPE = "specific";
+  private static final String CUSTOM_ENDPOINT_TYPE = "custom-endpoint";
+
+  @Override
+  public AwsRegion deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
+      throws IOException {
+    final ObjectNode awsRegionNode = jsonParser.readValueAs(ObjectNode.class);
+    final String typeString = awsRegionNode.get("type").asText();
+
+    switch (typeString) {
+      case DEFAULT_TYPE:
+        return AwsRegion.fromDefaultProviderChain();
+      case SPECIFIED_ID_TYPE:
+        return AwsRegion.ofId(awsRegionNode.get("id").asText());
+      case CUSTOM_ENDPOINT_TYPE:
+        return AwsRegion.ofCustomEndpoint(
+            awsRegionNode.get("endpoint").asText(), awsRegionNode.get("id").asText());
+      default:
+        final List<String> validValues =
+            Arrays.asList(DEFAULT_TYPE, SPECIFIED_ID_TYPE, CUSTOM_ENDPOINT_TYPE);
+        throw new IllegalArgumentException(
+            "Invalid AWS region type: "
+                + typeString
+                + "; valid values are ["
+                + String.join(", ", validValues)
+                + "]");
+    }
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/GenericKinesisEgressBinderV1.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/GenericKinesisEgressBinderV1.java
new file mode 100644
index 0000000..e4a2243
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/GenericKinesisEgressBinderV1.java
@@ -0,0 +1,95 @@
+/*
+ * 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.io.kinesis.binders.egress.v1;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.statefun.extensions.ComponentBinder;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.common.json.StateFunObjectMapper;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.statefun.sdk.egress.generated.KinesisEgressRecord;
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+
+/**
+ * Version 1 {@link ComponentBinder} for binding a Kinesis egress which expects {@link
+ * KinesisEgressRecord} as input, and writes the wrapped value bytes to Kinesis. Corresponding
+ * {@link TypeName} is {@code io.statefun.kinesis.v1/egress}.
+ *
+ * <p>Below is an example YAML document of the {@link ComponentJsonObject} recognized by this
+ * binder, with the expected types of each field:
+ *
+ * <pre>
+ * kind: io.statefun.kinesis.v1/egress                                (typename)
+ * spec:                                                              (object)
+ *   id: com.foo.bar/my-egress                                        (typename)
+ *   awsRegion:                                                       (object, optional)
+ *     type: specific                                                 (string)
+ *     id: us-west-2                                                  (string)
+ *   awsCredentials:                                                  (object, optional)
+ *     type: basic                                                    (string)
+ *     accessKeyId: my_access_key_id                                  (string)
+ *     secretAccessKey: my_secret_access_key                          (string)
+ *   maxOutstandingRecords: 9999                                      (int, optional)
+ *   clientConfigProperties:                                          (array, optional)
+ *     - SocketTimeout: 9999                                          (string)
+ *     - MaxConnections: 15                                           (string)
+ *     - ...
+ * </pre>
+ *
+ * <p>The {@code awsRegion} and {@code awsCredentials} options all have multiple options to choose
+ * from. Please see {@link GenericKinesisEgressSpec} for further details.
+ */
+final class GenericKinesisEgressBinderV1 implements ComponentBinder {
+
+  private static final ObjectMapper SPEC_OBJ_MAPPER = StateFunObjectMapper.create();
+
+  static final GenericKinesisEgressBinderV1 INSTANCE = new GenericKinesisEgressBinderV1();
+
+  static final TypeName KIND_TYPE = TypeName.parseFrom("io.statefun.kinesis.v1/egress");
+
+  private GenericKinesisEgressBinderV1() {}
+
+  @Override
+  public void bind(
+      ComponentJsonObject component, StatefulFunctionModule.Binder remoteModuleBinder) {
+    validateComponent(component);
+
+    final JsonNode specJsonNode = component.specJsonNode();
+    final GenericKinesisEgressSpec spec = parseSpec(specJsonNode);
+    remoteModuleBinder.bindEgress(spec.toUniversalKinesisEgressSpec());
+  }
+
+  private static void validateComponent(ComponentJsonObject componentJsonObject) {
+    final TypeName targetBinderType = componentJsonObject.binderTypename();
+    if (!targetBinderType.equals(KIND_TYPE)) {
+      throw new IllegalStateException(
+          "Received unexpected ModuleComponent to bind: " + componentJsonObject);
+    }
+  }
+
+  private static GenericKinesisEgressSpec parseSpec(JsonNode specJsonNode) {
+    try {
+      return SPEC_OBJ_MAPPER.treeToValue(specJsonNode, GenericKinesisEgressSpec.class);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException("Error parsing a GenericKinesisEgressSpec.", e);
+    }
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/GenericKinesisEgressSerializer.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/GenericKinesisEgressSerializer.java
similarity index 97%
rename from statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/GenericKinesisEgressSerializer.java
rename to statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/GenericKinesisEgressSerializer.java
index 1459b15..2f4bd79 100644
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/GenericKinesisEgressSerializer.java
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/GenericKinesisEgressSerializer.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.statefun.flink.io.kinesis.polyglot;
+package org.apache.flink.statefun.flink.io.kinesis.binders.egress.v1;
 
 import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.flink.statefun.flink.common.types.TypedValueUtil;
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/GenericKinesisEgressSpec.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/GenericKinesisEgressSpec.java
new file mode 100644
index 0000000..90694f1
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/GenericKinesisEgressSpec.java
@@ -0,0 +1,123 @@
+/*
+ * 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.io.kinesis.binders.egress.v1;
+
+import java.util.Objects;
+import java.util.Properties;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder;
+import org.apache.flink.statefun.flink.io.common.json.EgressIdentifierJsonDeserializer;
+import org.apache.flink.statefun.flink.io.common.json.PropertiesJsonDeserializer;
+import org.apache.flink.statefun.flink.io.kinesis.binders.AwsCredentialsJsonDeserializer;
+import org.apache.flink.statefun.flink.io.kinesis.binders.AwsRegionJsonDeserializer;
+import org.apache.flink.statefun.sdk.io.EgressIdentifier;
+import org.apache.flink.statefun.sdk.kinesis.auth.AwsCredentials;
+import org.apache.flink.statefun.sdk.kinesis.auth.AwsRegion;
+import org.apache.flink.statefun.sdk.kinesis.egress.KinesisEgressBuilder;
+import org.apache.flink.statefun.sdk.kinesis.egress.KinesisEgressSpec;
+import org.apache.flink.statefun.sdk.reqreply.generated.TypedValue;
+
+@JsonDeserialize(builder = GenericKinesisEgressSpec.Builder.class)
+public final class GenericKinesisEgressSpec {
+
+  private final EgressIdentifier<TypedValue> id;
+  private final AwsRegion awsRegion;
+  private final AwsCredentials awsCredentials;
+  private final int maxOutstandingRecords;
+  private final Properties properties;
+
+  private GenericKinesisEgressSpec(
+      EgressIdentifier<TypedValue> id,
+      AwsRegion awsRegion,
+      AwsCredentials awsCredentials,
+      int maxOutstandingRecords,
+      Properties properties) {
+    this.id = Objects.requireNonNull(id);
+    this.awsRegion = Objects.requireNonNull(awsRegion);
+    this.awsCredentials = Objects.requireNonNull(awsCredentials);
+    this.maxOutstandingRecords = Objects.requireNonNull(maxOutstandingRecords);
+    this.properties = Objects.requireNonNull(properties);
+  }
+
+  public KinesisEgressSpec<TypedValue> toUniversalKinesisEgressSpec() {
+    final KinesisEgressBuilder<TypedValue> builder =
+        KinesisEgressBuilder.forIdentifier(id)
+            .withAwsRegion(awsRegion)
+            .withAwsCredentials(awsCredentials)
+            .withMaxOutstandingRecords(maxOutstandingRecords)
+            .withProperties(properties)
+            .withSerializer(GenericKinesisEgressSerializer.class);
+    return builder.build();
+  }
+
+  public EgressIdentifier<TypedValue> id() {
+    return id;
+  }
+
+  @JsonPOJOBuilder
+  public static class Builder {
+    private final EgressIdentifier<TypedValue> id;
+
+    private AwsRegion awsRegion = AwsRegion.fromDefaultProviderChain();
+    private AwsCredentials awsCredentials = AwsCredentials.fromDefaultProviderChain();
+    private int maxOutstandingRecords = 1000;
+    private Properties properties = new Properties();
+
+    @JsonCreator
+    private Builder(
+        @JsonProperty("id") @JsonDeserialize(using = EgressIdentifierJsonDeserializer.class)
+            EgressIdentifier<TypedValue> id) {
+      this.id = Objects.requireNonNull(id);
+    }
+
+    @JsonProperty("awsRegion")
+    @JsonDeserialize(using = AwsRegionJsonDeserializer.class)
+    public Builder withAwsRegion(AwsRegion awsRegion) {
+      this.awsRegion = Objects.requireNonNull(awsRegion);
+      return this;
+    }
+
+    @JsonProperty("awsCredentials")
+    @JsonDeserialize(using = AwsCredentialsJsonDeserializer.class)
+    public Builder withAwsCredentials(AwsCredentials awsCredentials) {
+      this.awsCredentials = Objects.requireNonNull(awsCredentials);
+      return this;
+    }
+
+    @JsonProperty("maxOutstandingRecords")
+    public Builder withMaxOutstandingRecords(int maxOutstandingRecords) {
+      this.maxOutstandingRecords = maxOutstandingRecords;
+      return this;
+    }
+
+    @JsonProperty("clientConfigProperties")
+    @JsonDeserialize(using = PropertiesJsonDeserializer.class)
+    public Builder withProperties(Properties properties) {
+      this.properties = Objects.requireNonNull(properties);
+      return this;
+    }
+
+    public GenericKinesisEgressSpec build() {
+      return new GenericKinesisEgressSpec(
+          id, awsRegion, awsCredentials, maxOutstandingRecords, properties);
+    }
+  }
+}
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/Module.java
similarity index 60%
copy from statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
copy to statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/Module.java
index 097c3cf..11e686e 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/Module.java
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.statefun.flink.core.jsonmodule;
 
-import org.apache.flink.statefun.sdk.FunctionType;
+package org.apache.flink.statefun.flink.io.kinesis.binders.egress.v1;
 
-public interface FunctionSpec {
+import com.google.auto.service.AutoService;
+import java.util.Map;
+import org.apache.flink.statefun.extensions.ExtensionModule;
 
-  FunctionType functionType();
+@AutoService(ExtensionModule.class)
+public final class Module implements ExtensionModule {
 
-  Kind kind();
-
-  enum Kind {
-    HTTP,
-    GRPC
+  @Override
+  public void configure(Map<String, String> globalConfigurations, Binder universeBinder) {
+    universeBinder.bindExtension(
+        GenericKinesisEgressBinderV1.KIND_TYPE, GenericKinesisEgressBinderV1.INSTANCE);
   }
 }
diff --git a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/Module.java
similarity index 60%
copy from statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
copy to statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/Module.java
index 097c3cf..6850bdc 100644
--- a/statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/jsonmodule/FunctionSpec.java
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/Module.java
@@ -15,18 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.statefun.flink.core.jsonmodule;
 
-import org.apache.flink.statefun.sdk.FunctionType;
+package org.apache.flink.statefun.flink.io.kinesis.binders.ingress.v1;
 
-public interface FunctionSpec {
+import com.google.auto.service.AutoService;
+import java.util.Map;
+import org.apache.flink.statefun.extensions.ExtensionModule;
 
-  FunctionType functionType();
+@AutoService(ExtensionModule.class)
+public final class Module implements ExtensionModule {
 
-  Kind kind();
-
-  enum Kind {
-    HTTP,
-    GRPC
+  @Override
+  public void configure(Map<String, String> globalConfigurations, Binder universeBinder) {
+    universeBinder.bindExtension(
+        RoutableKinesisIngressBinderV1.KIND_TYPE, RoutableKinesisIngressBinderV1.INSTANCE);
   }
 }
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/RoutableKinesisIngressBinderV1.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/RoutableKinesisIngressBinderV1.java
new file mode 100644
index 0000000..22fc74d
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/RoutableKinesisIngressBinderV1.java
@@ -0,0 +1,106 @@
+/*
+ * 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.io.kinesis.binders.ingress.v1;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.statefun.extensions.ComponentBinder;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.common.json.StateFunObjectMapper;
+import org.apache.flink.statefun.flink.io.common.AutoRoutableProtobufRouter;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+
+/**
+ * Version 1 {@link ComponentBinder} for binding a Kinesis ingress which automatically routes
+ * records to target functions using the record key as the function id. Corresponding {@link
+ * TypeName} is {@code io.statefun.kinesis.v1/ingress}.
+ *
+ * <p>Below is an example YAML document of the {@link ComponentJsonObject} recognized by this
+ * binder, with the expected types of each field:
+ *
+ * <pre>
+ * kind: io.statefun.kinesis.v1/ingress                               (typename)
+ * spec:                                                              (object)
+ *   id: com.foo.bar/my-ingress                                       (typename)
+ *   awsRegion:                                                       (object, optional)
+ *     type: specific                                                 (string)
+ *     id: us-west-2                                                  (string)
+ *   awsCredentials:                                                  (object, optional)
+ *     type: basic                                                    (string)
+ *     accessKeyId: my_access_key_id                                  (string)
+ *     secretAccessKey: my_secret_access_key                          (string)
+ *   startupPosition:                                                 (object, optional)
+ *     type: earliest                                                 (string)
+ *   streams:                                                         (array)
+ *     - stream: stream-1                                             (string)
+ *       valueType: com.foo.bar/my-type-1                             (typename)
+ *       targets:                                                     (array)
+ *         - com.mycomp.foo/function-1                                (typename)
+ *         - ...
+ *     - ...
+ *   clientConfigProperties:                                          (array, optional)
+ *     - SocketTimeout: 9999                                          (string)
+ *     - MaxConnections: 15                                           (string)
+ *     - ...
+ * </pre>
+ *
+ * <p>The {@code awsRegion}, {@code awsCredentials}, {@code startupPosition} options all have
+ * multiple options to choose from. Please see {@link RoutableKinesisIngressSpec} for further
+ * details.
+ */
+final class RoutableKinesisIngressBinderV1 implements ComponentBinder {
+
+  private static final ObjectMapper SPEC_OBJ_MAPPER = StateFunObjectMapper.create();
+
+  static final RoutableKinesisIngressBinderV1 INSTANCE = new RoutableKinesisIngressBinderV1();
+
+  static final TypeName KIND_TYPE = TypeName.parseFrom("io.statefun.kinesis.v1/ingress");
+
+  private RoutableKinesisIngressBinderV1() {}
+
+  @Override
+  public void bind(
+      ComponentJsonObject component, StatefulFunctionModule.Binder remoteModuleBinder) {
+    validateComponent(component);
+
+    final JsonNode specJsonNode = component.specJsonNode();
+    final RoutableKinesisIngressSpec spec = parseSpec(specJsonNode);
+
+    remoteModuleBinder.bindIngress(spec.toUniversalKinesisIngressSpec());
+    remoteModuleBinder.bindIngressRouter(spec.id(), new AutoRoutableProtobufRouter());
+  }
+
+  private static void validateComponent(ComponentJsonObject componentJsonObject) {
+    final TypeName targetBinderType = componentJsonObject.binderTypename();
+    if (!targetBinderType.equals(KIND_TYPE)) {
+      throw new IllegalStateException(
+          "Received unexpected ModuleComponent to bind: " + componentJsonObject);
+    }
+  }
+
+  private static RoutableKinesisIngressSpec parseSpec(JsonNode specJsonNode) {
+    try {
+      return SPEC_OBJ_MAPPER.treeToValue(specJsonNode, RoutableKinesisIngressSpec.class);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException("Error parsing an AutoRoutableKinesisIngressSpec.", e);
+    }
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/RoutableProtobufKinesisIngressDeserializer.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/RoutableKinesisIngressDeserializer.java
similarity index 84%
rename from statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/RoutableProtobufKinesisIngressDeserializer.java
rename to statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/RoutableKinesisIngressDeserializer.java
index 7affc2d..aa6f79b 100644
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/RoutableProtobufKinesisIngressDeserializer.java
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/RoutableKinesisIngressDeserializer.java
@@ -15,26 +15,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.statefun.flink.io.kinesis.polyglot;
+package org.apache.flink.statefun.flink.io.kinesis.binders.ingress.v1;
 
 import com.google.protobuf.Message;
 import com.google.protobuf.MoreByteStrings;
 import java.util.Map;
 import org.apache.flink.statefun.flink.io.generated.AutoRoutable;
 import org.apache.flink.statefun.flink.io.generated.RoutingConfig;
-import org.apache.flink.statefun.flink.io.kinesis.PolyglotKinesisIOTypes;
-import org.apache.flink.statefun.sdk.IngressType;
+import org.apache.flink.statefun.sdk.TypeName;
 import org.apache.flink.statefun.sdk.kinesis.ingress.IngressRecord;
 import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressDeserializer;
 
-public final class RoutableProtobufKinesisIngressDeserializer
+public final class RoutableKinesisIngressDeserializer
     implements KinesisIngressDeserializer<Message> {
 
   private static final long serialVersionUID = 1L;
 
   private final Map<String, RoutingConfig> routingConfigs;
 
-  RoutableProtobufKinesisIngressDeserializer(Map<String, RoutingConfig> routingConfigs) {
+  public RoutableKinesisIngressDeserializer(Map<String, RoutingConfig> routingConfigs) {
     if (routingConfigs == null || routingConfigs.isEmpty()) {
       throw new IllegalArgumentException(
           "Routing config for routable Kinesis ingress cannot be empty.");
@@ -62,12 +61,12 @@
 
   private String requireNonNullKey(String partitionKey) {
     if (partitionKey == null) {
-      IngressType tpe = PolyglotKinesisIOTypes.ROUTABLE_PROTOBUF_KINESIS_INGRESS_TYPE;
+      TypeName tpe = RoutableKinesisIngressBinderV1.KIND_TYPE;
       throw new IllegalStateException(
           "The "
               + tpe.namespace()
               + "/"
-              + tpe.type()
+              + tpe.name()
               + " ingress requires a UTF-8 partition key set for each stream record.");
     }
     return partitionKey;
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/RoutableKinesisIngressSpec.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/RoutableKinesisIngressSpec.java
new file mode 100644
index 0000000..e0bb1f0
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/RoutableKinesisIngressSpec.java
@@ -0,0 +1,243 @@
+/*
+ * 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.io.kinesis.binders.ingress.v1;
+
+import com.google.protobuf.Message;
+import java.io.IOException;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeParseException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonDeserializer;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.statefun.flink.io.common.json.IngressIdentifierJsonDeserializer;
+import org.apache.flink.statefun.flink.io.common.json.PropertiesJsonDeserializer;
+import org.apache.flink.statefun.flink.io.generated.RoutingConfig;
+import org.apache.flink.statefun.flink.io.generated.TargetFunctionType;
+import org.apache.flink.statefun.flink.io.kinesis.binders.AwsCredentialsJsonDeserializer;
+import org.apache.flink.statefun.flink.io.kinesis.binders.AwsRegionJsonDeserializer;
+import org.apache.flink.statefun.sdk.TypeName;
+import org.apache.flink.statefun.sdk.io.IngressIdentifier;
+import org.apache.flink.statefun.sdk.kinesis.auth.AwsCredentials;
+import org.apache.flink.statefun.sdk.kinesis.auth.AwsRegion;
+import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressBuilder;
+import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressBuilderApiExtension;
+import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressSpec;
+import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressStartupPosition;
+
+@JsonDeserialize(builder = RoutableKinesisIngressSpec.Builder.class)
+public class RoutableKinesisIngressSpec {
+
+  private final IngressIdentifier<Message> id;
+  private final AwsRegion awsRegion;
+  private final AwsCredentials awsCredentials;
+  private final KinesisIngressStartupPosition startupPosition;
+  private final Map<String, RoutingConfig> streamRoutings;
+  private final Properties properties;
+
+  private RoutableKinesisIngressSpec(
+      IngressIdentifier<Message> id,
+      AwsRegion awsRegion,
+      AwsCredentials awsCredentials,
+      KinesisIngressStartupPosition startupPosition,
+      Map<String, RoutingConfig> streamRoutings,
+      Properties properties) {
+    this.id = Objects.requireNonNull(id);
+    this.awsRegion = Objects.requireNonNull(awsRegion);
+    this.awsCredentials = Objects.requireNonNull(awsCredentials);
+    this.startupPosition = Objects.requireNonNull(startupPosition);
+    this.streamRoutings = Objects.requireNonNull(streamRoutings);
+    this.properties = Objects.requireNonNull(properties);
+  }
+
+  public KinesisIngressSpec<Message> toUniversalKinesisIngressSpec() {
+    final KinesisIngressBuilder<Message> builder =
+        KinesisIngressBuilder.forIdentifier(id)
+            .withAwsRegion(awsRegion)
+            .withAwsCredentials(awsCredentials)
+            .withStartupPosition(startupPosition)
+            .withProperties(properties);
+    streamRoutings.keySet().forEach(builder::withStream);
+    KinesisIngressBuilderApiExtension.withDeserializer(
+        builder, new RoutableKinesisIngressDeserializer(streamRoutings));
+
+    return builder.build();
+  }
+
+  public IngressIdentifier<Message> id() {
+    return id;
+  }
+
+  @JsonPOJOBuilder
+  public static class Builder {
+    private final IngressIdentifier<Message> id;
+
+    private AwsRegion awsRegion = AwsRegion.fromDefaultProviderChain();
+    private AwsCredentials awsCredentials = AwsCredentials.fromDefaultProviderChain();
+    private KinesisIngressStartupPosition startupPosition =
+        KinesisIngressStartupPosition.fromLatest();
+    private Map<String, RoutingConfig> streamRoutings = new HashMap<>();
+    private Properties properties = new Properties();
+
+    @JsonCreator
+    private Builder(
+        @JsonProperty("id") @JsonDeserialize(using = IngressIdentifierJsonDeserializer.class)
+            IngressIdentifier<Message> id) {
+      this.id = Objects.requireNonNull(id);
+    }
+
+    @JsonProperty("awsRegion")
+    @JsonDeserialize(using = AwsRegionJsonDeserializer.class)
+    public Builder withAwsRegion(AwsRegion awsRegion) {
+      this.awsRegion = Objects.requireNonNull(awsRegion);
+      return this;
+    }
+
+    @JsonProperty("awsCredentials")
+    @JsonDeserialize(using = AwsCredentialsJsonDeserializer.class)
+    public Builder withAwsCredentials(AwsCredentials awsCredentials) {
+      this.awsCredentials = Objects.requireNonNull(awsCredentials);
+      return this;
+    }
+
+    @JsonProperty("startupPosition")
+    @JsonDeserialize(using = StartupPositionJsonDeserializer.class)
+    public Builder withStartupPosition(KinesisIngressStartupPosition startupPosition) {
+      this.startupPosition = Objects.requireNonNull(startupPosition);
+      return this;
+    }
+
+    @JsonProperty("streams")
+    @JsonDeserialize(using = StreamRoutingsJsonDeserializer.class)
+    public Builder withStreamRoutings(Map<String, RoutingConfig> streamRoutings) {
+      this.streamRoutings = Objects.requireNonNull(streamRoutings);
+      return this;
+    }
+
+    @JsonProperty("clientConfigProperties")
+    @JsonDeserialize(using = PropertiesJsonDeserializer.class)
+    public Builder withProperties(Properties properties) {
+      this.properties = Objects.requireNonNull(properties);
+      return this;
+    }
+
+    public RoutableKinesisIngressSpec build() {
+      return new RoutableKinesisIngressSpec(
+          id, awsRegion, awsCredentials, startupPosition, streamRoutings, properties);
+    }
+  }
+
+  private static class StreamRoutingsJsonDeserializer
+      extends JsonDeserializer<Map<String, RoutingConfig>> {
+    @Override
+    public Map<String, RoutingConfig> deserialize(
+        JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+      final ObjectNode[] routingJsonNodes = jsonParser.readValueAs(ObjectNode[].class);
+
+      final Map<String, RoutingConfig> result = new HashMap<>(routingJsonNodes.length);
+      for (ObjectNode routingJsonNode : routingJsonNodes) {
+        final RoutingConfig routingConfig =
+            RoutingConfig.newBuilder()
+                .setTypeUrl(routingJsonNode.get("valueType").textValue())
+                .addAllTargetFunctionTypes(parseTargetFunctions(routingJsonNode))
+                .build();
+        result.put(routingJsonNode.get("stream").asText(), routingConfig);
+      }
+      return result;
+    }
+  }
+
+  private static class StartupPositionJsonDeserializer
+      extends JsonDeserializer<KinesisIngressStartupPosition> {
+    private static final String EARLIEST_TYPE = "earliest";
+    private static final String LATEST_TYPE = "latest";
+    private static final String DATE_TYPE = "date";
+
+    private static final String DATE_PATTERN = "yyyy-MM-dd HH:mm:ss.SSS Z";
+    private static final DateTimeFormatter DATE_FORMATTER =
+        DateTimeFormatter.ofPattern(DATE_PATTERN);
+
+    @Override
+    public KinesisIngressStartupPosition deserialize(
+        JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException {
+      final ObjectNode startupPositionNode = jsonParser.readValueAs(ObjectNode.class);
+      final String startupTypeString = startupPositionNode.get("type").asText();
+      switch (startupTypeString) {
+        case EARLIEST_TYPE:
+          return KinesisIngressStartupPosition.fromEarliest();
+        case LATEST_TYPE:
+          return KinesisIngressStartupPosition.fromLatest();
+        case DATE_TYPE:
+          return KinesisIngressStartupPosition.fromDate(parseStartupDate(startupPositionNode));
+        default:
+          final List<String> validValues = Arrays.asList(EARLIEST_TYPE, LATEST_TYPE, DATE_TYPE);
+          throw new IllegalArgumentException(
+              "Invalid startup position type: "
+                  + startupTypeString
+                  + "; valid values are ["
+                  + String.join(", ", validValues)
+                  + "]");
+      }
+    }
+  }
+
+  private static List<TargetFunctionType> parseTargetFunctions(JsonNode routingJsonNode) {
+    final Iterable<JsonNode> targetFunctionNodes = routingJsonNode.get("targets");
+    return StreamSupport.stream(targetFunctionNodes.spliterator(), false)
+        .map(RoutableKinesisIngressSpec::parseTargetFunctionType)
+        .collect(Collectors.toList());
+  }
+
+  private static TargetFunctionType parseTargetFunctionType(JsonNode targetFunctionNode) {
+    final TypeName targetType = TypeName.parseFrom(targetFunctionNode.asText());
+    return TargetFunctionType.newBuilder()
+        .setNamespace(targetType.namespace())
+        .setType(targetType.name())
+        .build();
+  }
+
+  private static ZonedDateTime parseStartupDate(ObjectNode startupPositionNode) {
+
+    final String dateString = startupPositionNode.get("date").asText();
+    try {
+      return ZonedDateTime.parse(dateString, StartupPositionJsonDeserializer.DATE_FORMATTER);
+    } catch (DateTimeParseException e) {
+      throw new IllegalArgumentException(
+          "Unable to parse date string for startup position: "
+              + dateString
+              + "; the date should conform to the pattern "
+              + StartupPositionJsonDeserializer.DATE_PATTERN,
+          e);
+    }
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/AwsAuthSpecJsonParser.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/AwsAuthSpecJsonParser.java
deleted file mode 100644
index 47c26e4..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/AwsAuthSpecJsonParser.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * 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.io.kinesis.polyglot;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.Optional;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.common.json.Selectors;
-import org.apache.flink.statefun.sdk.kinesis.auth.AwsCredentials;
-import org.apache.flink.statefun.sdk.kinesis.auth.AwsRegion;
-
-final class AwsAuthSpecJsonParser {
-
-  private AwsAuthSpecJsonParser() {}
-
-  private static final JsonPointer AWS_REGION_POINTER = JsonPointer.compile("/awsRegion");
-  private static final JsonPointer AWS_CREDENTIALS_POINTER = JsonPointer.compile("/awsCredentials");
-
-  private static final class Region {
-    private static final String DEFAULT_TYPE = "default";
-    private static final String SPECIFIED_ID_TYPE = "specific";
-    private static final String CUSTOM_ENDPOINT_TYPE = "custom-endpoint";
-
-    private static final JsonPointer TYPE_POINTER = JsonPointer.compile("/type");
-    private static final JsonPointer ID_POINTER = JsonPointer.compile("/id");
-    private static final JsonPointer ENDPOINT_POINTER = JsonPointer.compile("/endpoint");
-  }
-
-  private static final class Credentials {
-    private static final String DEFAULT_TYPE = "default";
-    private static final String BASIC_TYPE = "basic";
-    private static final String PROFILE_TYPE = "profile";
-
-    private static final JsonPointer TYPE_POINTER = JsonPointer.compile("/type");
-    private static final JsonPointer ACCESS_KEY_ID_POINTER = JsonPointer.compile("/accessKeyId");
-    private static final JsonPointer SECRET_ACCESS_KEY_POINTER =
-        JsonPointer.compile("/secretAccessKey");
-    private static final JsonPointer PROFILE_NAME_POINTER = JsonPointer.compile("/profileName");
-    private static final JsonPointer PROFILE_PATH_POINTER = JsonPointer.compile("/profilePath");
-  }
-
-  static Optional<AwsRegion> optionalAwsRegion(JsonNode specNode) {
-    final JsonNode awsRegionSpecNode = specNode.at(AWS_REGION_POINTER);
-    if (awsRegionSpecNode.isMissingNode()) {
-      return Optional.empty();
-    }
-
-    final String type = Selectors.textAt(awsRegionSpecNode, Region.TYPE_POINTER);
-    switch (type) {
-      case Region.DEFAULT_TYPE:
-        return Optional.of(AwsRegion.fromDefaultProviderChain());
-      case Region.SPECIFIED_ID_TYPE:
-        return Optional.of(AwsRegion.ofId(Selectors.textAt(awsRegionSpecNode, Region.ID_POINTER)));
-      case Region.CUSTOM_ENDPOINT_TYPE:
-        return Optional.of(
-            AwsRegion.ofCustomEndpoint(
-                Selectors.textAt(awsRegionSpecNode, Region.ENDPOINT_POINTER),
-                Selectors.textAt(awsRegionSpecNode, Region.ID_POINTER)));
-      default:
-        final List<String> validValues =
-            Arrays.asList(
-                Region.DEFAULT_TYPE, Region.SPECIFIED_ID_TYPE, Region.CUSTOM_ENDPOINT_TYPE);
-        throw new IllegalArgumentException(
-            "Invalid AWS region type: "
-                + type
-                + "; valid values are ["
-                + String.join(", ", validValues)
-                + "]");
-    }
-  }
-
-  static Optional<AwsCredentials> optionalAwsCredentials(JsonNode specNode) {
-    final JsonNode awsCredentialsSpecNode = specNode.at(AWS_CREDENTIALS_POINTER);
-    if (awsCredentialsSpecNode.isMissingNode()) {
-      return Optional.empty();
-    }
-
-    final String type = Selectors.textAt(awsCredentialsSpecNode, Credentials.TYPE_POINTER);
-    switch (type) {
-      case Credentials.DEFAULT_TYPE:
-        return Optional.of(AwsCredentials.fromDefaultProviderChain());
-      case Credentials.BASIC_TYPE:
-        return Optional.of(
-            AwsCredentials.basic(
-                Selectors.textAt(awsCredentialsSpecNode, Credentials.ACCESS_KEY_ID_POINTER),
-                Selectors.textAt(awsCredentialsSpecNode, Credentials.SECRET_ACCESS_KEY_POINTER)));
-      case Credentials.PROFILE_TYPE:
-        final Optional<String> path =
-            Selectors.optionalTextAt(awsCredentialsSpecNode, Credentials.PROFILE_PATH_POINTER);
-        if (path.isPresent()) {
-          return Optional.of(
-              AwsCredentials.profile(
-                  Selectors.textAt(awsCredentialsSpecNode, Credentials.PROFILE_NAME_POINTER),
-                  path.get()));
-        } else {
-          return Optional.of(
-              AwsCredentials.profile(
-                  Selectors.textAt(awsCredentialsSpecNode, Credentials.PROFILE_NAME_POINTER)));
-        }
-      default:
-        final List<String> validValues =
-            Arrays.asList(
-                Credentials.DEFAULT_TYPE, Credentials.BASIC_TYPE, Credentials.PROFILE_TYPE);
-        throw new IllegalArgumentException(
-            "Invalid AWS credential type: "
-                + type
-                + "; valid values are ["
-                + String.join(", ", validValues)
-                + "]");
-    }
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/GenericKinesisSinkProvider.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/GenericKinesisSinkProvider.java
deleted file mode 100644
index d5f5f29..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/GenericKinesisSinkProvider.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.io.kinesis.polyglot;
-
-import static org.apache.flink.statefun.flink.io.kinesis.polyglot.AwsAuthSpecJsonParser.optionalAwsCredentials;
-import static org.apache.flink.statefun.flink.io.kinesis.polyglot.AwsAuthSpecJsonParser.optionalAwsRegion;
-import static org.apache.flink.statefun.flink.io.kinesis.polyglot.KinesisEgressSpecJsonParser.clientConfigProperties;
-import static org.apache.flink.statefun.flink.io.kinesis.polyglot.KinesisEgressSpecJsonParser.optionalMaxOutstandingRecords;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.io.kinesis.KinesisSinkProvider;
-import org.apache.flink.statefun.flink.io.spi.JsonEgressSpec;
-import org.apache.flink.statefun.flink.io.spi.SinkProvider;
-import org.apache.flink.statefun.sdk.io.EgressIdentifier;
-import org.apache.flink.statefun.sdk.io.EgressSpec;
-import org.apache.flink.statefun.sdk.kinesis.egress.KinesisEgressBuilder;
-import org.apache.flink.statefun.sdk.kinesis.egress.KinesisEgressSpec;
-import org.apache.flink.statefun.sdk.reqreply.generated.TypedValue;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-
-public final class GenericKinesisSinkProvider implements SinkProvider {
-
-  private final KinesisSinkProvider delegateProvider = new KinesisSinkProvider();
-
-  @Override
-  public <T> SinkFunction<T> forSpec(EgressSpec<T> spec) {
-    final KinesisEgressSpec<T> kinesisEgressSpec = asKinesisEgressSpec(spec);
-    return delegateProvider.forSpec(kinesisEgressSpec);
-  }
-
-  private static <T> KinesisEgressSpec<T> asKinesisEgressSpec(EgressSpec<T> spec) {
-    if (!(spec instanceof JsonEgressSpec)) {
-      throw new IllegalArgumentException("Wrong type " + spec.type());
-    }
-    JsonEgressSpec<T> casted = (JsonEgressSpec<T>) spec;
-
-    EgressIdentifier<T> id = casted.id();
-    validateConsumedType(id);
-
-    JsonNode specJson = casted.specJson();
-
-    KinesisEgressBuilder<T> kinesisEgressBuilder = KinesisEgressBuilder.forIdentifier(id);
-
-    optionalAwsRegion(specJson).ifPresent(kinesisEgressBuilder::withAwsRegion);
-    optionalAwsCredentials(specJson).ifPresent(kinesisEgressBuilder::withAwsCredentials);
-    optionalMaxOutstandingRecords(specJson)
-        .ifPresent(kinesisEgressBuilder::withMaxOutstandingRecords);
-    clientConfigProperties(specJson)
-        .entrySet()
-        .forEach(
-            entry ->
-                kinesisEgressBuilder.withClientConfigurationProperty(
-                    entry.getKey(), entry.getValue()));
-
-    kinesisEgressBuilder.withSerializer(serializerClass());
-
-    return kinesisEgressBuilder.build();
-  }
-
-  private static void validateConsumedType(EgressIdentifier<?> id) {
-    Class<?> consumedType = id.consumedType();
-    if (TypedValue.class != consumedType) {
-      throw new IllegalArgumentException(
-          "Generic Kinesis egress is only able to consume messages types of "
-              + TypedValue.class.getName()
-              + " but "
-              + consumedType.getName()
-              + " is provided.");
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  private static <T> Class<T> serializerClass() {
-    // this cast is safe, because we've already validated that the consumed type is Any.
-    return (Class<T>) GenericKinesisEgressSerializer.class;
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/KinesisEgressSpecJsonParser.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/KinesisEgressSpecJsonParser.java
deleted file mode 100644
index 594623c..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/KinesisEgressSpecJsonParser.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.io.kinesis.polyglot;
-
-import java.util.Map;
-import java.util.OptionalInt;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.common.json.Selectors;
-
-final class KinesisEgressSpecJsonParser {
-
-  private KinesisEgressSpecJsonParser() {}
-
-  private static final JsonPointer MAX_OUTSTANDING_RECORDS_POINTER =
-      JsonPointer.compile("/maxOutstandingRecords");
-  private static final JsonPointer CLIENT_CONFIG_PROPS_POINTER =
-      JsonPointer.compile("/clientConfigProperties");
-
-  static OptionalInt optionalMaxOutstandingRecords(JsonNode ingressSpecNode) {
-    return Selectors.optionalIntegerAt(ingressSpecNode, MAX_OUTSTANDING_RECORDS_POINTER);
-  }
-
-  static Map<String, String> clientConfigProperties(JsonNode ingressSpecNode) {
-    return Selectors.propertiesAt(ingressSpecNode, CLIENT_CONFIG_PROPS_POINTER);
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/KinesisIngressSpecJsonParser.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/KinesisIngressSpecJsonParser.java
deleted file mode 100644
index 129a2bb..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/KinesisIngressSpecJsonParser.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * 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.io.kinesis.polyglot;
-
-import java.time.ZonedDateTime;
-import java.time.format.DateTimeFormatter;
-import java.time.format.DateTimeParseException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.common.json.NamespaceNamePair;
-import org.apache.flink.statefun.flink.common.json.Selectors;
-import org.apache.flink.statefun.flink.io.generated.RoutingConfig;
-import org.apache.flink.statefun.flink.io.generated.TargetFunctionType;
-import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressStartupPosition;
-
-final class KinesisIngressSpecJsonParser {
-
-  private KinesisIngressSpecJsonParser() {}
-
-  private static final JsonPointer STREAMS_POINTER = JsonPointer.compile("/streams");
-  private static final JsonPointer STARTUP_POSITION_POINTER =
-      JsonPointer.compile("/startupPosition");
-  private static final JsonPointer CLIENT_CONFIG_PROPS_POINTER =
-      JsonPointer.compile("/clientConfigProperties");
-
-  private static final class Streams {
-    private static final JsonPointer NAME_POINTER = JsonPointer.compile("/stream");
-    private static final JsonPointer VALUE_TYPE_POINTER = JsonPointer.compile("/valueType");
-    private static final JsonPointer TARGETS_POINTER = JsonPointer.compile("/targets");
-  }
-
-  private static final class StartupPosition {
-    private static final String EARLIEST_TYPE = "earliest";
-    private static final String LATEST_TYPE = "latest";
-    private static final String DATE_TYPE = "date";
-
-    private static final String DATE_PATTERN = "yyyy-MM-dd HH:mm:ss.SSS Z";
-    private static final DateTimeFormatter DATE_FORMATTER =
-        DateTimeFormatter.ofPattern(DATE_PATTERN);
-
-    private static final JsonPointer TYPE_POINTER = JsonPointer.compile("/type");
-    private static final JsonPointer DATE_POINTER = JsonPointer.compile("/date");
-  }
-
-  static Optional<KinesisIngressStartupPosition> optionalStartupPosition(JsonNode ingressSpecNode) {
-    final JsonNode startupPositionSpecNode = ingressSpecNode.at(STARTUP_POSITION_POINTER);
-    if (startupPositionSpecNode.isMissingNode()) {
-      return Optional.empty();
-    }
-
-    final String type = Selectors.textAt(startupPositionSpecNode, StartupPosition.TYPE_POINTER);
-    switch (type) {
-      case StartupPosition.EARLIEST_TYPE:
-        return Optional.of(KinesisIngressStartupPosition.fromEarliest());
-      case StartupPosition.LATEST_TYPE:
-        return Optional.of(KinesisIngressStartupPosition.fromLatest());
-      case StartupPosition.DATE_TYPE:
-        return Optional.of(
-            KinesisIngressStartupPosition.fromDate(startupDate(startupPositionSpecNode)));
-      default:
-        final List<String> validValues =
-            Arrays.asList(
-                StartupPosition.EARLIEST_TYPE,
-                StartupPosition.LATEST_TYPE,
-                StartupPosition.DATE_TYPE);
-        throw new IllegalArgumentException(
-            "Invalid startup position type: "
-                + type
-                + "; valid values are ["
-                + String.join(", ", validValues)
-                + "]");
-    }
-  }
-
-  static Map<String, String> clientConfigProperties(JsonNode ingressSpecNode) {
-    return Selectors.propertiesAt(ingressSpecNode, CLIENT_CONFIG_PROPS_POINTER);
-  }
-
-  static Map<String, RoutingConfig> routableStreams(JsonNode ingressSpecNode) {
-    Map<String, RoutingConfig> routableStreams = new HashMap<>();
-    for (JsonNode routableStreamNode : Selectors.listAt(ingressSpecNode, STREAMS_POINTER)) {
-      final String streamName = Selectors.textAt(routableStreamNode, Streams.NAME_POINTER);
-      final String typeUrl = Selectors.textAt(routableStreamNode, Streams.VALUE_TYPE_POINTER);
-      final List<TargetFunctionType> targets = parseRoutableTargetFunctionTypes(routableStreamNode);
-
-      routableStreams.put(
-          streamName,
-          RoutingConfig.newBuilder()
-              .setTypeUrl(typeUrl)
-              .addAllTargetFunctionTypes(targets)
-              .build());
-    }
-    return routableStreams;
-  }
-
-  private static List<TargetFunctionType> parseRoutableTargetFunctionTypes(
-      JsonNode routableStreamNode) {
-    final List<TargetFunctionType> targets = new ArrayList<>();
-    for (String namespaceAndName :
-        Selectors.textListAt(routableStreamNode, Streams.TARGETS_POINTER)) {
-      NamespaceNamePair namespaceNamePair = NamespaceNamePair.from(namespaceAndName);
-      targets.add(
-          TargetFunctionType.newBuilder()
-              .setNamespace(namespaceNamePair.namespace())
-              .setType(namespaceNamePair.name())
-              .build());
-    }
-    return targets;
-  }
-
-  private static ZonedDateTime startupDate(JsonNode startupPositionSpecNode) {
-    final String dateStr = Selectors.textAt(startupPositionSpecNode, StartupPosition.DATE_POINTER);
-    try {
-      return ZonedDateTime.parse(dateStr, StartupPosition.DATE_FORMATTER);
-    } catch (DateTimeParseException e) {
-      throw new IllegalArgumentException(
-          "Unable to parse date string for startup position: "
-              + dateStr
-              + "; the date should conform to the pattern "
-              + StartupPosition.DATE_PATTERN,
-          e);
-    }
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/RoutableProtobufKinesisSourceProvider.java b/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/RoutableProtobufKinesisSourceProvider.java
deleted file mode 100644
index 99776a3..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/main/java/org/apache/flink/statefun/flink/io/kinesis/polyglot/RoutableProtobufKinesisSourceProvider.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * 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.io.kinesis.polyglot;
-
-import static org.apache.flink.statefun.flink.io.kinesis.polyglot.AwsAuthSpecJsonParser.optionalAwsCredentials;
-import static org.apache.flink.statefun.flink.io.kinesis.polyglot.AwsAuthSpecJsonParser.optionalAwsRegion;
-import static org.apache.flink.statefun.flink.io.kinesis.polyglot.KinesisIngressSpecJsonParser.clientConfigProperties;
-import static org.apache.flink.statefun.flink.io.kinesis.polyglot.KinesisIngressSpecJsonParser.optionalStartupPosition;
-import static org.apache.flink.statefun.flink.io.kinesis.polyglot.KinesisIngressSpecJsonParser.routableStreams;
-
-import com.google.protobuf.Message;
-import java.util.ArrayList;
-import java.util.Map;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.io.generated.RoutingConfig;
-import org.apache.flink.statefun.flink.io.kinesis.KinesisSourceProvider;
-import org.apache.flink.statefun.flink.io.spi.JsonIngressSpec;
-import org.apache.flink.statefun.flink.io.spi.SourceProvider;
-import org.apache.flink.statefun.sdk.io.IngressIdentifier;
-import org.apache.flink.statefun.sdk.io.IngressSpec;
-import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressBuilder;
-import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressBuilderApiExtension;
-import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressDeserializer;
-import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressSpec;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-
-public final class RoutableProtobufKinesisSourceProvider implements SourceProvider {
-
-  private final KinesisSourceProvider delegateProvider = new KinesisSourceProvider();
-
-  @Override
-  public <T> SourceFunction<T> forSpec(IngressSpec<T> spec) {
-    final KinesisIngressSpec<T> kinesisIngressSpec = asKinesisIngressSpec(spec);
-    return delegateProvider.forSpec(kinesisIngressSpec);
-  }
-
-  private static <T> KinesisIngressSpec<T> asKinesisIngressSpec(IngressSpec<T> spec) {
-    if (!(spec instanceof JsonIngressSpec)) {
-      throw new IllegalArgumentException("Wrong type " + spec.type());
-    }
-    JsonIngressSpec<T> casted = (JsonIngressSpec<T>) spec;
-
-    IngressIdentifier<T> id = casted.id();
-    Class<T> producedType = casted.id().producedType();
-    if (!Message.class.isAssignableFrom(producedType)) {
-      throw new IllegalArgumentException(
-          "ProtocolBuffer based Kinesis ingress is only able to produce types that derive from "
-              + Message.class.getName()
-              + " but "
-              + producedType.getName()
-              + " is provided.");
-    }
-
-    JsonNode specJson = casted.specJson();
-
-    KinesisIngressBuilder<T> kinesisIngressBuilder = KinesisIngressBuilder.forIdentifier(id);
-
-    optionalAwsRegion(specJson).ifPresent(kinesisIngressBuilder::withAwsRegion);
-    optionalAwsCredentials(specJson).ifPresent(kinesisIngressBuilder::withAwsCredentials);
-    optionalStartupPosition(specJson).ifPresent(kinesisIngressBuilder::withStartupPosition);
-    clientConfigProperties(specJson)
-        .entrySet()
-        .forEach(
-            entry ->
-                kinesisIngressBuilder.withClientConfigurationProperty(
-                    entry.getKey(), entry.getValue()));
-
-    Map<String, RoutingConfig> routableStreams = routableStreams(specJson);
-    KinesisIngressBuilderApiExtension.withDeserializer(
-        kinesisIngressBuilder, deserializer(routableStreams));
-    kinesisIngressBuilder.withStreams(new ArrayList<>(routableStreams.keySet()));
-
-    return kinesisIngressBuilder.build();
-  }
-
-  @SuppressWarnings("unchecked")
-  private static <T> KinesisIngressDeserializer<T> deserializer(
-      Map<String, RoutingConfig> routingConfig) {
-    // this cast is safe since we've already checked that T is a Message
-    return (KinesisIngressDeserializer<T>)
-        new RoutableProtobufKinesisIngressDeserializer(routingConfig);
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/GenericKafkaSinkProviderTest.java b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/GenericKafkaSinkProviderTest.java
deleted file mode 100644
index d0dcc50..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/GenericKafkaSinkProviderTest.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.io.kafka;
-
-import static org.apache.flink.statefun.flink.io.testutils.YamlUtils.loadAsJsonFromClassResource;
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.hamcrest.MatcherAssert.assertThat;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.io.spi.JsonEgressSpec;
-import org.apache.flink.statefun.sdk.io.EgressIdentifier;
-import org.apache.flink.statefun.sdk.reqreply.generated.TypedValue;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer;
-import org.junit.Test;
-
-public class GenericKafkaSinkProviderTest {
-
-  @Test
-  public void exampleUsage() {
-    JsonNode egressDefinition =
-        loadAsJsonFromClassResource(getClass().getClassLoader(), "generic-kafka-egress.yaml");
-    JsonEgressSpec<?> spec =
-        new JsonEgressSpec<>(
-            KafkaEgressTypes.GENERIC_KAFKA_EGRESS_TYPE,
-            new EgressIdentifier<>("foo", "bar", TypedValue.class),
-            egressDefinition);
-
-    GenericKafkaSinkProvider provider = new GenericKafkaSinkProvider();
-    SinkFunction<?> sink = provider.forSpec(spec);
-
-    assertThat(sink, instanceOf(FlinkKafkaProducer.class));
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/ProtobufKafkaSourceProviderTest.java b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/ProtobufKafkaSourceProviderTest.java
deleted file mode 100644
index e9418f1..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/ProtobufKafkaSourceProviderTest.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.io.kafka;
-
-import static org.apache.flink.statefun.flink.io.testutils.YamlUtils.loadAsJsonFromClassResource;
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.junit.Assert.assertThat;
-
-import com.google.protobuf.Message;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.io.spi.JsonIngressSpec;
-import org.apache.flink.statefun.sdk.io.IngressIdentifier;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
-import org.junit.Test;
-
-public class ProtobufKafkaSourceProviderTest {
-
-  @Test
-  public void exampleUsage() {
-    JsonNode ingressDefinition =
-        loadAsJsonFromClassResource(getClass().getClassLoader(), "protobuf-kafka-ingress.yaml");
-    JsonIngressSpec<?> spec =
-        new JsonIngressSpec<>(
-            ProtobufKafkaIngressTypes.PROTOBUF_KAFKA_INGRESS_TYPE,
-            new IngressIdentifier<>(Message.class, "foo", "bar"),
-            ingressDefinition);
-
-    ProtobufKafkaSourceProvider provider = new ProtobufKafkaSourceProvider();
-    SourceFunction<?> source = provider.forSpec(spec);
-
-    assertThat(source, instanceOf(FlinkKafkaConsumer.class));
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/RoutableProtobufKafkaSourceProviderTest.java b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/RoutableProtobufKafkaSourceProviderTest.java
deleted file mode 100644
index 4747a0f..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/RoutableProtobufKafkaSourceProviderTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.io.kafka;
-
-import static org.apache.flink.statefun.flink.io.testutils.YamlUtils.loadAsJsonFromClassResource;
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.junit.Assert.assertThat;
-
-import com.google.protobuf.Message;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.io.spi.JsonIngressSpec;
-import org.apache.flink.statefun.sdk.io.IngressIdentifier;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
-import org.junit.Test;
-
-public class RoutableProtobufKafkaSourceProviderTest {
-
-  @Test
-  public void exampleUsage() {
-    JsonNode ingressDefinition =
-        loadAsJsonFromClassResource(
-            getClass().getClassLoader(), "routable-protobuf-kafka-ingress.yaml");
-    JsonIngressSpec<?> spec =
-        new JsonIngressSpec<>(
-            ProtobufKafkaIngressTypes.ROUTABLE_PROTOBUF_KAFKA_INGRESS_TYPE,
-            new IngressIdentifier<>(Message.class, "foo", "bar"),
-            ingressDefinition);
-
-    RoutableProtobufKafkaSourceProvider provider = new RoutableProtobufKafkaSourceProvider();
-    SourceFunction<?> source = provider.forSpec(spec);
-
-    assertThat(source, instanceOf(FlinkKafkaConsumer.class));
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/GenericKafkaEgressBinderV1Test.java b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/GenericKafkaEgressBinderV1Test.java
new file mode 100644
index 0000000..d92382c
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/binders/egress/v1/GenericKafkaEgressBinderV1Test.java
@@ -0,0 +1,58 @@
+/*
+ * 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.io.kafka.binders.egress.v1;
+
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.assertThat;
+
+import java.net.URL;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.io.testutils.TestModuleBinder;
+import org.apache.flink.statefun.sdk.io.EgressIdentifier;
+import org.apache.flink.statefun.sdk.kafka.KafkaEgressSpec;
+import org.apache.flink.statefun.sdk.reqreply.generated.TypedValue;
+import org.junit.Test;
+
+public class GenericKafkaEgressBinderV1Test {
+
+  private static final ObjectMapper OBJ_MAPPER = new ObjectMapper(new YAMLFactory());
+
+  private static final String SPEC_YAML_PATH = "kafka-io-binders/generic-kafka-egress-v1.yaml";
+
+  @Test
+  public void exampleUsage() throws Exception {
+    final ComponentJsonObject component = loadComponentJsonObject(SPEC_YAML_PATH);
+    final TestModuleBinder testModuleBinder = new TestModuleBinder();
+
+    GenericKafkaEgressBinderV1.INSTANCE.bind(component, testModuleBinder);
+
+    final EgressIdentifier<TypedValue> expectedEgressId =
+        new EgressIdentifier<>("com.foo.bar", "test-egress", TypedValue.class);
+    assertThat(testModuleBinder.getEgress(expectedEgressId), instanceOf(KafkaEgressSpec.class));
+  }
+
+  private static ComponentJsonObject loadComponentJsonObject(String yamlPath) throws Exception {
+    final URL url = GenericKafkaEgressBinderV1Test.class.getClassLoader().getResource(yamlPath);
+    final ObjectNode componentObject = OBJ_MAPPER.readValue(url, ObjectNode.class);
+    return new ComponentJsonObject(componentObject);
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/RoutableKafkaIngressBinderV1Test.java b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/RoutableKafkaIngressBinderV1Test.java
new file mode 100644
index 0000000..c38acd5
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kafka/binders/ingress/v1/RoutableKafkaIngressBinderV1Test.java
@@ -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://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.io.kafka.binders.ingress.v1;
+
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.assertThat;
+
+import com.google.protobuf.Message;
+import java.net.URL;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.io.common.AutoRoutableProtobufRouter;
+import org.apache.flink.statefun.flink.io.testutils.TestModuleBinder;
+import org.apache.flink.statefun.sdk.io.IngressIdentifier;
+import org.apache.flink.statefun.sdk.kafka.KafkaIngressSpec;
+import org.junit.Test;
+
+public class RoutableKafkaIngressBinderV1Test {
+
+  private static final ObjectMapper OBJ_MAPPER = new ObjectMapper(new YAMLFactory());
+
+  private static final String SPEC_YAML_PATH = "kafka-io-binders/routable-kafka-ingress-v1.yaml";
+
+  @Test
+  public void exampleUsage() throws Exception {
+    final ComponentJsonObject component = loadComponentJsonObject(SPEC_YAML_PATH);
+    final TestModuleBinder testModuleBinder = new TestModuleBinder();
+
+    RoutableKafkaIngressBinderV1.INSTANCE.bind(component, testModuleBinder);
+
+    final IngressIdentifier<Message> expectedIngressId =
+        new IngressIdentifier<>(Message.class, "com.foo.bar", "test-ingress");
+    assertThat(testModuleBinder.getIngress(expectedIngressId), instanceOf(KafkaIngressSpec.class));
+    assertThat(
+        testModuleBinder.getRouters(expectedIngressId),
+        hasItem(instanceOf(AutoRoutableProtobufRouter.class)));
+  }
+
+  private static ComponentJsonObject loadComponentJsonObject(String yamlPath) throws Exception {
+    final URL url = RoutableKafkaIngressBinderV1Test.class.getClassLoader().getResource(yamlPath);
+    final ObjectNode componentObject = OBJ_MAPPER.readValue(url, ObjectNode.class);
+    return new ComponentJsonObject(componentObject);
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kinesis/GenericKinesisSinkProviderTest.java b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kinesis/GenericKinesisSinkProviderTest.java
deleted file mode 100644
index adfc8f6..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kinesis/GenericKinesisSinkProviderTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.io.kinesis;
-
-import static org.apache.flink.statefun.flink.io.testutils.YamlUtils.loadAsJsonFromClassResource;
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.hamcrest.MatcherAssert.assertThat;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.io.kinesis.polyglot.GenericKinesisSinkProvider;
-import org.apache.flink.statefun.flink.io.spi.JsonEgressSpec;
-import org.apache.flink.statefun.sdk.io.EgressIdentifier;
-import org.apache.flink.statefun.sdk.reqreply.generated.TypedValue;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
-import org.junit.Test;
-
-public class GenericKinesisSinkProviderTest {
-
-  @Test
-  public void exampleUsage() {
-    JsonNode egressDefinition =
-        loadAsJsonFromClassResource(getClass().getClassLoader(), "generic-kinesis-egress.yaml");
-    JsonEgressSpec<?> spec =
-        new JsonEgressSpec<>(
-            PolyglotKinesisIOTypes.GENERIC_KINESIS_EGRESS_TYPE,
-            new EgressIdentifier<>("foo", "bar", TypedValue.class),
-            egressDefinition);
-
-    GenericKinesisSinkProvider provider = new GenericKinesisSinkProvider();
-    SinkFunction<?> source = provider.forSpec(spec);
-
-    assertThat(source, instanceOf(FlinkKinesisProducer.class));
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kinesis/RoutableProtobufKinesisSourceProviderTest.java b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kinesis/RoutableProtobufKinesisSourceProviderTest.java
deleted file mode 100644
index 81e6ff1..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kinesis/RoutableProtobufKinesisSourceProviderTest.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.io.kinesis;
-
-import static org.apache.flink.statefun.flink.io.testutils.YamlUtils.loadAsJsonFromClassResource;
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.hamcrest.MatcherAssert.assertThat;
-
-import com.google.protobuf.Message;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.flink.io.kinesis.polyglot.RoutableProtobufKinesisSourceProvider;
-import org.apache.flink.statefun.flink.io.spi.JsonIngressSpec;
-import org.apache.flink.statefun.sdk.io.IngressIdentifier;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
-import org.junit.Test;
-
-public class RoutableProtobufKinesisSourceProviderTest {
-
-  @Test
-  public void exampleUsage() {
-    JsonNode ingressDefinition =
-        loadAsJsonFromClassResource(
-            getClass().getClassLoader(), "routable-protobuf-kinesis-ingress.yaml");
-    JsonIngressSpec<?> spec =
-        new JsonIngressSpec<>(
-            PolyglotKinesisIOTypes.ROUTABLE_PROTOBUF_KINESIS_INGRESS_TYPE,
-            new IngressIdentifier<>(Message.class, "foo", "bar"),
-            ingressDefinition);
-
-    RoutableProtobufKinesisSourceProvider provider = new RoutableProtobufKinesisSourceProvider();
-    SourceFunction<?> source = provider.forSpec(spec);
-
-    assertThat(source, instanceOf(FlinkKinesisConsumer.class));
-  }
-}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/GenericKinesisEgressBinderV1Test.java b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/GenericKinesisEgressBinderV1Test.java
new file mode 100644
index 0000000..2cf0b73
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kinesis/binders/egress/v1/GenericKinesisEgressBinderV1Test.java
@@ -0,0 +1,39 @@
+package org.apache.flink.statefun.flink.io.kinesis.binders.egress.v1;
+
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.assertThat;
+
+import java.net.URL;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.io.testutils.TestModuleBinder;
+import org.apache.flink.statefun.sdk.io.EgressIdentifier;
+import org.apache.flink.statefun.sdk.kinesis.egress.KinesisEgressSpec;
+import org.apache.flink.statefun.sdk.reqreply.generated.TypedValue;
+import org.junit.Test;
+
+public class GenericKinesisEgressBinderV1Test {
+  private static final ObjectMapper OBJ_MAPPER = new ObjectMapper(new YAMLFactory());
+
+  private static final String SPEC_YAML_PATH = "kinesis-io-binders/generic-kinesis-egress-v1.yaml";
+
+  @Test
+  public void exampleUsage() throws Exception {
+    final ComponentJsonObject component = loadComponentJsonObject(SPEC_YAML_PATH);
+    final TestModuleBinder testModuleBinder = new TestModuleBinder();
+
+    GenericKinesisEgressBinderV1.INSTANCE.bind(component, testModuleBinder);
+
+    final EgressIdentifier<TypedValue> expectedEgressId =
+        new EgressIdentifier<>("com.foo.bar", "test-egress", TypedValue.class);
+    assertThat(testModuleBinder.getEgress(expectedEgressId), instanceOf(KinesisEgressSpec.class));
+  }
+
+  private static ComponentJsonObject loadComponentJsonObject(String yamlPath) throws Exception {
+    final URL url = GenericKinesisEgressBinderV1Test.class.getClassLoader().getResource(yamlPath);
+    final ObjectNode componentObject = OBJ_MAPPER.readValue(url, ObjectNode.class);
+    return new ComponentJsonObject(componentObject);
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/RoutableKinesisIngressBinderV1Test.java b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/RoutableKinesisIngressBinderV1Test.java
new file mode 100644
index 0000000..2acd13c
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/kinesis/binders/ingress/v1/RoutableKinesisIngressBinderV1Test.java
@@ -0,0 +1,65 @@
+/*
+ * 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.io.kinesis.binders.ingress.v1;
+
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.assertThat;
+
+import com.google.protobuf.Message;
+import java.net.URL;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import org.apache.flink.statefun.extensions.ComponentJsonObject;
+import org.apache.flink.statefun.flink.io.common.AutoRoutableProtobufRouter;
+import org.apache.flink.statefun.flink.io.testutils.TestModuleBinder;
+import org.apache.flink.statefun.sdk.io.IngressIdentifier;
+import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressSpec;
+import org.junit.Test;
+
+public class RoutableKinesisIngressBinderV1Test {
+
+  private static final ObjectMapper OBJ_MAPPER = new ObjectMapper(new YAMLFactory());
+
+  private static final String SPEC_YAML_PATH =
+      "kinesis-io-binders/routable-kinesis-ingress-v1.yaml";
+
+  @Test
+  public void exampleUsage() throws Exception {
+    final ComponentJsonObject component = loadComponentJsonObject(SPEC_YAML_PATH);
+    final TestModuleBinder testModuleBinder = new TestModuleBinder();
+
+    RoutableKinesisIngressBinderV1.INSTANCE.bind(component, testModuleBinder);
+
+    final IngressIdentifier<Message> expectedIngressId =
+        new IngressIdentifier<>(Message.class, "com.foo.bar", "test-ingress");
+    assertThat(
+        testModuleBinder.getIngress(expectedIngressId), instanceOf(KinesisIngressSpec.class));
+    assertThat(
+        testModuleBinder.getRouters(expectedIngressId),
+        hasItem(instanceOf(AutoRoutableProtobufRouter.class)));
+  }
+
+  private static ComponentJsonObject loadComponentJsonObject(String yamlPath) throws Exception {
+    final URL url = RoutableKinesisIngressBinderV1Test.class.getClassLoader().getResource(yamlPath);
+    final ObjectNode componentObject = OBJ_MAPPER.readValue(url, ObjectNode.class);
+    return new ComponentJsonObject(componentObject);
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/testutils/TestModuleBinder.java b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/testutils/TestModuleBinder.java
new file mode 100644
index 0000000..211890e
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/test/java/org/apache/flink/statefun/flink/io/testutils/TestModuleBinder.java
@@ -0,0 +1,96 @@
+/*
+ * 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.io.testutils;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.flink.statefun.sdk.FunctionType;
+import org.apache.flink.statefun.sdk.FunctionTypeNamespaceMatcher;
+import org.apache.flink.statefun.sdk.StatefulFunctionProvider;
+import org.apache.flink.statefun.sdk.io.EgressIdentifier;
+import org.apache.flink.statefun.sdk.io.EgressSpec;
+import org.apache.flink.statefun.sdk.io.IngressIdentifier;
+import org.apache.flink.statefun.sdk.io.IngressSpec;
+import org.apache.flink.statefun.sdk.io.Router;
+import org.apache.flink.statefun.sdk.spi.StatefulFunctionModule;
+
+public final class TestModuleBinder implements StatefulFunctionModule.Binder {
+  private final Map<IngressIdentifier<?>, IngressSpec<?>> ingress = new HashMap<>();
+  private final Map<EgressIdentifier<?>, EgressSpec<?>> egress = new HashMap<>();
+  private final Map<IngressIdentifier<?>, List<Router<?>>> routers = new HashMap<>();
+  private final Map<FunctionType, StatefulFunctionProvider> specificFunctionProviders =
+      new HashMap<>();
+  private final Map<String, StatefulFunctionProvider> namespaceFunctionProviders = new HashMap<>();
+
+  @Override
+  public <T> void bindIngress(IngressSpec<T> spec) {
+    Objects.requireNonNull(spec);
+    IngressIdentifier<T> id = spec.id();
+    ingress.put(id, spec);
+  }
+
+  @Override
+  public <T> void bindIngressRouter(IngressIdentifier<T> ingressIdentifier, Router<T> router) {
+    Objects.requireNonNull(ingressIdentifier);
+    Objects.requireNonNull(router);
+
+    List<Router<?>> ingressRouters =
+        routers.computeIfAbsent(ingressIdentifier, unused -> new ArrayList<>());
+    ingressRouters.add(router);
+  }
+
+  @Override
+  public <T> void bindEgress(EgressSpec<T> spec) {
+    Objects.requireNonNull(spec);
+    EgressIdentifier<T> id = spec.id();
+    egress.put(id, spec);
+  }
+
+  @Override
+  public void bindFunctionProvider(FunctionType functionType, StatefulFunctionProvider provider) {
+    Objects.requireNonNull(functionType);
+    Objects.requireNonNull(provider);
+    specificFunctionProviders.put(functionType, provider);
+  }
+
+  @Override
+  public void bindFunctionProvider(
+      FunctionTypeNamespaceMatcher namespaceMatcher, StatefulFunctionProvider provider) {
+    Objects.requireNonNull(namespaceMatcher);
+    Objects.requireNonNull(provider);
+    namespaceFunctionProviders.put(namespaceMatcher.targetNamespace(), provider);
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T> IngressSpec<T> getIngress(IngressIdentifier<T> ingressIdentifier) {
+    return (IngressSpec<T>) ingress.get(ingressIdentifier);
+  }
+
+  public <T> List<Router<?>> getRouters(IngressIdentifier<T> ingressIdentifier) {
+    return routers.get(ingressIdentifier);
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T> EgressSpec<T> getEgress(EgressIdentifier<T> egressIdentifier) {
+    return (EgressSpec<T>) egress.get(egressIdentifier);
+  }
+}
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kinesis-egress.yaml b/statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kinesis-egress.yaml
deleted file mode 100644
index a882143..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kinesis-egress.yaml
+++ /dev/null
@@ -1,32 +0,0 @@
-# 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.
-
-egress:
-  meta:
-    type: io.statefun.kinesis/egress
-    id: com.mycomp.foo/bar
-  spec:
-    awsRegion:
-      type: custom-endpoint
-      endpoint: https://localhost:4567
-      id: us-west-1
-    awsCredentials:
-      type: profile
-      profileName: john-doe
-      profilePath: /path/to/profile/config
-    maxOutstandingRecords: 9999
-    clientConfigProperties:
-      - ThreadingModel: POOLED
-      - ThreadPoolSize: 10
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml b/statefun-flink/statefun-flink-io-bundle/src/test/resources/kafka-io-binders/generic-kafka-egress-v1.yaml
similarity index 76%
copy from statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml
copy to statefun-flink/statefun-flink-io-bundle/src/test/resources/kafka-io-binders/generic-kafka-egress-v1.yaml
index 2014cb3..82be176 100644
--- a/statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml
+++ b/statefun-flink/statefun-flink-io-bundle/src/test/resources/kafka-io-binders/generic-kafka-egress-v1.yaml
@@ -13,14 +13,12 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-egress:
-  meta:
-    type: io.statefun.kafka/egress
-    id: com.mycomp.foo/bar
-  spec:
-    address: kafka-broker:9092
-    deliverySemantic:
-      type: exactly-once
-      transactionTimeout: 15min
-    properties:
-      - foo.config: bar
+kind: io.statefun.kafka.v1/egress
+spec:
+  id: com.foo.bar/test-egress
+  address: kafka-broker:9092
+  deliverySemantic:
+    type: exactly-once
+    transactionTimeout: 15min
+  properties:
+    - foo.config: bar
\ No newline at end of file
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/resources/kafka-io-binders/routable-kafka-ingress-v1.yaml b/statefun-flink/statefun-flink-io-bundle/src/test/resources/kafka-io-binders/routable-kafka-ingress-v1.yaml
new file mode 100644
index 0000000..6762258
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/test/resources/kafka-io-binders/routable-kafka-ingress-v1.yaml
@@ -0,0 +1,35 @@
+# 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.
+
+kind: io.statefun.kafka.v1/ingress
+spec:
+  id: com.foo.bar/test-ingress
+  address: kafka-broker:9092
+  consumerGroupId: my-group-id
+  topics:
+    - topic: topic-1
+      valueType: com.googleapis/com.mycomp.foo.MessageA
+      targets:
+        - com.mycomp.foo/function-1
+        - com.mycomp.foo/function-2
+    - topic: topic-2
+      valueType: com.googleapis/com.mycomp.foo.MessageB
+      targets:
+        - com.mycomp.foo/function-2
+  autoOffsetResetPosition: earliest
+  startupPosition:
+    type: earliest
+  properties:
+    - foo.config: bar
\ No newline at end of file
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml b/statefun-flink/statefun-flink-io-bundle/src/test/resources/kinesis-io-binders/generic-kinesis-egress-v1.yaml
similarity index 67%
copy from statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml
copy to statefun-flink/statefun-flink-io-bundle/src/test/resources/kinesis-io-binders/generic-kinesis-egress-v1.yaml
index 2014cb3..369200f 100644
--- a/statefun-flink/statefun-flink-io-bundle/src/test/resources/generic-kafka-egress.yaml
+++ b/statefun-flink/statefun-flink-io-bundle/src/test/resources/kinesis-io-binders/generic-kinesis-egress-v1.yaml
@@ -13,14 +13,18 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-egress:
-  meta:
-    type: io.statefun.kafka/egress
-    id: com.mycomp.foo/bar
-  spec:
-    address: kafka-broker:9092
-    deliverySemantic:
-      type: exactly-once
-      transactionTimeout: 15min
-    properties:
-      - foo.config: bar
+kind: io.statefun.kinesis.v1/egress
+spec:
+  id: com.foo.bar/test-egress
+  awsRegion:
+    type: custom-endpoint
+    endpoint: https://localhost:4567
+    id: us-west-1
+  awsCredentials:
+    type: profile
+    profileName: john-doe
+    profilePath: /path/to/profile/config
+  maxOutstandingRecords: 9999
+  clientConfigProperties:
+    - ThreadingModel: POOLED
+    - ThreadPoolSize: 10
\ No newline at end of file
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/resources/kinesis-io-binders/routable-kinesis-ingress-v1.yaml b/statefun-flink/statefun-flink-io-bundle/src/test/resources/kinesis-io-binders/routable-kinesis-ingress-v1.yaml
new file mode 100644
index 0000000..2746a0f
--- /dev/null
+++ b/statefun-flink/statefun-flink-io-bundle/src/test/resources/kinesis-io-binders/routable-kinesis-ingress-v1.yaml
@@ -0,0 +1,40 @@
+# 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.
+
+kind: io.statefun.kinesis.v1/ingress
+spec:
+  id: com.foo.bar/test-ingress
+  awsRegion:
+    type: specific
+    id: us-west-2
+  awsCredentials:
+    type: basic
+    accessKeyId: my_access_key_id
+    secretAccessKey: my_secret_access_key
+  startupPosition:
+    type: earliest
+  streams:
+    - stream: stream-1
+      valueType: com.googleapis/com.mycomp.foo.MessageA
+      targets:
+        - com.mycomp.foo/function-1
+        - com.mycomp.foo/function-2
+    - stream: topic-2
+      valueType: com.googleapis/com.mycomp.foo.MessageB
+      targets:
+        - com.mycomp.foo/function-2
+  clientConfigProperties:
+    - SocketTimeout: 9999
+    - MaxConnections: 15
\ No newline at end of file
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/resources/protobuf-kafka-ingress.yaml b/statefun-flink/statefun-flink-io-bundle/src/test/resources/protobuf-kafka-ingress.yaml
deleted file mode 100644
index 0a8ee24..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/test/resources/protobuf-kafka-ingress.yaml
+++ /dev/null
@@ -1,36 +0,0 @@
-# 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.
-
-ingress:
-  meta:
-    type: statefun.kafka.io/protobuf-ingress
-    id: com.mycomp.igal/names
-  spec:
-    address: kafka-broker:9092
-    consumerGroupId: greeter
-    topics:
-      - names
-    autoOffsetResetPosition: earliest
-    startupPosition:
-      type: specific-offsets
-      offsets:
-        - names/0: 91
-        - names/1: 11
-        - names/2: 8
-    properties:
-      - foo.config: bar
-    messageType: org.apache.flink.test.SimpleMessage
-    descriptorSet: classpath:test.desc
-    
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/resources/routable-protobuf-kafka-ingress.yaml b/statefun-flink/statefun-flink-io-bundle/src/test/resources/routable-protobuf-kafka-ingress.yaml
deleted file mode 100644
index 5633aa3..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/test/resources/routable-protobuf-kafka-ingress.yaml
+++ /dev/null
@@ -1,37 +0,0 @@
-# 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.
-
-ingress:
-  meta:
-    type: io.statefun.kafka/ingress
-    id: com.mycomp.foo/bar
-  spec:
-    address: kafka-broker:9092
-    consumerGroupId: my-group-id
-    topics:
-      - topic: topic-1
-        valueType: com.googleapis/com.mycomp.foo.MessageA
-        targets:
-          - com.mycomp.foo/function-1
-          - com.mycomp.foo/function-2
-      - topic: topic-2
-        valueType: com.googleapis/com.mycomp.foo.MessageB
-        targets:
-          - com.mycomp.foo/function-2
-    autoOffsetResetPosition: earliest
-    startupPosition:
-      type: earliest
-    properties:
-      - foo.config: bar
diff --git a/statefun-flink/statefun-flink-io-bundle/src/test/resources/routable-protobuf-kinesis-ingress.yaml b/statefun-flink/statefun-flink-io-bundle/src/test/resources/routable-protobuf-kinesis-ingress.yaml
deleted file mode 100644
index 79e75a7..0000000
--- a/statefun-flink/statefun-flink-io-bundle/src/test/resources/routable-protobuf-kinesis-ingress.yaml
+++ /dev/null
@@ -1,42 +0,0 @@
-# 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.
-
-ingress:
-  meta:
-    type: io.statefun.kinesis/ingress
-    id: com.mycomp.foo/bar
-  spec:
-    awsRegion:
-      type: specific
-      id: us-west-2
-    awsCredentials:
-      type: basic
-      accessKeyId: my_access_key_id
-      secretAccessKey: my_secret_access_key
-    startupPosition:
-      type: earliest
-    streams:
-      - stream: stream-1
-        valueType: com.googleapis/com.mycomp.foo.MessageA
-        targets:
-          - com.mycomp.foo/function-1
-          - com.mycomp.foo/function-2
-      - stream: topic-2
-        valueType: com.googleapis/com.mycomp.foo.MessageB
-        targets:
-          - com.mycomp.foo/function-2
-    clientConfigProperties:
-      - SocketTimeout: 9999
-      - MaxConnections: 15
diff --git a/statefun-flink/statefun-flink-io/pom.xml b/statefun-flink/statefun-flink-io/pom.xml
index 960d729..a801958 100644
--- a/statefun-flink/statefun-flink-io/pom.xml
+++ b/statefun-flink/statefun-flink-io/pom.xml
@@ -41,6 +41,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.flink</groupId>
+            <artifactId>statefun-flink-extensions</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
             <artifactId>statefun-sdk-protos</artifactId>
             <version>${project.version}</version>
         </dependency>
diff --git a/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaEgressTypes.java b/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaEgressTypes.java
deleted file mode 100644
index f92c6c1..0000000
--- a/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/kafka/KafkaEgressTypes.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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.io.kafka;
-
-import org.apache.flink.statefun.sdk.EgressType;
-
-public final class KafkaEgressTypes {
-
-  private KafkaEgressTypes() {}
-
-  public static final EgressType GENERIC_KAFKA_EGRESS_TYPE =
-      new EgressType("io.statefun.kafka", "egress");
-}
diff --git a/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/kafka/ProtobufKafkaIngressTypes.java b/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/kafka/ProtobufKafkaIngressTypes.java
deleted file mode 100644
index 800746b..0000000
--- a/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/kafka/ProtobufKafkaIngressTypes.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.io.kafka;
-
-import org.apache.flink.statefun.sdk.IngressType;
-
-public final class ProtobufKafkaIngressTypes {
-
-  private ProtobufKafkaIngressTypes() {}
-
-  public static final IngressType PROTOBUF_KAFKA_INGRESS_TYPE =
-      new IngressType("statefun.kafka.io", "protobuf-ingress");
-
-  public static final IngressType ROUTABLE_PROTOBUF_KAFKA_INGRESS_TYPE =
-      new IngressType("io.statefun.kafka", "ingress");
-}
diff --git a/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/kinesis/PolyglotKinesisIOTypes.java b/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/kinesis/PolyglotKinesisIOTypes.java
deleted file mode 100644
index 2c0b1d3..0000000
--- a/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/kinesis/PolyglotKinesisIOTypes.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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.io.kinesis;
-
-import org.apache.flink.statefun.sdk.EgressType;
-import org.apache.flink.statefun.sdk.IngressType;
-
-public final class PolyglotKinesisIOTypes {
-
-  private PolyglotKinesisIOTypes() {}
-
-  public static final IngressType ROUTABLE_PROTOBUF_KINESIS_INGRESS_TYPE =
-      new IngressType("io.statefun.kinesis", "ingress");
-
-  public static final EgressType GENERIC_KINESIS_EGRESS_TYPE =
-      new EgressType("io.statefun.kinesis", "egress");
-}
diff --git a/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/spi/JsonEgressSpec.java b/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/spi/JsonEgressSpec.java
deleted file mode 100644
index 5868840..0000000
--- a/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/spi/JsonEgressSpec.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.io.spi;
-
-import java.util.Objects;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.sdk.EgressType;
-import org.apache.flink.statefun.sdk.io.EgressIdentifier;
-import org.apache.flink.statefun.sdk.io.EgressSpec;
-
-public final class JsonEgressSpec<T> implements EgressSpec<T> {
-
-  private static final JsonPointer SPEC_POINTER = JsonPointer.compile("/egress/spec");
-
-  private final JsonNode json;
-  private final EgressIdentifier<T> id;
-  private final EgressType type;
-
-  public JsonEgressSpec(EgressType type, EgressIdentifier<T> id, JsonNode json) {
-    this.type = Objects.requireNonNull(type);
-    this.id = Objects.requireNonNull(id);
-    this.json = Objects.requireNonNull(json);
-  }
-
-  @Override
-  public EgressType type() {
-    return type;
-  }
-
-  @Override
-  public EgressIdentifier<T> id() {
-    return id;
-  }
-
-  public JsonNode json() {
-    return json;
-  }
-
-  public JsonNode specJson() {
-    return json.requiredAt(SPEC_POINTER);
-  }
-}
diff --git a/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/spi/JsonIngressSpec.java b/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/spi/JsonIngressSpec.java
deleted file mode 100644
index 43cbcdf..0000000
--- a/statefun-flink/statefun-flink-io/src/main/java/org/apache/flink/statefun/flink/io/spi/JsonIngressSpec.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.io.spi;
-
-import java.util.Objects;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonPointer;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
-import org.apache.flink.statefun.sdk.IngressType;
-import org.apache.flink.statefun.sdk.io.IngressIdentifier;
-import org.apache.flink.statefun.sdk.io.IngressSpec;
-
-public final class JsonIngressSpec<T> implements IngressSpec<T> {
-
-  private static final JsonPointer SPEC_POINTER = JsonPointer.compile("/ingress/spec");
-
-  private final JsonNode json;
-  private final IngressIdentifier<T> id;
-  private final IngressType type;
-
-  public JsonIngressSpec(IngressType type, IngressIdentifier<T> id, JsonNode json) {
-    this.type = Objects.requireNonNull(type);
-    this.id = Objects.requireNonNull(id);
-    this.json = Objects.requireNonNull(json);
-  }
-
-  @Override
-  public IngressType type() {
-    return type;
-  }
-
-  @Override
-  public IngressIdentifier<T> id() {
-    return id;
-  }
-
-  public JsonNode json() {
-    return json;
-  }
-
-  public JsonNode specJson() {
-    return json.requiredAt(SPEC_POINTER);
-  }
-}
diff --git a/statefun-kafka-io/src/main/java/org/apache/flink/statefun/sdk/kafka/KafkaEgressBuilder.java b/statefun-kafka-io/src/main/java/org/apache/flink/statefun/sdk/kafka/KafkaEgressBuilder.java
index 9abe95f..270cfb8 100644
--- a/statefun-kafka-io/src/main/java/org/apache/flink/statefun/sdk/kafka/KafkaEgressBuilder.java
+++ b/statefun-kafka-io/src/main/java/org/apache/flink/statefun/sdk/kafka/KafkaEgressBuilder.java
@@ -35,8 +35,7 @@
   private String kafkaAddress;
   private Properties properties = new Properties();
   private int kafkaProducerPoolSize = 5;
-  private KafkaProducerSemantic semantic = KafkaProducerSemantic.AT_LEAST_ONCE;
-  private Duration transactionTimeoutDuration = Duration.ZERO;
+  private KafkaProducerSemantic semantic = KafkaProducerSemantic.atLeastOnce();
 
   private KafkaEgressBuilder(EgressIdentifier<OutT> id) {
     this.id = Objects.requireNonNull(id);
@@ -101,15 +100,7 @@
    */
   public KafkaEgressBuilder<OutT> withExactlyOnceProducerSemantics(
       Duration transactionTimeoutDuration) {
-    Objects.requireNonNull(
-        transactionTimeoutDuration, "a transaction timeout duration must be provided.");
-    if (transactionTimeoutDuration == Duration.ZERO) {
-      throw new IllegalArgumentException(
-          "Transaction timeout durations must be larger than 0 when using exactly-once producer semantics.");
-    }
-
-    this.semantic = KafkaProducerSemantic.EXACTLY_ONCE;
-    this.transactionTimeoutDuration = transactionTimeoutDuration;
+    this.semantic = KafkaProducerSemantic.exactlyOnce(transactionTimeoutDuration);
     return this;
   }
 
@@ -118,7 +109,7 @@
    * Kafka buffers to be acknowledged by the Kafka producer on a checkpoint.
    */
   public KafkaEgressBuilder<OutT> withAtLeastOnceProducerSemantics() {
-    this.semantic = KafkaProducerSemantic.AT_LEAST_ONCE;
+    this.semantic = KafkaProducerSemantic.atLeastOnce();
     return this;
   }
 
@@ -127,19 +118,18 @@
    * duplicated in case of failure.
    */
   public KafkaEgressBuilder<OutT> withNoProducerSemantics() {
-    this.semantic = KafkaProducerSemantic.NONE;
+    this.semantic = KafkaProducerSemantic.none();
+    return this;
+  }
+
+  public KafkaEgressBuilder<OutT> withProducerSemantic(KafkaProducerSemantic producerSemantic) {
+    this.semantic = Objects.requireNonNull(producerSemantic);
     return this;
   }
 
   /** @return An {@link EgressSpec} that can be used in a {@code StatefulFunctionModule}. */
   public KafkaEgressSpec<OutT> build() {
     return new KafkaEgressSpec<>(
-        id,
-        serializer,
-        kafkaAddress,
-        properties,
-        kafkaProducerPoolSize,
-        semantic,
-        transactionTimeoutDuration);
+        id, serializer, kafkaAddress, properties, kafkaProducerPoolSize, semantic);
   }
 }
diff --git a/statefun-kafka-io/src/main/java/org/apache/flink/statefun/sdk/kafka/KafkaEgressSpec.java b/statefun-kafka-io/src/main/java/org/apache/flink/statefun/sdk/kafka/KafkaEgressSpec.java
index a7a292f..0afe690 100644
--- a/statefun-kafka-io/src/main/java/org/apache/flink/statefun/sdk/kafka/KafkaEgressSpec.java
+++ b/statefun-kafka-io/src/main/java/org/apache/flink/statefun/sdk/kafka/KafkaEgressSpec.java
@@ -17,7 +17,6 @@
  */
 package org.apache.flink.statefun.sdk.kafka;
 
-import java.time.Duration;
 import java.util.Objects;
 import java.util.Properties;
 import org.apache.flink.statefun.sdk.EgressType;
@@ -31,7 +30,6 @@
   private final EgressIdentifier<OutT> id;
   private final int kafkaProducerPoolSize;
   private final KafkaProducerSemantic semantic;
-  private final Duration transactionTimeoutDuration;
 
   KafkaEgressSpec(
       EgressIdentifier<OutT> id,
@@ -39,15 +37,13 @@
       String kafkaAddress,
       Properties properties,
       int kafkaProducerPoolSize,
-      KafkaProducerSemantic semantic,
-      Duration transactionTimeoutDuration) {
+      KafkaProducerSemantic semantic) {
     this.serializer = Objects.requireNonNull(serializer);
     this.kafkaAddress = Objects.requireNonNull(kafkaAddress);
     this.properties = Objects.requireNonNull(properties);
     this.id = Objects.requireNonNull(id);
     this.kafkaProducerPoolSize = kafkaProducerPoolSize;
     this.semantic = Objects.requireNonNull(semantic);
-    this.transactionTimeoutDuration = Objects.requireNonNull(transactionTimeoutDuration);
   }
 
   @Override
@@ -79,8 +75,4 @@
   public KafkaProducerSemantic semantic() {
     return semantic;
   }
-
-  public Duration transactionTimeoutDuration() {
-    return transactionTimeoutDuration;
-  }
 }
diff --git a/statefun-kafka-io/src/main/java/org/apache/flink/statefun/sdk/kafka/KafkaProducerSemantic.java b/statefun-kafka-io/src/main/java/org/apache/flink/statefun/sdk/kafka/KafkaProducerSemantic.java
index e9056a5..9f2bd70 100644
--- a/statefun-kafka-io/src/main/java/org/apache/flink/statefun/sdk/kafka/KafkaProducerSemantic.java
+++ b/statefun-kafka-io/src/main/java/org/apache/flink/statefun/sdk/kafka/KafkaProducerSemantic.java
@@ -17,8 +17,68 @@
  */
 package org.apache.flink.statefun.sdk.kafka;
 
-public enum KafkaProducerSemantic {
-  EXACTLY_ONCE,
-  AT_LEAST_ONCE,
-  NONE
+import java.time.Duration;
+import java.util.Objects;
+
+public abstract class KafkaProducerSemantic {
+
+  public static KafkaProducerSemantic exactlyOnce(Duration transactionTimeout) {
+    return new ExactlyOnce(transactionTimeout);
+  }
+
+  public static KafkaProducerSemantic atLeastOnce() {
+    return new AtLeastOnce();
+  }
+
+  public static KafkaProducerSemantic none() {
+    return new NoSemantics();
+  }
+
+  public boolean isExactlyOnceSemantic() {
+    return getClass() == ExactlyOnce.class;
+  }
+
+  public ExactlyOnce asExactlyOnceSemantic() {
+    return (ExactlyOnce) this;
+  }
+
+  public boolean isAtLeastOnceSemantic() {
+    return getClass() == AtLeastOnce.class;
+  }
+
+  public AtLeastOnce asAtLeastOnceSemantic() {
+    return (AtLeastOnce) this;
+  }
+
+  public boolean isNoSemantic() {
+    return getClass() == NoSemantics.class;
+  }
+
+  public NoSemantics asNoSemantic() {
+    return (NoSemantics) this;
+  }
+
+  public static class ExactlyOnce extends KafkaProducerSemantic {
+    private final Duration transactionTimeout;
+
+    private ExactlyOnce(Duration transactionTimeout) {
+      if (transactionTimeout == Duration.ZERO) {
+        throw new IllegalArgumentException(
+            "Transaction timeout durations must be larger than 0 when using exactly-once producer semantics.");
+      }
+      this.transactionTimeout = Objects.requireNonNull(transactionTimeout);
+    }
+
+    public Duration transactionTimeout() {
+      return transactionTimeout;
+    }
+  }
+
+  public static class AtLeastOnce extends KafkaProducerSemantic {
+    private AtLeastOnce() {}
+  }
+
+  public static class NoSemantics extends KafkaProducerSemantic {
+    private NoSemantics() {}
+  }
 }
diff --git a/statefun-kinesis-io/src/main/java/org/apache/flink/statefun/sdk/kinesis/egress/KinesisEgressBuilder.java b/statefun-kinesis-io/src/main/java/org/apache/flink/statefun/sdk/kinesis/egress/KinesisEgressBuilder.java
index 98406c7..01d482e 100644
--- a/statefun-kinesis-io/src/main/java/org/apache/flink/statefun/sdk/kinesis/egress/KinesisEgressBuilder.java
+++ b/statefun-kinesis-io/src/main/java/org/apache/flink/statefun/sdk/kinesis/egress/KinesisEgressBuilder.java
@@ -37,7 +37,7 @@
   private int maxOutstandingRecords = 1000;
   private AwsRegion awsRegion = AwsRegion.fromDefaultProviderChain();
   private AwsCredentials awsCredentials = AwsCredentials.fromDefaultProviderChain();
-  private final Properties clientConfigurationProperties = new Properties();
+  private final Properties properties = new Properties();
 
   private KinesisEgressBuilder(EgressIdentifier<T> id) {
     this.id = Objects.requireNonNull(id);
@@ -128,22 +128,31 @@
    * @param value the value for the property.
    * @see <a
    *     href="https://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/ClientConfiguration.html">com.aws.ClientConfiguration</a>.
+   * @deprecated Please use {@link #withProperty(String, String)} instead.
    */
   public KinesisEgressBuilder<T> withClientConfigurationProperty(String key, String value) {
     Objects.requireNonNull(key);
     Objects.requireNonNull(value);
-    this.clientConfigurationProperties.setProperty(key, value);
+    this.properties.setProperty(key, value);
+    return this;
+  }
+
+  public KinesisEgressBuilder<T> withProperty(String key, String value) {
+    Objects.requireNonNull(key);
+    Objects.requireNonNull(value);
+    this.properties.setProperty(key, value);
+    return this;
+  }
+
+  public KinesisEgressBuilder<T> withProperties(Properties properties) {
+    Objects.requireNonNull(properties);
+    this.properties.putAll(properties);
     return this;
   }
 
   /** @return A new {@link KinesisEgressSpec}. */
   public KinesisEgressSpec<T> build() {
     return new KinesisEgressSpec<>(
-        id,
-        serializerClass,
-        maxOutstandingRecords,
-        awsRegion,
-        awsCredentials,
-        clientConfigurationProperties);
+        id, serializerClass, maxOutstandingRecords, awsRegion, awsCredentials, properties);
   }
 }
diff --git a/statefun-kinesis-io/src/main/java/org/apache/flink/statefun/sdk/kinesis/ingress/KinesisIngressBuilder.java b/statefun-kinesis-io/src/main/java/org/apache/flink/statefun/sdk/kinesis/ingress/KinesisIngressBuilder.java
index f9325ca..2a03337 100644
--- a/statefun-kinesis-io/src/main/java/org/apache/flink/statefun/sdk/kinesis/ingress/KinesisIngressBuilder.java
+++ b/statefun-kinesis-io/src/main/java/org/apache/flink/statefun/sdk/kinesis/ingress/KinesisIngressBuilder.java
@@ -175,6 +175,12 @@
     return this;
   }
 
+  public KinesisIngressBuilder<T> withProperties(Properties properties) {
+    Objects.requireNonNull(properties);
+    this.properties.putAll(properties);
+    return this;
+  }
+
   /** @return A new {@link KinesisIngressSpec}. */
   public KinesisIngressSpec<T> build() {
     return new KinesisIngressSpec<>(