11use bytes:: Bytes ;
2+ use clickhouse:: { error:: Result , Client , Compression , Row } ;
3+ use clickhouse_types:: { Column , DataTypeNode } ;
24use criterion:: { criterion_group, criterion_main, Criterion , Throughput } ;
3- use http_body_util:: Empty ;
5+ use futures_util:: stream;
6+ use http_body_util:: StreamBody ;
7+ use hyper:: body:: { Body , Frame } ;
48use hyper:: { body:: Incoming , Request , Response } ;
59use serde:: Serialize ;
10+ use std:: convert:: Infallible ;
611use std:: hint:: black_box;
7- use std:: net:: SocketAddr ;
12+ use std:: net:: { Ipv4Addr , SocketAddr , SocketAddrV4 } ;
813use std:: {
914 future:: Future ,
1015 mem,
1116 time:: { Duration , Instant } ,
1217} ;
1318
14- use clickhouse:: { error:: Result , Client , Compression , Row } ;
15-
1619mod common;
1720
18- async fn serve ( request : Request < Incoming > ) -> Response < Empty < Bytes > > {
21+ async fn serve (
22+ request : Request < Incoming > ,
23+ compression : Compression ,
24+ with_validation : bool ,
25+ ) -> Response < impl Body < Data = Bytes , Error = Infallible > > {
1926 common:: skip_incoming ( request) . await ;
20- Response :: new ( Empty :: new ( ) )
27+
28+ let bytes = if with_validation {
29+ let schema = vec ! [
30+ Column :: new( "a" . to_string( ) , DataTypeNode :: UInt64 ) ,
31+ Column :: new( "b" . to_string( ) , DataTypeNode :: Int64 ) ,
32+ Column :: new( "c" . to_string( ) , DataTypeNode :: Int32 ) ,
33+ Column :: new( "d" . to_string( ) , DataTypeNode :: UInt32 ) ,
34+ Column :: new( "e" . to_string( ) , DataTypeNode :: UInt64 ) ,
35+ Column :: new( "f" . to_string( ) , DataTypeNode :: UInt32 ) ,
36+ Column :: new( "g" . to_string( ) , DataTypeNode :: UInt64 ) ,
37+ Column :: new( "h" . to_string( ) , DataTypeNode :: Int64 ) ,
38+ ] ;
39+
40+ let mut buffer = Vec :: new ( ) ;
41+ clickhouse_types:: put_rbwnat_columns_header ( & schema, & mut buffer) . unwrap ( ) ;
42+
43+ match compression {
44+ Compression :: None => Bytes :: from ( buffer) ,
45+ #[ cfg( feature = "lz4" ) ]
46+ Compression :: Lz4 => clickhouse:: _priv:: lz4_compress ( & buffer) . unwrap ( ) ,
47+ _ => unreachable ! ( ) ,
48+ }
49+ } else {
50+ Bytes :: new ( )
51+ } ;
52+
53+ let stream = StreamBody :: new ( stream:: once ( async { Ok ( Frame :: data ( bytes) ) } ) ) ;
54+ Response :: new ( stream)
2155}
2256
2357#[ derive( Row , Serialize ) ]
@@ -47,11 +81,18 @@ impl SomeRow {
4781 }
4882}
4983
50- async fn run_insert ( client : Client , addr : SocketAddr , iters : u64 ) -> Result < Duration > {
51- let _server = common:: start_server ( addr, serve) . await ;
84+ const ADDR : SocketAddr = SocketAddr :: V4 ( SocketAddrV4 :: new ( Ipv4Addr :: LOCALHOST , 6524 ) ) ;
85+
86+ async fn run_insert (
87+ client : Client ,
88+ iters : u64 ,
89+ compression : Compression ,
90+ validation : bool ,
91+ ) -> Result < Duration > {
92+ let _server = common:: start_server ( ADDR , move |req| serve ( req, compression, validation) ) . await ;
5293
5394 let start = Instant :: now ( ) ;
54- let mut insert = client. insert :: < SomeRow > ( "table" ) ?;
95+ let mut insert = client. insert :: < SomeRow > ( "table" ) . await ?;
5596
5697 for _ in 0 ..iters {
5798 insert. write ( & SomeRow :: sample ( ) ) . await ?;
@@ -64,65 +105,68 @@ async fn run_insert(client: Client, addr: SocketAddr, iters: u64) -> Result<Dura
64105#[ cfg( feature = "inserter" ) ]
65106async fn run_inserter < const WITH_PERIOD : bool > (
66107 client : Client ,
67- addr : SocketAddr ,
68108 iters : u64 ,
109+ compression : Compression ,
110+ validation : bool ,
69111) -> Result < Duration > {
70- let _server = common:: start_server ( addr , serve) . await ;
112+ let _server = common:: start_server ( ADDR , move |req| serve ( req , compression , validation ) ) . await ;
71113
72114 let start = Instant :: now ( ) ;
73- let mut inserter = client. inserter :: < SomeRow > ( "table" ) ? . with_max_rows ( iters) ;
115+ let mut inserter = client. inserter :: < SomeRow > ( "table" ) . with_max_rows ( iters) ;
74116
75117 if WITH_PERIOD {
76118 // Just to measure overhead, not to actually use it.
77119 inserter = inserter. with_period ( Some ( Duration :: from_secs ( 1000 ) ) ) ;
78120 }
79121
80122 for _ in 0 ..iters {
81- inserter. write ( & SomeRow :: sample ( ) ) ?;
123+ inserter. write ( & SomeRow :: sample ( ) ) . await ?;
82124 inserter. commit ( ) . await ?;
83125 }
84126
85127 inserter. end ( ) . await ?;
86128 Ok ( start. elapsed ( ) )
87129}
88130
89- fn run < F > ( c : & mut Criterion , name : & str , port : u16 , f : impl Fn ( Client , SocketAddr , u64 ) -> F )
131+ fn run < F > ( c : & mut Criterion , name : & str , f : impl Fn ( Client , u64 , Compression , bool ) -> F )
90132where
91133 F : Future < Output = Result < Duration > > + Send + ' static ,
92134{
93- let addr: SocketAddr = format ! ( "127.0.0.1:{port}" ) . parse ( ) . unwrap ( ) ;
94135 let runner = common:: start_runner ( ) ;
95-
96136 let mut group = c. benchmark_group ( name) ;
97137 group. throughput ( Throughput :: Bytes ( mem:: size_of :: < SomeRow > ( ) as u64 ) ) ;
98- group. bench_function ( "uncompressed" , |b| {
99- b. iter_custom ( |iters| {
100- let client = Client :: default ( )
101- . with_url ( format ! ( "http://{addr}" ) )
102- . with_compression ( Compression :: None ) ;
103- runner. run ( ( f) ( client, addr, iters) )
104- } )
105- } ) ;
106- #[ cfg( feature = "lz4" ) ]
107- group. bench_function ( "lz4" , |b| {
108- b. iter_custom ( |iters| {
109- let client = Client :: default ( )
110- . with_url ( format ! ( "http://{addr}" ) )
111- . with_compression ( Compression :: Lz4 ) ;
112- runner. run ( ( f) ( client, addr, iters) )
113- } )
114- } ) ;
138+ for validation in [ true , false ] {
139+ #[ allow( clippy:: single_element_loop) ]
140+ for compression in [
141+ Compression :: None ,
142+ #[ cfg( feature = "lz4" ) ]
143+ Compression :: Lz4 ,
144+ ] {
145+ group. bench_function (
146+ format ! ( "validation={validation}/compression={compression:?}" ) ,
147+ |b| {
148+ b. iter_custom ( |iters| {
149+ let client = Client :: default ( )
150+ . with_url ( format ! ( "http://{ADDR}" ) )
151+ . with_compression ( compression)
152+ . with_validation ( validation) ;
153+ runner. run ( ( f) ( client, iters, compression, validation) )
154+ } )
155+ } ,
156+ ) ;
157+ }
158+ }
115159 group. finish ( ) ;
116160}
117161
118162fn insert ( c : & mut Criterion ) {
119- run ( c, "insert" , 6543 , run_insert) ;
163+ run ( c, "insert" , run_insert) ;
120164}
121165
122166#[ cfg( feature = "inserter" ) ]
123167fn inserter ( c : & mut Criterion ) {
124- run ( c, "inserter" , 6544 , run_inserter :: < false > ) ;
125- run ( c, "inserter-period" , 6545 , run_inserter :: < true > ) ;
168+ run ( c, "inserter" , run_inserter :: < false > ) ;
169+ run ( c, "inserter-period" , run_inserter :: < true > ) ;
126170}
127171
128172#[ cfg( not( feature = "inserter" ) ) ]
0 commit comments