aboutsummaryrefslogtreecommitdiff
path: root/src
diff options
context:
space:
mode:
authorAlex <lx@deuxfleurs.fr>2025-02-05 19:52:00 +0000
committerAlex <lx@deuxfleurs.fr>2025-02-05 19:52:00 +0000
commitd3226bfa91d4500063c5c287c6256729dcbb3f88 (patch)
treef27dad5a52a165d91edc4ffbddca16d2b1015249 /src
parent47e87c8739d7f0d420ea6a01fea99e638414baeb (diff)
parentaf67626ab2bd32e94ab521607574737939a7edf3 (diff)
downloadgarage-d3226bfa91d4500063c5c287c6256729dcbb3f88.tar.gz
garage-d3226bfa91d4500063c5c287c6256729dcbb3f88.zip
Merge pull request 'remove uses of #[async_trait]' (#952) from remove-async-trait into mainHEADmain
Reviewed-on: https://git.deuxfleurs.fr/Deuxfleurs/garage/pulls/952
Diffstat (limited to 'src')
-rw-r--r--src/api/admin/api_server.rs2
-rw-r--r--src/api/common/Cargo.toml1
-rw-r--r--src/api/common/generic_server.rs12
-rw-r--r--src/api/k2v/Cargo.toml1
-rw-r--r--src/api/k2v/api_server.rs3
-rw-r--r--src/api/s3/Cargo.toml1
-rw-r--r--src/api/s3/api_server.rs3
-rw-r--r--src/block/manager.rs2
-rw-r--r--src/garage/admin/mod.rs31
-rw-r--r--src/garage/repair/online.rs9
-rw-r--r--src/model/k2v/rpc.rs2
-rw-r--r--src/net/Cargo.toml1
-rw-r--r--src/net/client.rs2
-rw-r--r--src/net/endpoint.rs32
-rw-r--r--src/net/netapp.rs2
-rw-r--r--src/net/peering.rs3
-rw-r--r--src/net/recv.rs2
-rw-r--r--src/net/send.rs2
-rw-r--r--src/net/server.rs2
-rw-r--r--src/rpc/system.rs2
-rw-r--r--src/table/gc.rs2
-rw-r--r--src/table/sync.rs1
-rw-r--r--src/table/table.rs2
23 files changed, 40 insertions, 80 deletions
diff --git a/src/api/admin/api_server.rs b/src/api/admin/api_server.rs
index e39fa1ba..6f0c474f 100644
--- a/src/api/admin/api_server.rs
+++ b/src/api/admin/api_server.rs
@@ -2,7 +2,6 @@ use std::collections::HashMap;
use std::sync::Arc;
use argon2::password_hash::PasswordHash;
-use async_trait::async_trait;
use http::header::{ACCESS_CONTROL_ALLOW_METHODS, ACCESS_CONTROL_ALLOW_ORIGIN, ALLOW};
use hyper::{body::Incoming as IncomingBody, Request, Response, StatusCode};
@@ -221,7 +220,6 @@ impl AdminApiServer {
}
}
-#[async_trait]
impl ApiHandler for AdminApiServer {
const API_NAME: &'static str = "admin";
const API_NAME_DISPLAY: &'static str = "Admin";
diff --git a/src/api/common/Cargo.toml b/src/api/common/Cargo.toml
index 842662c4..5b9cf479 100644
--- a/src/api/common/Cargo.toml
+++ b/src/api/common/Cargo.toml
@@ -18,7 +18,6 @@ garage_model.workspace = true
garage_table.workspace = true
garage_util.workspace = true
-async-trait.workspace = true
bytes.workspace = true
chrono.workspace = true
crypto-common.workspace = true
diff --git a/src/api/common/generic_server.rs b/src/api/common/generic_server.rs
index d92a3465..6ddc2ff2 100644
--- a/src/api/common/generic_server.rs
+++ b/src/api/common/generic_server.rs
@@ -4,8 +4,6 @@ use std::os::unix::fs::PermissionsExt;
use std::sync::Arc;
use std::time::Duration;
-use async_trait::async_trait;
-
use futures::future::Future;
use futures::stream::{futures_unordered::FuturesUnordered, StreamExt};
@@ -47,7 +45,6 @@ pub trait ApiError: std::error::Error + Send + Sync + 'static {
fn http_body(&self, garage_region: &str, path: &str) -> ErrorBody;
}
-#[async_trait]
pub trait ApiHandler: Send + Sync + 'static {
const API_NAME: &'static str;
const API_NAME_DISPLAY: &'static str;
@@ -56,11 +53,11 @@ pub trait ApiHandler: Send + Sync + 'static {
type Error: ApiError;
fn parse_endpoint(&self, r: &Request<IncomingBody>) -> Result<Self::Endpoint, Self::Error>;
- async fn handle(
+ fn handle(
&self,
req: Request<IncomingBody>,
endpoint: Self::Endpoint,
- ) -> Result<Response<BoxBody<Self::Error>>, Self::Error>;
+ ) -> impl Future<Output = Result<Response<BoxBody<Self::Error>>, Self::Error>> + Send;
}
pub struct ApiServer<A: ApiHandler> {
@@ -248,13 +245,11 @@ impl<A: ApiHandler> ApiServer<A> {
// ==== helper functions ====
-#[async_trait]
pub trait Accept: Send + Sync + 'static {
type Stream: AsyncRead + AsyncWrite + Send + Sync + 'static;
- async fn accept(&self) -> std::io::Result<(Self::Stream, String)>;
+ fn accept(&self) -> impl Future<Output = std::io::Result<(Self::Stream, String)>> + Send;
}
-#[async_trait]
impl Accept for TcpListener {
type Stream = TcpStream;
async fn accept(&self) -> std::io::Result<(Self::Stream, String)> {
@@ -266,7 +261,6 @@ impl Accept for TcpListener {
pub struct UnixListenerOn(pub UnixListener, pub String);
-#[async_trait]
impl Accept for UnixListenerOn {
type Stream = UnixStream;
async fn accept(&self) -> std::io::Result<(Self::Stream, String)> {
diff --git a/src/api/k2v/Cargo.toml b/src/api/k2v/Cargo.toml
index d4e26efa..e3ebedca 100644
--- a/src/api/k2v/Cargo.toml
+++ b/src/api/k2v/Cargo.toml
@@ -19,7 +19,6 @@ garage_table.workspace = true
garage_util = { workspace = true, features = [ "k2v" ] }
garage_api_common.workspace = true
-async-trait.workspace = true
base64.workspace = true
err-derive.workspace = true
tracing.workspace = true
diff --git a/src/api/k2v/api_server.rs b/src/api/k2v/api_server.rs
index 31e07762..eb276f5b 100644
--- a/src/api/k2v/api_server.rs
+++ b/src/api/k2v/api_server.rs
@@ -1,7 +1,5 @@
use std::sync::Arc;
-use async_trait::async_trait;
-
use hyper::{body::Incoming as IncomingBody, Method, Request, Response};
use tokio::sync::watch;
@@ -48,7 +46,6 @@ impl K2VApiServer {
}
}
-#[async_trait]
impl ApiHandler for K2VApiServer {
const API_NAME: &'static str = "k2v";
const API_NAME_DISPLAY: &'static str = "K2V";
diff --git a/src/api/s3/Cargo.toml b/src/api/s3/Cargo.toml
index a1751c9f..387e45db 100644
--- a/src/api/s3/Cargo.toml
+++ b/src/api/s3/Cargo.toml
@@ -24,7 +24,6 @@ garage_api_common.workspace = true
aes-gcm.workspace = true
async-compression.workspace = true
-async-trait.workspace = true
base64.workspace = true
bytes.workspace = true
chrono.workspace = true
diff --git a/src/api/s3/api_server.rs b/src/api/s3/api_server.rs
index ed71b108..bf48bba1 100644
--- a/src/api/s3/api_server.rs
+++ b/src/api/s3/api_server.rs
@@ -1,7 +1,5 @@
use std::sync::Arc;
-use async_trait::async_trait;
-
use hyper::header;
use hyper::{body::Incoming as IncomingBody, Request, Response};
use tokio::sync::watch;
@@ -70,7 +68,6 @@ impl S3ApiServer {
}
}
-#[async_trait]
impl ApiHandler for S3ApiServer {
const API_NAME: &'static str = "s3";
const API_NAME_DISPLAY: &'static str = "S3";
diff --git a/src/block/manager.rs b/src/block/manager.rs
index 40b177a2..537e1fc1 100644
--- a/src/block/manager.rs
+++ b/src/block/manager.rs
@@ -4,7 +4,6 @@ use std::sync::Arc;
use std::time::Duration;
use arc_swap::{ArcSwap, ArcSwapOption};
-use async_trait::async_trait;
use bytes::Bytes;
use rand::prelude::*;
use serde::{Deserialize, Serialize};
@@ -688,7 +687,6 @@ impl BlockManager {
}
}
-#[async_trait]
impl StreamingEndpointHandler<BlockRpc> for BlockManager {
async fn handle(self: &Arc<Self>, mut message: Req<BlockRpc>, _from: NodeID) -> Resp<BlockRpc> {
match message.msg() {
diff --git a/src/garage/admin/mod.rs b/src/garage/admin/mod.rs
index e2468143..ea414b56 100644
--- a/src/garage/admin/mod.rs
+++ b/src/garage/admin/mod.rs
@@ -4,9 +4,11 @@ mod key;
use std::collections::HashMap;
use std::fmt::Write;
+use std::future::Future;
use std::sync::Arc;
-use async_trait::async_trait;
+use futures::future::FutureExt;
+
use serde::{Deserialize, Serialize};
use format_table::format_table_to_string;
@@ -505,22 +507,25 @@ impl AdminRpcHandler {
}
}
-#[async_trait]
impl EndpointHandler<AdminRpc> for AdminRpcHandler {
- async fn handle(
+ fn handle(
self: &Arc<Self>,
message: &AdminRpc,
_from: NodeID,
- ) -> Result<AdminRpc, Error> {
- match message {
- AdminRpc::BucketOperation(bo) => self.handle_bucket_cmd(bo).await,
- AdminRpc::KeyOperation(ko) => self.handle_key_cmd(ko).await,
- AdminRpc::LaunchRepair(opt) => self.handle_launch_repair(opt.clone()).await,
- AdminRpc::Stats(opt) => self.handle_stats(opt.clone()).await,
- AdminRpc::Worker(wo) => self.handle_worker_cmd(wo).await,
- AdminRpc::BlockOperation(bo) => self.handle_block_cmd(bo).await,
- AdminRpc::MetaOperation(mo) => self.handle_meta_cmd(mo).await,
- m => Err(GarageError::unexpected_rpc_message(m).into()),
+ ) -> impl Future<Output = Result<AdminRpc, Error>> + Send {
+ let self2 = self.clone();
+ async move {
+ match message {
+ AdminRpc::BucketOperation(bo) => self2.handle_bucket_cmd(bo).await,
+ AdminRpc::KeyOperation(ko) => self2.handle_key_cmd(ko).await,
+ AdminRpc::LaunchRepair(opt) => self2.handle_launch_repair(opt.clone()).await,
+ AdminRpc::Stats(opt) => self2.handle_stats(opt.clone()).await,
+ AdminRpc::Worker(wo) => self2.handle_worker_cmd(wo).await,
+ AdminRpc::BlockOperation(bo) => self2.handle_block_cmd(bo).await,
+ AdminRpc::MetaOperation(mo) => self2.handle_meta_cmd(mo).await,
+ m => Err(GarageError::unexpected_rpc_message(m).into()),
+ }
}
+ .boxed()
}
}
diff --git a/src/garage/repair/online.rs b/src/garage/repair/online.rs
index 2c5227d2..47883f97 100644
--- a/src/garage/repair/online.rs
+++ b/src/garage/repair/online.rs
@@ -1,3 +1,4 @@
+use std::future::Future;
use std::sync::Arc;
use std::time::Duration;
@@ -93,17 +94,16 @@ pub async fn launch_online_repair(
// ----
-#[async_trait]
trait TableRepair: Send + Sync + 'static {
type T: TableSchema;
fn table(garage: &Garage) -> &Table<Self::T, TableShardedReplication>;
- async fn process(
+ fn process(
&mut self,
garage: &Garage,
entry: <<Self as TableRepair>::T as TableSchema>::E,
- ) -> Result<bool, Error>;
+ ) -> impl Future<Output = Result<bool, Error>> + Send;
}
struct TableRepairWorker<T: TableRepair> {
@@ -174,7 +174,6 @@ impl<R: TableRepair> Worker for TableRepairWorker<R> {
struct RepairVersions;
-#[async_trait]
impl TableRepair for RepairVersions {
type T = VersionTable;
@@ -221,7 +220,6 @@ impl TableRepair for RepairVersions {
struct RepairBlockRefs;
-#[async_trait]
impl TableRepair for RepairBlockRefs {
type T = BlockRefTable;
@@ -257,7 +255,6 @@ impl TableRepair for RepairBlockRefs {
struct RepairMpu;
-#[async_trait]
impl TableRepair for RepairMpu {
type T = MultipartUploadTable;
diff --git a/src/model/k2v/rpc.rs b/src/model/k2v/rpc.rs
index a1bf6ee0..821f4549 100644
--- a/src/model/k2v/rpc.rs
+++ b/src/model/k2v/rpc.rs
@@ -10,7 +10,6 @@ use std::convert::TryInto;
use std::sync::{Arc, Mutex, MutexGuard};
use std::time::{Duration, Instant};
-use async_trait::async_trait;
use futures::stream::FuturesUnordered;
use futures::StreamExt;
use serde::{Deserialize, Serialize};
@@ -537,7 +536,6 @@ impl K2VRpcHandler {
}
}
-#[async_trait]
impl EndpointHandler<K2VRpc> for K2VRpcHandler {
async fn handle(self: &Arc<Self>, message: &K2VRpc, _from: NodeID) -> Result<K2VRpc, Error> {
match message {
diff --git a/src/net/Cargo.toml b/src/net/Cargo.toml
index 686aaaea..c0b47a6e 100644
--- a/src/net/Cargo.toml
+++ b/src/net/Cargo.toml
@@ -30,7 +30,6 @@ rand.workspace = true
log.workspace = true
arc-swap.workspace = true
-async-trait.workspace = true
err-derive.workspace = true
bytes.workspace = true
cfg-if.workspace = true
diff --git a/src/net/client.rs b/src/net/client.rs
index 607dd173..20e1dacd 100644
--- a/src/net/client.rs
+++ b/src/net/client.rs
@@ -6,7 +6,6 @@ use std::sync::{Arc, Mutex};
use std::task::Poll;
use arc_swap::ArcSwapOption;
-use async_trait::async_trait;
use bytes::Bytes;
use log::{debug, error, trace};
@@ -220,7 +219,6 @@ impl ClientConn {
impl SendLoop for ClientConn {}
-#[async_trait]
impl RecvLoop for ClientConn {
fn recv_handler(self: &Arc<Self>, id: RequestID, stream: ByteStream) {
trace!("ClientConn recv_handler {}", id);
diff --git a/src/net/endpoint.rs b/src/net/endpoint.rs
index 3cafafeb..d46acc42 100644
--- a/src/net/endpoint.rs
+++ b/src/net/endpoint.rs
@@ -1,8 +1,9 @@
+use std::future::Future;
use std::marker::PhantomData;
use std::sync::Arc;
use arc_swap::ArcSwapOption;
-use async_trait::async_trait;
+use futures::future::{BoxFuture, FutureExt};
use crate::error::Error;
use crate::message::*;
@@ -14,19 +15,17 @@ use crate::netapp::*;
/// attached to the response..
///
/// The handler object should be in an Arc, see `Endpoint::set_handler`
-#[async_trait]
pub trait StreamingEndpointHandler<M>: Send + Sync
where
M: Message,
{
- async fn handle(self: &Arc<Self>, m: Req<M>, from: NodeID) -> Resp<M>;
+ fn handle(self: &Arc<Self>, m: Req<M>, from: NodeID) -> impl Future<Output = Resp<M>> + Send;
}
/// If one simply wants to use an endpoint in a client fashion,
/// without locally serving requests to that endpoint,
/// use the unit type `()` as the handler type:
/// it will panic if it is ever made to handle request.
-#[async_trait]
impl<M: Message> EndpointHandler<M> for () {
async fn handle(self: &Arc<()>, _m: &M, _from: NodeID) -> M::Response {
panic!("This endpoint should not have a local handler.");
@@ -38,15 +37,13 @@ impl<M: Message> EndpointHandler<M> for () {
/// This trait should be implemented by an object of your application
/// that can handle a message of type `M`, in the cases where it doesn't
/// care about attached stream in the request nor in the response.
-#[async_trait]
pub trait EndpointHandler<M>: Send + Sync
where
M: Message,
{
- async fn handle(self: &Arc<Self>, m: &M, from: NodeID) -> M::Response;
+ fn handle(self: &Arc<Self>, m: &M, from: NodeID) -> impl Future<Output = M::Response> + Send;
}
-#[async_trait]
impl<T, M> StreamingEndpointHandler<M> for T
where
T: EndpointHandler<M>,
@@ -161,9 +158,8 @@ where
pub(crate) type DynEndpoint = Box<dyn GenericEndpoint + Send + Sync>;
-#[async_trait]
pub(crate) trait GenericEndpoint {
- async fn handle(&self, req_enc: ReqEnc, from: NodeID) -> Result<RespEnc, Error>;
+ fn handle(&self, req_enc: ReqEnc, from: NodeID) -> BoxFuture<Result<RespEnc, Error>>;
fn drop_handler(&self);
fn clone_endpoint(&self) -> DynEndpoint;
}
@@ -174,21 +170,23 @@ where
M: Message,
H: StreamingEndpointHandler<M>;
-#[async_trait]
impl<M, H> GenericEndpoint for EndpointArc<M, H>
where
M: Message,
H: StreamingEndpointHandler<M> + 'static,
{
- async fn handle(&self, req_enc: ReqEnc, from: NodeID) -> Result<RespEnc, Error> {
- match self.0.handler.load_full() {
- None => Err(Error::NoHandler),
- Some(h) => {
- let req = Req::from_enc(req_enc)?;
- let res = h.handle(req, from).await;
- Ok(res.into_enc()?)
+ fn handle(&self, req_enc: ReqEnc, from: NodeID) -> BoxFuture<Result<RespEnc, Error>> {
+ async move {
+ match self.0.handler.load_full() {
+ None => Err(Error::NoHandler),
+ Some(h) => {
+ let req = Req::from_enc(req_enc)?;
+ let res = h.handle(req, from).await;
+ Ok(res.into_enc()?)
+ }
}
}
+ .boxed()
}
fn drop_handler(&self) {
diff --git a/src/net/netapp.rs b/src/net/netapp.rs
index 77e55774..36c6fc88 100644
--- a/src/net/netapp.rs
+++ b/src/net/netapp.rs
@@ -5,7 +5,6 @@ use std::sync::{Arc, RwLock};
use log::{debug, error, info, trace, warn};
use arc_swap::ArcSwapOption;
-use async_trait::async_trait;
use serde::{Deserialize, Serialize};
use sodiumoxide::crypto::auth;
@@ -457,7 +456,6 @@ impl NetApp {
}
}
-#[async_trait]
impl EndpointHandler<HelloMessage> for NetApp {
async fn handle(self: &Arc<Self>, msg: &HelloMessage, from: NodeID) {
debug!("Hello from {:?}: {:?}", hex::encode(&from[..8]), msg);
diff --git a/src/net/peering.rs b/src/net/peering.rs
index a8d271ec..08378a08 100644
--- a/src/net/peering.rs
+++ b/src/net/peering.rs
@@ -5,7 +5,6 @@ use std::sync::{Arc, RwLock};
use std::time::{Duration, Instant};
use arc_swap::ArcSwap;
-use async_trait::async_trait;
use log::{debug, info, trace, warn};
use serde::{Deserialize, Serialize};
@@ -592,7 +591,6 @@ impl PeeringManager {
}
}
-#[async_trait]
impl EndpointHandler<PingMessage> for PeeringManager {
async fn handle(self: &Arc<Self>, ping: &PingMessage, from: NodeID) -> PingMessage {
let ping_resp = PingMessage {
@@ -604,7 +602,6 @@ impl EndpointHandler<PingMessage> for PeeringManager {
}
}
-#[async_trait]
impl EndpointHandler<PeerListMessage> for PeeringManager {
async fn handle(
self: &Arc<Self>,
diff --git a/src/net/recv.rs b/src/net/recv.rs
index 0de7bef2..35a6d71a 100644
--- a/src/net/recv.rs
+++ b/src/net/recv.rs
@@ -1,7 +1,6 @@
use std::collections::HashMap;
use std::sync::Arc;
-use async_trait::async_trait;
use bytes::Bytes;
use log::*;
@@ -50,7 +49,6 @@ impl Drop for Sender {
/// according to the protocol defined above: chunks of message in progress of being
/// received are stored in a buffer, and when the last chunk of a message is received,
/// the full message is passed to the receive handler.
-#[async_trait]
pub(crate) trait RecvLoop: Sync + 'static {
fn recv_handler(self: &Arc<Self>, id: RequestID, stream: ByteStream);
fn cancel_handler(self: &Arc<Self>, _id: RequestID) {}
diff --git a/src/net/send.rs b/src/net/send.rs
index 1454eeb7..6f1ac02c 100644
--- a/src/net/send.rs
+++ b/src/net/send.rs
@@ -3,7 +3,6 @@ use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};
-use async_trait::async_trait;
use bytes::{BufMut, Bytes, BytesMut};
use log::*;
@@ -273,7 +272,6 @@ impl DataFrame {
///
/// The `.send_loop()` exits when the sending end of the channel is closed,
/// or if there is an error at any time writing to the async writer.
-#[async_trait]
pub(crate) trait SendLoop: Sync {
async fn send_loop<W>(
self: Arc<Self>,
diff --git a/src/net/server.rs b/src/net/server.rs
index 36dccb2f..fb6c6366 100644
--- a/src/net/server.rs
+++ b/src/net/server.rs
@@ -3,7 +3,6 @@ use std::net::SocketAddr;
use std::sync::{Arc, Mutex};
use arc_swap::ArcSwapOption;
-use async_trait::async_trait;
use log::*;
use futures::io::{AsyncReadExt, AsyncWriteExt};
@@ -174,7 +173,6 @@ impl ServerConn {
impl SendLoop for ServerConn {}
-#[async_trait]
impl RecvLoop for ServerConn {
fn recv_handler(self: &Arc<Self>, id: RequestID, stream: ByteStream) {
let resp_send = match self.resp_send.load_full() {
diff --git a/src/rpc/system.rs b/src/rpc/system.rs
index 0fa68218..2a52ae5d 100644
--- a/src/rpc/system.rs
+++ b/src/rpc/system.rs
@@ -7,7 +7,6 @@ use std::sync::{Arc, RwLock, RwLockReadGuard};
use std::time::{Duration, Instant};
use arc_swap::ArcSwapOption;
-use async_trait::async_trait;
use futures::join;
use serde::{Deserialize, Serialize};
use sodiumoxide::crypto::sign::ed25519;
@@ -749,7 +748,6 @@ impl System {
}
}
-#[async_trait]
impl EndpointHandler<SystemRpc> for System {
async fn handle(self: &Arc<Self>, msg: &SystemRpc, from: NodeID) -> Result<SystemRpc, Error> {
match msg {
diff --git a/src/table/gc.rs b/src/table/gc.rs
index 9e060390..28ea119d 100644
--- a/src/table/gc.rs
+++ b/src/table/gc.rs
@@ -4,6 +4,7 @@ use std::sync::Arc;
use std::time::Duration;
use async_trait::async_trait;
+
use serde::{Deserialize, Serialize};
use serde_bytes::ByteBuf;
@@ -272,7 +273,6 @@ impl<F: TableSchema, R: TableReplication> TableGc<F, R> {
}
}
-#[async_trait]
impl<F: TableSchema, R: TableReplication> EndpointHandler<GcRpc> for TableGc<F, R> {
async fn handle(self: &Arc<Self>, message: &GcRpc, _from: NodeID) -> Result<GcRpc, Error> {
match message {
diff --git a/src/table/sync.rs b/src/table/sync.rs
index 234ee8ea..2d43b9fc 100644
--- a/src/table/sync.rs
+++ b/src/table/sync.rs
@@ -444,7 +444,6 @@ impl<F: TableSchema, R: TableReplication> TableSyncer<F, R> {
// ======= SYNCHRONIZATION PROCEDURE -- RECEIVER SIDE ======
-#[async_trait]
impl<F: TableSchema, R: TableReplication> EndpointHandler<SyncRpc> for TableSyncer<F, R> {
async fn handle(self: &Arc<Self>, message: &SyncRpc, from: NodeID) -> Result<SyncRpc, Error> {
match message {
diff --git a/src/table/table.rs b/src/table/table.rs
index 255947e7..c96f4731 100644
--- a/src/table/table.rs
+++ b/src/table/table.rs
@@ -2,7 +2,6 @@ use std::borrow::Borrow;
use std::collections::{BTreeMap, BTreeSet, HashMap};
use std::sync::Arc;
-use async_trait::async_trait;
use futures::stream::*;
use serde::{Deserialize, Serialize};
use serde_bytes::ByteBuf;
@@ -500,7 +499,6 @@ impl<F: TableSchema, R: TableReplication> Table<F, R> {
}
}
-#[async_trait]
impl<F: TableSchema, R: TableReplication> EndpointHandler<TableRpc<F>> for Table<F, R> {
async fn handle(
self: &Arc<Self>,