diff --git a/CMakeLists.txt b/CMakeLists.txt
index d038f59e8..d624971a3 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -63,7 +63,8 @@ add_subdirectory(silkworm)
 option(SILKRPC_CLANG_COVERAGE "Clang instrumentation for code coverage reports" OFF)
 
 if(SILKRPC_CLANG_COVERAGE)
-  add_compile_options(-fprofile-instr-generate -fcoverage-mapping)
+
+  add_compile_options(-fprofile-instr-generate -fcoverage-mapping -DBUILD_COVERAGE)
   add_link_options(-fprofile-instr-generate -fcoverage-mapping)
 endif()
 
diff --git a/cmd/ethbackend_coroutines.cpp b/cmd/ethbackend_coroutines.cpp
index 118afc515..ed8ad39b7 100644
--- a/cmd/ethbackend_coroutines.cpp
+++ b/cmd/ethbackend_coroutines.cpp
@@ -28,7 +28,7 @@
 #include <silkrpc/common/constants.hpp>
 #include <silkrpc/common/util.hpp>
 #include <silkrpc/context_pool.hpp>
-#include <silkrpc/ethbackend/backend.hpp>
+#include <silkrpc/ethbackend/backend_grpc.hpp>
 #include <silkrpc/interfaces/types/types.pb.h>
 
 inline std::ostream& operator<<(std::ostream& out, const types::H160& address) {
@@ -74,7 +74,7 @@ int ethbackend_coroutines(const std::string& target) {
         const auto channel = grpc::CreateChannel(target, grpc::InsecureChannelCredentials());
 
         // Etherbase
-        silkrpc::ethbackend::BackEnd eth_backend{*io_context, channel, grpc_queue.get()};
+        silkrpc::ethbackend::BackEndGrpc eth_backend{*io_context, channel, grpc_queue.get()};
         asio::co_spawn(*io_context, ethbackend_etherbase(eth_backend), [&](std::exception_ptr exptr) {
             context_pool.stop();
         });
diff --git a/silkrpc/commands/engine_api.cpp b/silkrpc/commands/engine_api.cpp
new file mode 100644
index 000000000..c6b963f9a
--- /dev/null
+++ b/silkrpc/commands/engine_api.cpp
@@ -0,0 +1,49 @@
+/*
+    Copyright 2022 The Silkrpc Authors
+
+    Licensed 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.
+*/
+
+#include "engine_api.hpp"
+
+#include <string>
+
+namespace silkrpc::commands {
+
+asio::awaitable<void> EngineRpcApi::handle_engine_get_payload_v1(const nlohmann::json& request, nlohmann::json& reply) {
+    auto params = request.at("params");
+
+    if (params.size() != 1) {
+        auto error_msg = "invalid engine_getPayloadV1 params: " + params.dump();
+        SILKRPC_ERROR << error_msg << "\n";
+        reply = make_json_error(request.at("id"), 100, error_msg);
+        co_return;
+    }
+    #ifndef BUILD_COVERAGE
+    try {
+    #endif
+        const auto payload_id = params[0].get<std::string>();
+        reply = co_await backend_->engine_get_payload_v1(std::stoul(payload_id, 0, 16));
+    #ifndef BUILD_COVERAGE
+    } catch (const std::exception& e) {
+        SILKRPC_ERROR << "exception: " << e.what() << " processing request: " << request.dump() << "\n";
+        reply = make_json_error(request.at("id"), 100, e.what());
+    } catch (...) {
+        SILKRPC_ERROR << "unexpected exception processing request: " << request.dump() << "\n";
+        reply = make_json_error(request.at("id"), 100, "unexpected exception");
+    }
+    #endif
+}
+
+
+} // namespace silkrpc::commands
diff --git a/silkrpc/commands/engine_api.hpp b/silkrpc/commands/engine_api.hpp
new file mode 100644
index 000000000..b878eb9a4
--- /dev/null
+++ b/silkrpc/commands/engine_api.hpp
@@ -0,0 +1,55 @@
+/*
+   Copyright 2022 The Silkrpc Authors
+
+   Licensed 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.
+*/
+
+#ifndef SILKRPC_COMMANDS_ENGINE_API_HPP_
+#define SILKRPC_COMMANDS_ENGINE_API_HPP_
+
+#include <memory>
+#include <vector>
+
+#include <asio/awaitable.hpp>
+#include <asio/thread_pool.hpp>
+#include <nlohmann/json.hpp>
+
+#include <silkrpc/context_pool.hpp>
+#include <silkrpc/json/types.hpp>
+#include <silkrpc/ethbackend/backend.hpp>
+
+
+namespace silkrpc::http { class RequestHandler; }
+
+namespace silkrpc::commands {
+
+class EngineRpcApi {
+public:
+    explicit EngineRpcApi(std::unique_ptr<ethbackend::BackEnd>& backend): backend_(backend) {}
+    virtual ~EngineRpcApi() {}
+
+    EngineRpcApi(const EngineRpcApi&) = delete;
+    EngineRpcApi& operator=(const EngineRpcApi&) = delete;
+
+protected:
+    asio::awaitable<void> handle_engine_get_payload_v1(const nlohmann::json& request, nlohmann::json& reply);
+
+private:
+    std::unique_ptr<ethbackend::BackEnd>& backend_;
+
+    friend class silkrpc::http::RequestHandler;
+};
+
+} // namespace silkrpc::commands
+
+#endif  // SILKRPC_COMMANDS_ENGINE_API_HPP_
diff --git a/silkrpc/commands/engine_api_test.cpp b/silkrpc/commands/engine_api_test.cpp
new file mode 100644
index 000000000..8089e912e
--- /dev/null
+++ b/silkrpc/commands/engine_api_test.cpp
@@ -0,0 +1,135 @@
+/*
+   Copyright 2021 The Silkrpc Authors
+
+   Licensed 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.
+*/
+
+#include "engine_api.hpp"
+
+#include <silkrpc/json/types.hpp>
+#include <silkrpc/http/methods.hpp>
+#include <catch2/catch.hpp>
+#include <gmock/gmock.h>
+#include <asio/awaitable.hpp>
+#include <asio/use_future.hpp>
+#include <asio/co_spawn.hpp>
+#include <utility>
+#include <string>
+
+namespace silkrpc::commands {
+
+using Catch::Matchers::Message;
+
+namespace {
+class BackEndMock : public ethbackend::BackEnd {
+public:
+    MOCK_METHOD((asio::awaitable<evmc::address>), etherbase, ());
+    MOCK_METHOD((asio::awaitable<uint64_t>), protocol_version, ());
+    MOCK_METHOD((asio::awaitable<uint64_t>), net_version, ());
+    MOCK_METHOD((asio::awaitable<std::string>), client_version, ());
+    MOCK_METHOD((asio::awaitable<uint64_t>), net_peer_count, ());
+    MOCK_METHOD((asio::awaitable<ExecutionPayload>), engine_get_payload_v1, (uint64_t payload_id));
+};
+
+} // namespace
+
+class EngineRpcApiTest : public EngineRpcApi{
+public:
+    explicit EngineRpcApiTest(std::unique_ptr<ethbackend::BackEnd>& backend): EngineRpcApi(backend) {}
+
+    using EngineRpcApi::handle_engine_get_payload_v1;
+};
+
+using testing::InvokeWithoutArgs;
+
+TEST_CASE("handle_engine_get_payload_v1 succeeds if request is expected payload", "[silkrpc][engine_api]") {
+    SILKRPC_LOG_VERBOSITY(LogLevel::None);
+
+    BackEndMock backend;
+    EXPECT_CALL(backend, engine_get_payload_v1(1)).WillOnce(InvokeWithoutArgs(
+        []() -> asio::awaitable<ExecutionPayload> {
+            co_return ExecutionPayload{1};
+        }
+    ));
+
+    std::unique_ptr<ethbackend::BackEnd> backend_ptr(&backend);
+
+    nlohmann::json reply;
+    nlohmann::json request = R"({
+        "jsonrpc":"2.0",
+        "id":1,
+        "method":"engine_getPayloadV1",
+        "params":["0x0000000000000001"]
+    })"_json;
+    // Initialize contex pool
+    ContextPool cp{1, []() { return grpc::CreateChannel("localhost", grpc::InsecureChannelCredentials()); }};
+    auto context_pool_thread = std::thread([&]() { cp.run(); });
+    // Initialise components
+    EngineRpcApiTest rpc(backend_ptr);
+
+    // spawn routine
+    auto result{asio::co_spawn(cp.get_io_context(), [&rpc, &reply, &request]() {
+        return rpc.handle_engine_get_payload_v1(
+            request,
+            reply
+        );
+    }, asio::use_future)};
+    result.get();
+
+    CHECK(reply == ExecutionPayload{1});
+
+    cp.stop();
+    context_pool_thread.join();
+    backend_ptr.release();
+}
+
+TEST_CASE("handle_engine_get_payload_v1 fails with invalid amount of params", "[silkrpc][engine_api]") {
+    SILKRPC_LOG_VERBOSITY(LogLevel::None);
+
+    nlohmann::json reply;
+    nlohmann::json request = R"({
+        "jsonrpc":"2.0",
+        "id":1,
+        "method":"engine_getPayloadV1",
+        "params":[]
+    })"_json;
+    // Initialize contex pool
+    ContextPool cp{1, []() { return grpc::CreateChannel("localhost", grpc::InsecureChannelCredentials()); }};
+    auto context_pool_thread = std::thread([&]() { cp.run(); });
+    // Initialise components
+    std::unique_ptr<ethbackend::BackEnd> backend_ptr(new BackEndMock);
+    EngineRpcApiTest rpc(backend_ptr);
+
+    // spawn routine
+    auto result{asio::co_spawn(cp.get_io_context(), [&rpc, &reply, &request]() {
+        return rpc.handle_engine_get_payload_v1(
+            request,
+            reply
+        );
+    }, asio::use_future)};
+    result.get();
+
+    CHECK(reply ==  R"({
+        "error":{
+            "code":100,
+            "message":"invalid engine_getPayloadV1 params: []"
+        },
+        "id":1,
+        "jsonrpc":"2.0" 
+    })"_json);
+
+    cp.stop();
+    context_pool_thread.join();
+}
+
+} // namespace silkrpc::commands
diff --git a/silkrpc/commands/net_api.hpp b/silkrpc/commands/net_api.hpp
index 02d7d648f..067db1f4f 100644
--- a/silkrpc/commands/net_api.hpp
+++ b/silkrpc/commands/net_api.hpp
@@ -28,6 +28,7 @@
 #include <silkrpc/json/types.hpp>
 #include <silkrpc/types/log.hpp>
 #include <silkrpc/ethbackend/backend.hpp>
+#include <silkrpc/common/log.hpp>
 
 namespace silkrpc::http { class RequestHandler; }
 
diff --git a/silkrpc/commands/rpc_api.hpp b/silkrpc/commands/rpc_api.hpp
index 5286a5efe..ed2bb4729 100644
--- a/silkrpc/commands/rpc_api.hpp
+++ b/silkrpc/commands/rpc_api.hpp
@@ -28,6 +28,7 @@
 #include <silkrpc/commands/tg_api.hpp>
 #include <silkrpc/commands/trace_api.hpp>
 #include <silkrpc/commands/web3_api.hpp>
+#include <silkrpc/commands/engine_api.hpp>
 
 namespace silkrpc::http { class RequestHandler; }
 
@@ -35,11 +36,12 @@ namespace silkrpc::commands {
 
 class RpcApiTable;
 
-class RpcApi : protected EthereumRpcApi, NetRpcApi, Web3RpcApi, DebugRpcApi, ParityRpcApi, TurboGethRpcApi, TraceRpcApi {
+class RpcApi : protected EthereumRpcApi, NetRpcApi, Web3RpcApi, DebugRpcApi, ParityRpcApi, TurboGethRpcApi, TraceRpcApi, EngineRpcApi {
 public:
     explicit RpcApi(Context& context, asio::thread_pool& workers) :
         EthereumRpcApi{context, workers}, NetRpcApi{context.backend}, Web3RpcApi{context}, DebugRpcApi{context.database},
-        ParityRpcApi{context.database}, TurboGethRpcApi{context.database}, TraceRpcApi{context.database} {}
+        ParityRpcApi{context.database}, TurboGethRpcApi{context.database}, TraceRpcApi{context.database},
+        EngineRpcApi(context.backend) {}
     virtual ~RpcApi() {}
 
     RpcApi(const RpcApi&) = delete;
diff --git a/silkrpc/commands/rpc_api_table.cpp b/silkrpc/commands/rpc_api_table.cpp
index a43f700cd..ac2dc0964 100644
--- a/silkrpc/commands/rpc_api_table.cpp
+++ b/silkrpc/commands/rpc_api_table.cpp
@@ -62,6 +62,8 @@ void RpcApiTable::add_handlers(const std::string& api_namespace) {
         add_trace_handlers();
     } else if (api_namespace == kWeb3ApiNamespace) {
         add_web3_handlers();
+    } else if (api_namespace == kEngineApiNamespace) {
+        add_engine_handlers();
     } else {
         SILKRPC_WARN << "Server::add_handlers invalid namespace [" << api_namespace << "] ignored\n";
     }
@@ -156,4 +158,8 @@ void RpcApiTable::add_web3_handlers() {
     handlers_[http::method::k_web3_sha3] = &commands::RpcApi::handle_web3_sha3;
 }
 
+void RpcApiTable::add_engine_handlers() {
+    handlers_[http::method::k_engine_getPayloadV1] = &commands::RpcApi::handle_engine_get_payload_v1;
+}
+
 } // namespace silkrpc::commands
diff --git a/silkrpc/commands/rpc_api_table.hpp b/silkrpc/commands/rpc_api_table.hpp
index 289b90f92..8590bc26b 100644
--- a/silkrpc/commands/rpc_api_table.hpp
+++ b/silkrpc/commands/rpc_api_table.hpp
@@ -51,6 +51,7 @@ class RpcApiTable {
     void add_tg_handlers();
     void add_trace_handlers();
     void add_web3_handlers();
+    void add_engine_handlers();
 
     std::map<std::string, HandleMethod> handlers_;
 };
diff --git a/silkrpc/commands/web3_api.cpp b/silkrpc/commands/web3_api.cpp
index 2f222ac9b..70c3adce6 100644
--- a/silkrpc/commands/web3_api.cpp
+++ b/silkrpc/commands/web3_api.cpp
@@ -29,7 +29,7 @@ namespace silkrpc::commands {
 
 // https://eth.wiki/json-rpc/API#web3_clientversion
 asio::awaitable<void> Web3RpcApi::handle_web3_client_version(const nlohmann::json& request, nlohmann::json& reply) {
-   try {
+    try {
         const auto client_version = co_await backend_->client_version();
         reply = make_json_content(request["id"], client_version);
     } catch (const std::exception& e) {
diff --git a/silkrpc/context_pool.cpp b/silkrpc/context_pool.cpp
index e3d61e560..fcd02f718 100644
--- a/silkrpc/context_pool.cpp
+++ b/silkrpc/context_pool.cpp
@@ -22,6 +22,7 @@
 
 #include <silkrpc/common/log.hpp>
 #include <silkrpc/ethdb/kv/remote_database.hpp>
+#include <silkrpc/ethbackend/backend_grpc.hpp>
 
 namespace silkrpc {
 
@@ -52,7 +53,7 @@ ContextPool::ContextPool(std::size_t pool_size, ChannelFactory create_channel) :
         auto grpc_queue = std::make_unique<grpc::CompletionQueue>();
         auto grpc_runner = std::make_unique<CompletionRunner>(*grpc_queue, *io_context);
         auto database = std::make_unique<ethdb::kv::RemoteDatabase<>>(*io_context, grpc_channel, grpc_queue.get()); // TODO(canepat): move elsewhere
-        auto backend = std::make_unique<ethbackend::BackEnd>(*io_context, grpc_channel, grpc_queue.get()); // TODO(canepat): move elsewhere
+        auto backend = std::make_unique<ethbackend::BackEndGrpc>(*io_context, grpc_channel, grpc_queue.get()); // TODO(canepat): move elsewhere
         auto miner = std::make_unique<txpool::Miner>(*io_context, grpc_channel, grpc_queue.get()); // TODO(canepat): move elsewhere
         auto tx_pool = std::make_unique<txpool::TransactionPool>(*io_context, grpc_channel, grpc_queue.get()); // TODO(canepat): move elsewhere
         contexts_.push_back({
diff --git a/silkrpc/ethbackend/backend.hpp b/silkrpc/ethbackend/backend.hpp
index 20c02861b..eea55c0b1 100644
--- a/silkrpc/ethbackend/backend.hpp
+++ b/silkrpc/ethbackend/backend.hpp
@@ -1,175 +1,40 @@
 /*
-    Copyright 2020 The Silkrpc Authors
+   Copyright 2022 The Silkrpc Authors
 
-    Licensed 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
+   Licensed 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
+       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.
+   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.
 */
 
 #ifndef SILKRPC_ETHBACKEND_BACKEND_HPP_
 #define SILKRPC_ETHBACKEND_BACKEND_HPP_
 
-#include <utility>
 #include <string>
-#include <memory>
 
 #include <asio/io_context.hpp>
 #include <asio/use_awaitable.hpp>
 #include <evmc/evmc.hpp>
-
-#include <silkrpc/grpc/awaitables.hpp>
-#include <silkrpc/grpc/async_unary_client.hpp>
-#include <silkrpc/interfaces/remote/ethbackend.grpc.pb.h>
-#include <silkrpc/interfaces/types/types.pb.h>
 #include <silkrpc/types/execution_payload.hpp>
 
 namespace silkrpc::ethbackend {
 
-using EtherbaseClient = AsyncUnaryClient<
-    ::remote::ETHBACKEND::StubInterface,
-    ::remote::EtherbaseRequest,
-    ::remote::EtherbaseReply,
-    &::remote::ETHBACKEND::StubInterface::PrepareAsyncEtherbase
->;
-
-using ProtocolVersionClient = AsyncUnaryClient<
-    ::remote::ETHBACKEND::StubInterface,
-    ::remote::ProtocolVersionRequest,
-    ::remote::ProtocolVersionReply,
-    &::remote::ETHBACKEND::StubInterface::PrepareAsyncProtocolVersion
->;
-
-using NetVersionClient = AsyncUnaryClient<
-    ::remote::ETHBACKEND::StubInterface,
-    ::remote::NetVersionRequest,
-    ::remote::NetVersionReply,
-    &::remote::ETHBACKEND::StubInterface::PrepareAsyncNetVersion
->;
-
-using ClientVersionClient = AsyncUnaryClient<
-    ::remote::ETHBACKEND::StubInterface,
-    ::remote::ClientVersionRequest,
-    ::remote::ClientVersionReply,
-    &::remote::ETHBACKEND::StubInterface::PrepareAsyncClientVersion
->;
-
-using NetPeerCountClient = AsyncUnaryClient<
-    ::remote::ETHBACKEND::StubInterface,
-    ::remote::NetPeerCountRequest,
-    ::remote::NetPeerCountReply,
-    &::remote::ETHBACKEND::StubInterface::PrepareAsyncNetPeerCount
->;
-
-using EngineGetPayloadV1Client = AsyncUnaryClient<
-    ::remote::ETHBACKEND::StubInterface,
-    ::remote::EngineGetPayloadRequest,
-    ::types::ExecutionPayload,
-    &::remote::ETHBACKEND::StubInterface::PrepareAsyncEngineGetPayloadV1
->;
-
-using EtherbaseAwaitable = unary_awaitable<
-    asio::io_context::executor_type,
-    EtherbaseClient,
-    ::remote::ETHBACKEND::StubInterface,
-    ::remote::EtherbaseRequest,
-    ::remote::EtherbaseReply
->;
-
-using ProtocolVersionAwaitable = unary_awaitable<
-    asio::io_context::executor_type,
-    ProtocolVersionClient,
-    ::remote::ETHBACKEND::StubInterface,
-    ::remote::ProtocolVersionRequest,
-    ::remote::ProtocolVersionReply
->;
-
-using NetVersionAwaitable = unary_awaitable<
-    asio::io_context::executor_type,
-    NetVersionClient,
-    ::remote::ETHBACKEND::StubInterface,
-    ::remote::NetVersionRequest,
-    ::remote::NetVersionReply
->;
-
-using ClientVersionAwaitable = unary_awaitable<
-    asio::io_context::executor_type,
-    ClientVersionClient,
-    ::remote::ETHBACKEND::StubInterface,
-    ::remote::ClientVersionRequest,
-    ::remote::ClientVersionReply
->;
-
-using NetPeerCountAwaitable = unary_awaitable<
-    asio::io_context::executor_type,
-    NetPeerCountClient,
-    ::remote::ETHBACKEND::StubInterface,
-    ::remote::NetPeerCountRequest,
-    ::remote::NetPeerCountReply
->;
-
-using EngineGetPayloadV1Awaitable = unary_awaitable<
-    asio::io_context::executor_type,
-    EngineGetPayloadV1Client,
-    ::remote::ETHBACKEND::StubInterface,
-    ::remote::EngineGetPayloadRequest,
-    ::types::ExecutionPayload
->;
-
-class BackEnd final {
+class BackEnd {
 public:
-    explicit BackEnd(asio::io_context& context, std::shared_ptr<grpc::Channel> channel, grpc::CompletionQueue* queue)
-    : BackEnd(context.get_executor(), ::remote::ETHBACKEND::NewStub(channel), queue) {}
-
-    explicit BackEnd(asio::io_context::executor_type executor, std::unique_ptr<::remote::ETHBACKEND::StubInterface> stub, grpc::CompletionQueue* queue)
-    : executor_(executor), stub_(std::move(stub)), queue_(queue) {
-        SILKRPC_TRACE << "BackEnd::ctor " << this << "\n";
-    }
-
-    ~BackEnd() {
-        SILKRPC_TRACE << "BackEnd::dtor " << this << "\n";
-    }
-
-    asio::awaitable<evmc::address> etherbase();
-    asio::awaitable<uint64_t> protocol_version();
-    asio::awaitable<uint64_t> net_version();
-    asio::awaitable<std::string> client_version();
-    asio::awaitable<uint64_t> net_peer_count();
-    asio::awaitable<ExecutionPayload> engine_get_payload_v1(uint64_t payload_id);
-
-    // just for testing
-    asio::awaitable<::types::ExecutionPayload> execution_payload_to_proto(ExecutionPayload payload);
-
-private:
-    evmc::address address_from_H160(const types::H160& h160);
-    silkworm::Bytes bytes_from_H128(const types::H128& h128);
-    types::H128* H128_from_bytes(const uint8_t* bytes);
-    types::H160* H160_from_address(const evmc::address& address);
-    types::H256* H256_from_bytes(const uint8_t* bytes);
-    silkworm::Bytes bytes_from_H256(const types::H256& h256);
-    intx::uint256 uint256_from_H256(const types::H256& h256);
-    types::H256* H256_from_uint256(const intx::uint256& n);
-    evmc::bytes32 bytes32_from_H256(const types::H256& h256);
-    types::H512* H512_from_bytes(const uint8_t* bytes);
-    silkworm::Bytes bytes_from_H512(types::H512& h512);
-    types::H1024* H1024_from_bytes(const uint8_t* bytes);
-    silkworm::Bytes bytes_from_H1024(types::H1024& h1024);
-    types::H2048* H2048_from_bytes(const uint8_t* bytes);
-    silkworm::Bytes bytes_from_H2048(types::H2048& h2048);
-
-    ExecutionPayload decode_execution_payload(const types::ExecutionPayload& execution_payload_grpc);
-    types::ExecutionPayload encode_execution_payload(const ExecutionPayload& execution_payload);
-
-    asio::io_context::executor_type executor_;
-    std::unique_ptr<::remote::ETHBACKEND::StubInterface> stub_;
-    grpc::CompletionQueue* queue_;
+    virtual ~BackEnd() = default;
+    virtual asio::awaitable<evmc::address> etherbase() = 0;
+    virtual asio::awaitable<uint64_t> protocol_version() = 0;
+    virtual asio::awaitable<uint64_t> net_version() = 0;
+    virtual asio::awaitable<std::string> client_version() = 0;
+    virtual asio::awaitable<uint64_t> net_peer_count() = 0;
+    virtual asio::awaitable<ExecutionPayload> engine_get_payload_v1(uint64_t payload_id) = 0;
 };
 
 } // namespace silkrpc::ethbackend
diff --git a/silkrpc/ethbackend/backend.cpp b/silkrpc/ethbackend/backend_grpc.cpp
similarity index 84%
rename from silkrpc/ethbackend/backend.cpp
rename to silkrpc/ethbackend/backend_grpc.cpp
index 30c15e77e..0dfc5fb2e 100644
--- a/silkrpc/ethbackend/backend.cpp
+++ b/silkrpc/ethbackend/backend_grpc.cpp
@@ -1,5 +1,5 @@
 /*
-    Copyright 2020 The Silkrpc Authors
+    Copyright 2022 The Silkrpc Authors
 
     Licensed under the Apache License, Version 2.0 (the "License");
     you may not use this file except in compliance with the License.
@@ -14,7 +14,7 @@
     limitations under the License.
 */
 
-#include "backend.hpp"
+#include "backend_grpc.hpp"
 
 #include <optional>
 #include <vector>
@@ -28,7 +28,7 @@
 
 namespace silkrpc::ethbackend {
 
-asio::awaitable<evmc::address> BackEnd::etherbase() {
+asio::awaitable<evmc::address> BackEndGrpc::etherbase() {
         const auto start_time = clock_time::now();
         EtherbaseAwaitable eb_awaitable{executor_, stub_, queue_};
         const auto reply = co_await eb_awaitable.async_call(::remote::EtherbaseRequest{}, asio::use_awaitable);
@@ -41,7 +41,7 @@ asio::awaitable<evmc::address> BackEnd::etherbase() {
         co_return evmc_address;
 }
 
-asio::awaitable<uint64_t> BackEnd::protocol_version() {
+asio::awaitable<uint64_t> BackEndGrpc::protocol_version() {
     const auto start_time = clock_time::now();
     ProtocolVersionAwaitable pv_awaitable{executor_, stub_, queue_};
     const auto reply = co_await pv_awaitable.async_call(::remote::ProtocolVersionRequest{}, asio::use_awaitable);
@@ -50,7 +50,7 @@ asio::awaitable<uint64_t> BackEnd::protocol_version() {
     co_return pv;
 }
 
-asio::awaitable<uint64_t> BackEnd::net_version() {
+asio::awaitable<uint64_t> BackEndGrpc::net_version() {
     const auto start_time = clock_time::now();
     NetVersionAwaitable nv_awaitable{executor_, stub_, queue_};
     const auto reply = co_await nv_awaitable.async_call(::remote::NetVersionRequest{}, asio::use_awaitable);
@@ -59,7 +59,7 @@ asio::awaitable<uint64_t> BackEnd::net_version() {
     co_return nv;
 }
 
-asio::awaitable<std::string> BackEnd::client_version() {
+asio::awaitable<std::string> BackEndGrpc::client_version() {
     const auto start_time = clock_time::now();
     ClientVersionAwaitable cv_awaitable{executor_, stub_, queue_};
     const auto reply = co_await cv_awaitable.async_call(::remote::ClientVersionRequest{}, asio::use_awaitable);
@@ -68,7 +68,7 @@ asio::awaitable<std::string> BackEnd::client_version() {
     co_return cv;
 }
 
-asio::awaitable<uint64_t> BackEnd::net_peer_count() {
+asio::awaitable<uint64_t> BackEndGrpc::net_peer_count() {
     const auto start_time = clock_time::now();
     NetPeerCountAwaitable npc_awaitable{executor_, stub_, queue_};
     const auto reply = co_await npc_awaitable.async_call(::remote::NetPeerCountRequest{}, asio::use_awaitable);
@@ -77,7 +77,7 @@ asio::awaitable<uint64_t> BackEnd::net_peer_count() {
     co_return count;
 }
 
-asio::awaitable<ExecutionPayload> BackEnd::engine_get_payload_v1(uint64_t payload_id) {
+asio::awaitable<ExecutionPayload> BackEndGrpc::engine_get_payload_v1(uint64_t payload_id) {
     const auto start_time = clock_time::now();
     EngineGetPayloadV1Awaitable npc_awaitable{executor_, stub_, queue_};
     ::remote::EngineGetPayloadRequest req;
@@ -88,11 +88,11 @@ asio::awaitable<ExecutionPayload> BackEnd::engine_get_payload_v1(uint64_t payloa
     co_return execution_payload;
 }
 
-asio::awaitable<::types::ExecutionPayload> BackEnd::execution_payload_to_proto(ExecutionPayload payload) {
+asio::awaitable<::types::ExecutionPayload> BackEndGrpc::execution_payload_to_proto(ExecutionPayload payload) {
     co_return encode_execution_payload(payload);
 }
 
-evmc::address BackEnd::address_from_H160(const types::H160& h160) {
+evmc::address BackEndGrpc::address_from_H160(const types::H160& h160) {
     uint64_t hi_hi = h160.hi().hi();
     uint64_t hi_lo = h160.hi().lo();
     uint32_t lo = h160.lo();
@@ -103,21 +103,21 @@ evmc::address BackEnd::address_from_H160(const types::H160& h160) {
     return address;
 }
 
-silkworm::Bytes BackEnd::bytes_from_H128(const types::H128& h128) {
+silkworm::Bytes BackEndGrpc::bytes_from_H128(const types::H128& h128) {
     silkworm::Bytes bytes(16, '\0');
     boost::endian::store_big_u64(&bytes[0], h128.hi());
     boost::endian::store_big_u64(&bytes[8], h128.lo());
     return bytes;
 }
 
-types::H128* BackEnd::H128_from_bytes(const uint8_t* bytes) {
+types::H128* BackEndGrpc::H128_from_bytes(const uint8_t* bytes) {
     auto h128{new types::H128()};
     h128->set_hi(boost::endian::load_big_u64(bytes));
     h128->set_lo(boost::endian::load_big_u64(bytes + 8));
     return h128;
 }
 
-types::H160* BackEnd::H160_from_address(const evmc::address& address) {
+types::H160* BackEndGrpc::H160_from_address(const evmc::address& address) {
     auto h160{new types::H160()};
     auto hi{H128_from_bytes(address.bytes)};
     h160->set_allocated_hi(hi);
@@ -125,7 +125,7 @@ types::H160* BackEnd::H160_from_address(const evmc::address& address) {
     return h160;
 }
 
-types::H256* BackEnd::H256_from_bytes(const uint8_t* bytes) {
+types::H256* BackEndGrpc::H256_from_bytes(const uint8_t* bytes) {
     auto h256{new types::H256()};
     auto hi{H128_from_bytes(bytes)};
     auto lo{H128_from_bytes(bytes + 16)};
@@ -134,7 +134,7 @@ types::H256* BackEnd::H256_from_bytes(const uint8_t* bytes) {
     return h256;
 }
 
-silkworm::Bytes BackEnd::bytes_from_H256(const types::H256& h256) {
+silkworm::Bytes BackEndGrpc::bytes_from_H256(const types::H256& h256) {
     silkworm::Bytes bytes(32, '\0');
     auto hi{h256.hi()};
     auto lo{h256.lo()};
@@ -143,7 +143,7 @@ silkworm::Bytes BackEnd::bytes_from_H256(const types::H256& h256) {
     return bytes;
 }
 
-intx::uint256 BackEnd::uint256_from_H256(const types::H256& h256) {
+intx::uint256 BackEndGrpc::uint256_from_H256(const types::H256& h256) {
     intx::uint256 n;
     n[3] = h256.hi().hi();
     n[2] = h256.hi().lo();
@@ -152,7 +152,7 @@ intx::uint256 BackEnd::uint256_from_H256(const types::H256& h256) {
     return n;
 }
 
-types::H256* BackEnd::H256_from_uint256(const intx::uint256& n) {
+types::H256* BackEndGrpc::H256_from_uint256(const intx::uint256& n) {
     auto h256{new types::H256()};
     auto hi{new types::H128()};
     auto lo{new types::H128()};
@@ -167,13 +167,13 @@ types::H256* BackEnd::H256_from_uint256(const intx::uint256& n) {
     return h256;
 }
 
-evmc::bytes32 BackEnd::bytes32_from_H256(const types::H256& h256) {
+evmc::bytes32 BackEndGrpc::bytes32_from_H256(const types::H256& h256) {
     evmc::bytes32 bytes32;
     std::memcpy(bytes32.bytes, bytes_from_H256(h256).data(), 32);
     return bytes32;
 }
 
-types::H512* BackEnd::H512_from_bytes(const uint8_t* bytes) {
+types::H512* BackEndGrpc::H512_from_bytes(const uint8_t* bytes) {
     auto h512{new types::H512()};
     auto hi{H256_from_bytes(bytes)};
     auto lo{H256_from_bytes(bytes + 32)};
@@ -182,7 +182,7 @@ types::H512* BackEnd::H512_from_bytes(const uint8_t* bytes) {
     return h512;
 }
 
-silkworm::Bytes BackEnd::bytes_from_H512(types::H512& h512) {
+silkworm::Bytes BackEndGrpc::bytes_from_H512(types::H512& h512) {
     silkworm::Bytes bytes(64, '\0');
     auto hi{h512.hi()};
     auto lo{h512.lo()};
@@ -191,7 +191,7 @@ silkworm::Bytes BackEnd::bytes_from_H512(types::H512& h512) {
     return bytes;
 }
 
-types::H1024* BackEnd::H1024_from_bytes(const uint8_t* bytes) {
+types::H1024* BackEndGrpc::H1024_from_bytes(const uint8_t* bytes) {
     auto h1024{new types::H1024()};
     auto hi{H512_from_bytes(bytes)};
     auto lo{H512_from_bytes(bytes + 64)};
@@ -200,7 +200,7 @@ types::H1024* BackEnd::H1024_from_bytes(const uint8_t* bytes) {
     return h1024;
 }
 
-silkworm::Bytes BackEnd::bytes_from_H1024(types::H1024& h1024) {
+silkworm::Bytes BackEndGrpc::bytes_from_H1024(types::H1024& h1024) {
     silkworm::Bytes bytes(128, '\0');
     auto hi{h1024.hi()};
     auto lo{h1024.lo()};
@@ -209,7 +209,7 @@ silkworm::Bytes BackEnd::bytes_from_H1024(types::H1024& h1024) {
     return bytes;
 }
 
-types::H2048* BackEnd::H2048_from_bytes(const uint8_t* bytes) {
+types::H2048* BackEndGrpc::H2048_from_bytes(const uint8_t* bytes) {
     auto h2048{new types::H2048()};
     auto hi{H1024_from_bytes(bytes)};
     auto lo{H1024_from_bytes(bytes + 128)};
@@ -218,7 +218,7 @@ types::H2048* BackEnd::H2048_from_bytes(const uint8_t* bytes) {
     return h2048;
 }
 
-silkworm::Bytes BackEnd::bytes_from_H2048(types::H2048& h2048) {
+silkworm::Bytes BackEndGrpc::bytes_from_H2048(types::H2048& h2048) {
     silkworm::Bytes bytes(256, '\0');
     auto hi{h2048.hi()};
     auto lo{h2048.lo()};
@@ -227,7 +227,7 @@ silkworm::Bytes BackEnd::bytes_from_H2048(types::H2048& h2048) {
     return bytes;
 }
 
-ExecutionPayload BackEnd::decode_execution_payload(const types::ExecutionPayload& execution_payload_grpc) {
+ExecutionPayload BackEndGrpc::decode_execution_payload(const types::ExecutionPayload& execution_payload_grpc) {
     auto state_root_h256{execution_payload_grpc.stateroot()};
     auto receipts_root_h256{execution_payload_grpc.receiptroot()};
     auto block_hash_h256{execution_payload_grpc.blockhash()};
@@ -264,7 +264,7 @@ ExecutionPayload BackEnd::decode_execution_payload(const types::ExecutionPayload
     };
 }
 
-types::ExecutionPayload BackEnd::encode_execution_payload(const ExecutionPayload& execution_payload) {
+types::ExecutionPayload BackEndGrpc::encode_execution_payload(const ExecutionPayload& execution_payload) {
     types::ExecutionPayload execution_payload_grpc;
     // Numerical parameters
     execution_payload_grpc.set_blocknumber(execution_payload.number);
diff --git a/silkrpc/ethbackend/backend_grpc.hpp b/silkrpc/ethbackend/backend_grpc.hpp
new file mode 100644
index 000000000..53f755a05
--- /dev/null
+++ b/silkrpc/ethbackend/backend_grpc.hpp
@@ -0,0 +1,179 @@
+/*
+    Copyright 2022 The Silkrpc Authors
+
+    Licensed 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.
+*/
+
+#ifndef SILKRPC_ETHBACKEND_BACKEND_GRPC_HPP_
+#define SILKRPC_ETHBACKEND_BACKEND_GRPC_HPP_
+
+#include <utility>
+#include <string>
+#include <memory>
+
+#include <asio/io_context.hpp>
+#include <asio/use_awaitable.hpp>
+#include <evmc/evmc.hpp>
+
+#include <silkrpc/grpc/awaitables.hpp>
+#include <silkrpc/grpc/async_unary_client.hpp>
+#include <silkrpc/interfaces/remote/ethbackend.grpc.pb.h>
+#include <silkrpc/interfaces/types/types.pb.h>
+#include <silkrpc/types/execution_payload.hpp>
+#include <silkrpc/ethbackend/backend.hpp>
+
+namespace silkrpc::ethbackend {
+
+using EtherbaseClient = AsyncUnaryClient<
+    ::remote::ETHBACKEND::StubInterface,
+    ::remote::EtherbaseRequest,
+    ::remote::EtherbaseReply,
+    &::remote::ETHBACKEND::StubInterface::PrepareAsyncEtherbase
+>;
+
+using ProtocolVersionClient = AsyncUnaryClient<
+    ::remote::ETHBACKEND::StubInterface,
+    ::remote::ProtocolVersionRequest,
+    ::remote::ProtocolVersionReply,
+    &::remote::ETHBACKEND::StubInterface::PrepareAsyncProtocolVersion
+>;
+
+using NetVersionClient = AsyncUnaryClient<
+    ::remote::ETHBACKEND::StubInterface,
+    ::remote::NetVersionRequest,
+    ::remote::NetVersionReply,
+    &::remote::ETHBACKEND::StubInterface::PrepareAsyncNetVersion
+>;
+
+using ClientVersionClient = AsyncUnaryClient<
+    ::remote::ETHBACKEND::StubInterface,
+    ::remote::ClientVersionRequest,
+    ::remote::ClientVersionReply,
+    &::remote::ETHBACKEND::StubInterface::PrepareAsyncClientVersion
+>;
+
+using NetPeerCountClient = AsyncUnaryClient<
+    ::remote::ETHBACKEND::StubInterface,
+    ::remote::NetPeerCountRequest,
+    ::remote::NetPeerCountReply,
+    &::remote::ETHBACKEND::StubInterface::PrepareAsyncNetPeerCount
+>;
+
+using EngineGetPayloadV1Client = AsyncUnaryClient<
+    ::remote::ETHBACKEND::StubInterface,
+    ::remote::EngineGetPayloadRequest,
+    ::types::ExecutionPayload,
+    &::remote::ETHBACKEND::StubInterface::PrepareAsyncEngineGetPayloadV1
+>;
+
+using EtherbaseAwaitable = unary_awaitable<
+    asio::io_context::executor_type,
+    EtherbaseClient,
+    ::remote::ETHBACKEND::StubInterface,
+    ::remote::EtherbaseRequest,
+    ::remote::EtherbaseReply
+>;
+
+using ProtocolVersionAwaitable = unary_awaitable<
+    asio::io_context::executor_type,
+    ProtocolVersionClient,
+    ::remote::ETHBACKEND::StubInterface,
+    ::remote::ProtocolVersionRequest,
+    ::remote::ProtocolVersionReply
+>;
+
+using NetVersionAwaitable = unary_awaitable<
+    asio::io_context::executor_type,
+    NetVersionClient,
+    ::remote::ETHBACKEND::StubInterface,
+    ::remote::NetVersionRequest,
+    ::remote::NetVersionReply
+>;
+
+using ClientVersionAwaitable = unary_awaitable<
+    asio::io_context::executor_type,
+    ClientVersionClient,
+    ::remote::ETHBACKEND::StubInterface,
+    ::remote::ClientVersionRequest,
+    ::remote::ClientVersionReply
+>;
+
+using NetPeerCountAwaitable = unary_awaitable<
+    asio::io_context::executor_type,
+    NetPeerCountClient,
+    ::remote::ETHBACKEND::StubInterface,
+    ::remote::NetPeerCountRequest,
+    ::remote::NetPeerCountReply
+>;
+
+using EngineGetPayloadV1Awaitable = unary_awaitable<
+    asio::io_context::executor_type,
+    EngineGetPayloadV1Client,
+    ::remote::ETHBACKEND::StubInterface,
+    ::remote::EngineGetPayloadRequest,
+    ::types::ExecutionPayload
+>;
+
+
+class BackEndGrpc final: public BackEnd {
+public:
+    explicit BackEndGrpc(asio::io_context& context, std::shared_ptr<grpc::Channel> channel, grpc::CompletionQueue* queue)
+    : BackEndGrpc(context.get_executor(), ::remote::ETHBACKEND::NewStub(channel), queue) {}
+
+    explicit BackEndGrpc(asio::io_context::executor_type executor, std::unique_ptr<::remote::ETHBACKEND::StubInterface> stub, grpc::CompletionQueue* queue)
+    : executor_(executor), stub_(std::move(stub)), queue_(queue) {
+        SILKRPC_TRACE << "BackEnd::ctor " << this << "\n";
+    }
+
+    ~BackEndGrpc() {
+        SILKRPC_TRACE << "BackEnd::dtor " << this << "\n";
+    }
+
+    asio::awaitable<evmc::address> etherbase();
+    asio::awaitable<uint64_t> protocol_version();
+    asio::awaitable<uint64_t> net_version();
+    asio::awaitable<std::string> client_version();
+    asio::awaitable<uint64_t> net_peer_count();
+    asio::awaitable<ExecutionPayload> engine_get_payload_v1(uint64_t payload_id);
+
+    // just for testing
+    asio::awaitable<::types::ExecutionPayload> execution_payload_to_proto(ExecutionPayload payload);
+
+private:
+    evmc::address address_from_H160(const types::H160& h160);
+    silkworm::Bytes bytes_from_H128(const types::H128& h128);
+    types::H128* H128_from_bytes(const uint8_t* bytes);
+    types::H160* H160_from_address(const evmc::address& address);
+    types::H256* H256_from_bytes(const uint8_t* bytes);
+    silkworm::Bytes bytes_from_H256(const types::H256& h256);
+    intx::uint256 uint256_from_H256(const types::H256& h256);
+    types::H256* H256_from_uint256(const intx::uint256& n);
+    evmc::bytes32 bytes32_from_H256(const types::H256& h256);
+    types::H512* H512_from_bytes(const uint8_t* bytes);
+    silkworm::Bytes bytes_from_H512(types::H512& h512);
+    types::H1024* H1024_from_bytes(const uint8_t* bytes);
+    silkworm::Bytes bytes_from_H1024(types::H1024& h1024);
+    types::H2048* H2048_from_bytes(const uint8_t* bytes);
+    silkworm::Bytes bytes_from_H2048(types::H2048& h2048);
+
+    ExecutionPayload decode_execution_payload(const types::ExecutionPayload& execution_payload_grpc);
+    types::ExecutionPayload encode_execution_payload(const ExecutionPayload& execution_payload);
+
+    asio::io_context::executor_type executor_;
+    std::unique_ptr<::remote::ETHBACKEND::StubInterface> stub_;
+    grpc::CompletionQueue* queue_;
+};
+
+} // namespace silkrpc::ethbackend
+
+#endif // SILKRPC_ETHBACKEND_BACKEND_GRPC_HPP_
diff --git a/silkrpc/ethbackend/backend_test.cpp b/silkrpc/ethbackend/backend_grpc_test.cpp
similarity index 96%
rename from silkrpc/ethbackend/backend_test.cpp
rename to silkrpc/ethbackend/backend_grpc_test.cpp
index d07264ce7..b9f631f6a 100644
--- a/silkrpc/ethbackend/backend_test.cpp
+++ b/silkrpc/ethbackend/backend_grpc_test.cpp
@@ -1,5 +1,5 @@
 /*
-   Copyright 2021 The Silkrpc Authors
+   Copyright 2022 The Silkrpc Authors
 
    Licensed under the Apache License, Version 2.0 (the "License");
    you may not use this file except in compliance with the License.
@@ -14,7 +14,7 @@
    limitations under the License.
 */
 
-#include "backend.hpp"
+#include "backend_grpc.hpp"
 
 #include <string>
 #include <system_error>
@@ -242,13 +242,13 @@ asio::awaitable<R> test_comethod(::remote::ETHBACKEND::Service* service, Args...
     co_return co_await method_proxy(args...);
 }
 
-auto test_etherbase = test_comethod<ethbackend::BackEnd, &ethbackend::BackEnd::etherbase, evmc::address>;
-auto test_protocol_version = test_comethod<ethbackend::BackEnd, &ethbackend::BackEnd::protocol_version, uint64_t>;
-auto test_net_version = test_comethod<ethbackend::BackEnd, &ethbackend::BackEnd::net_version, uint64_t>;
-auto test_client_version = test_comethod<ethbackend::BackEnd, &ethbackend::BackEnd::client_version, std::string>;
-auto test_net_peer_count = test_comethod<ethbackend::BackEnd, &ethbackend::BackEnd::net_peer_count, uint64_t>;
-auto test_engine_get_payload_v1 = test_comethod<ethbackend::BackEnd, &ethbackend::BackEnd::engine_get_payload_v1, ExecutionPayload, uint64_t>;
-auto test_execution_payload_to_proto = test_comethod<ethbackend::BackEnd, &ethbackend::BackEnd::execution_payload_to_proto, ::types::ExecutionPayload, ExecutionPayload>;
+auto test_etherbase = test_comethod<ethbackend::BackEndGrpc, &ethbackend::BackEndGrpc::etherbase, evmc::address>;
+auto test_protocol_version = test_comethod<ethbackend::BackEndGrpc, &ethbackend::BackEndGrpc::protocol_version, uint64_t>;
+auto test_net_version = test_comethod<ethbackend::BackEndGrpc, &ethbackend::BackEndGrpc::net_version, uint64_t>;
+auto test_client_version = test_comethod<ethbackend::BackEndGrpc, &ethbackend::BackEndGrpc::client_version, std::string>;
+auto test_net_peer_count = test_comethod<ethbackend::BackEndGrpc, &ethbackend::BackEndGrpc::net_peer_count, uint64_t>;
+auto test_engine_get_payload_v1 = test_comethod<ethbackend::BackEndGrpc, &ethbackend::BackEndGrpc::engine_get_payload_v1, ExecutionPayload, uint64_t>;
+auto test_execution_payload_to_proto = test_comethod<ethbackend::BackEndGrpc, &ethbackend::BackEndGrpc::execution_payload_to_proto, ::types::ExecutionPayload, ExecutionPayload>;
 
 TEST_CASE("BackEnd::etherbase", "[silkrpc][ethbackend][backend]") {
     SILKRPC_LOG_VERBOSITY(LogLevel::None);
diff --git a/silkrpc/http/methods.hpp b/silkrpc/http/methods.hpp
index 1b22e14c2..205e969e0 100644
--- a/silkrpc/http/methods.hpp
+++ b/silkrpc/http/methods.hpp
@@ -104,6 +104,7 @@ constexpr const char* k_tg_issuance{"tg_issuance"};
 
 constexpr const char* k_parity_getBlockReceipts{"parity_getBlockReceipts"};
 
+constexpr const char* k_engine_getPayloadV1{"engine_getPayloadV1"};
 } // namespace silkrpc::http::method
 
 #endif // SILKRPC_HTTP_METHODS_HPP_
diff --git a/silkrpc/interfaces/remote/kv.pb.h b/silkrpc/interfaces/remote/kv.pb.h
index a7b10b230..b02e617d8 100644
--- a/silkrpc/interfaces/remote/kv.pb.h
+++ b/silkrpc/interfaces/remote/kv.pb.h
@@ -1070,6 +1070,7 @@ class StateChangeBatch PROTOBUF_FINAL :
     kChangeBatchFieldNumber = 2,
     kDatabaseViewIDFieldNumber = 1,
     kPendingBlockBaseFeeFieldNumber = 3,
+    kBlockGasLimitFieldNumber = 4,
   };
   // repeated .remote.StateChange changeBatch = 2;
   int changebatch_size() const;
@@ -1107,6 +1108,15 @@ class StateChangeBatch PROTOBUF_FINAL :
   void _internal_set_pendingblockbasefee(::PROTOBUF_NAMESPACE_ID::uint64 value);
   public:
 
+  // uint64 blockGasLimit = 4;
+  void clear_blockgaslimit();
+  ::PROTOBUF_NAMESPACE_ID::uint64 blockgaslimit() const;
+  void set_blockgaslimit(::PROTOBUF_NAMESPACE_ID::uint64 value);
+  private:
+  ::PROTOBUF_NAMESPACE_ID::uint64 _internal_blockgaslimit() const;
+  void _internal_set_blockgaslimit(::PROTOBUF_NAMESPACE_ID::uint64 value);
+  public:
+
   // @@protoc_insertion_point(class_scope:remote.StateChangeBatch)
  private:
   class _Internal;
@@ -1117,6 +1127,7 @@ class StateChangeBatch PROTOBUF_FINAL :
   ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::remote::StateChange > changebatch_;
   ::PROTOBUF_NAMESPACE_ID::uint64 databaseviewid_;
   ::PROTOBUF_NAMESPACE_ID::uint64 pendingblockbasefee_;
+  ::PROTOBUF_NAMESPACE_ID::uint64 blockgaslimit_;
   mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_;
   friend struct ::TableStruct_remote_2fkv_2eproto;
 };
@@ -2387,6 +2398,26 @@ inline void StateChangeBatch::set_pendingblockbasefee(::PROTOBUF_NAMESPACE_ID::u
   // @@protoc_insertion_point(field_set:remote.StateChangeBatch.pendingBlockBaseFee)
 }
 
+// uint64 blockGasLimit = 4;
+inline void StateChangeBatch::clear_blockgaslimit() {
+  blockgaslimit_ = PROTOBUF_ULONGLONG(0);
+}
+inline ::PROTOBUF_NAMESPACE_ID::uint64 StateChangeBatch::_internal_blockgaslimit() const {
+  return blockgaslimit_;
+}
+inline ::PROTOBUF_NAMESPACE_ID::uint64 StateChangeBatch::blockgaslimit() const {
+  // @@protoc_insertion_point(field_get:remote.StateChangeBatch.blockGasLimit)
+  return _internal_blockgaslimit();
+}
+inline void StateChangeBatch::_internal_set_blockgaslimit(::PROTOBUF_NAMESPACE_ID::uint64 value) {
+  
+  blockgaslimit_ = value;
+}
+inline void StateChangeBatch::set_blockgaslimit(::PROTOBUF_NAMESPACE_ID::uint64 value) {
+  _internal_set_blockgaslimit(value);
+  // @@protoc_insertion_point(field_set:remote.StateChangeBatch.blockGasLimit)
+}
+
 // -------------------------------------------------------------------
 
 // StateChange
diff --git a/tests/integration/jsonrpc_commands_goerli.json b/tests/integration/jsonrpc_commands_goerli.json
index ac8d2f807..518a35376 100644
--- a/tests/integration/jsonrpc_commands_goerli.json
+++ b/tests/integration/jsonrpc_commands_goerli.json
@@ -1438,5 +1438,21 @@
               }
             }
         }
+    },
+    {
+        "request": {
+            "jsonrpc": "2.0",
+            "method": "engine_getPayloadV1",
+            "params": ["0x0000000000000001"],
+            "id": 1
+        },
+        "response": {
+            "error": {
+                "code": 100,
+                "message": "unknown method EngineGetPayloadV1 for service remote.ETHBACKEND"
+            },
+            "id": 1,
+            "jsonrpc": "2.0"
+        }
     }
 ]
diff --git a/tests/integration/run_jsonrpc_commands.py b/tests/integration/run_jsonrpc_commands.py
index 0ad9eafa2..70724c1da 100755
--- a/tests/integration/run_jsonrpc_commands.py
+++ b/tests/integration/run_jsonrpc_commands.py
@@ -8,6 +8,14 @@
 import getopt
 import jsondiff
 
+def get_target(silk: bool, method: str):
+    "Determine where silkrpc is supposed to be serving at."
+    if "engine_" in method:
+        return "localhost:8550"
+    if silk:
+        return "localhost:51515"
+
+    return "localhost:8545"
 
 def run_shell_command(command: str, expected_response: str, exit_on_fail):
     """ Run the specified command as shell. If exact result or error don't care, they are null but present in expected_response. """
@@ -36,20 +44,16 @@ def run_tests(json_filename, verbose, silk, exit_on_fail, req_test):
         test_number = 0
         for json_rpc in jsonrpc_commands:
             if req_test in (-1, test_number):
-                request = json.dumps(json_rpc["request"])
+                request = json_rpc["request"]
+                request_dumps = json.dumps(request)
+                target = get_target(silk, request["method"])
                 if verbose:
-                    print (str(test_number) + ") " + request)
+                    print (str(test_number) + ") " + request_dumps)
                 response = json_rpc["response"]
-                if silk:
-                    run_shell_command(
-                        '''curl --silent -X POST -H "Content-Type: application/json" --data \'''' +
-                        request + '''\' localhost:51515''',
-                        response, exit_on_fail)
-                else:
-                    run_shell_command(
-                        '''curl --silent -X POST -H "Content-Type: application/json" --data \'''' +
-                        request + '''\' localhost:8545''',
-                        response, exit_on_fail)
+                run_shell_command(
+                    '''curl --silent -X POST -H "Content-Type: application/json" --data \'''' +
+                    request_dumps + '''\' ''' + target,
+                    response, exit_on_fail)
             test_number = test_number + 1
 
 #