Loïc Hoguin 2 лет назад
Родитель
Сommit
c02f95b058
6 измененных файлов с 447 добавлено и 42 удалено
  1. 1 1
      ebin/cowboy.app
  2. 30 0
      src/cow_http3.erl
  3. 58 0
      src/cow_http3_machine.erl
  4. 10 4
      src/cowboy.erl
  5. 339 37
      src/cowboy_http3.erl
  6. 9 0
      src/quic_hello_h.erl

+ 1 - 1
ebin/cowboy.app

@@ -1,7 +1,7 @@
 {application, 'cowboy', [
 	{description, "Small, fast, modern HTTP server."},
 	{vsn, "2.10.0"},
-	{modules, ['cow_http3','cow_http3_machine','cowboy','cowboy_app','cowboy_bstr','cowboy_children','cowboy_clear','cowboy_clock','cowboy_compress_h','cowboy_constraints','cowboy_handler','cowboy_http','cowboy_http2','cowboy_http3','cowboy_loop','cowboy_metrics_h','cowboy_middleware','cowboy_req','cowboy_rest','cowboy_router','cowboy_static','cowboy_stream','cowboy_stream_h','cowboy_sub_protocol','cowboy_sup','cowboy_tls','cowboy_tracer_h','cowboy_websocket']},
+	{modules, ['cow_http3','cow_http3_machine','cowboy','cowboy_app','cowboy_bstr','cowboy_children','cowboy_clear','cowboy_clock','cowboy_compress_h','cowboy_constraints','cowboy_handler','cowboy_http','cowboy_http2','cowboy_http3','cowboy_loop','cowboy_metrics_h','cowboy_middleware','cowboy_req','cowboy_rest','cowboy_router','cowboy_static','cowboy_stream','cowboy_stream_h','cowboy_sub_protocol','cowboy_sup','cowboy_tls','cowboy_tracer_h','cowboy_websocket','quic_hello_h']},
 	{registered, [cowboy_sup,cowboy_clock]},
 	{applications, [kernel,stdlib,crypto,cowlib,ranch,quicer]},
 	{mod, {cowboy_app, []}},

+ 30 - 0
src/cow_http3.erl

@@ -14,9 +14,16 @@
 
 -module(cow_http3).
 
+%% Parsing.
 -export([parse/1]).
 -export([parse_unidi_stream_header/1]).
 
+%% Building.
+-export([data/1]).
+-export([headers/1]).
+
+%% Parsing.
+
 -spec parse(_) -> _. %% @todo
 
 %%
@@ -258,3 +265,26 @@ parse_unidi_stream_header(<<2:2, _:30, Rest/bits>>) ->
 	{undefined, Rest};
 parse_unidi_stream_header(<<3:2, _:62, Rest/bits>>) ->
 	{undefined, Rest}.
+
+%% Building.
+
+-spec data(_) -> todo.
+
+data(Data) ->
+	Len = encode_int(iolist_size(Data)),
+	[<<0:8>>, Len, Data].
+
+-spec headers(_) -> todo.
+
+headers(HeaderBlock) ->
+	Len = encode_int(iolist_size(HeaderBlock)),
+	[<<1:8>>, Len, HeaderBlock].
+
+encode_int(I) when I < 64 ->
+	<<0:2, I:6>>;
+encode_int(I) when I < 16384 ->
+	<<1:2, I:14>>;
+encode_int(I) when I < 1073741824 ->
+	<<2:2, I:30>>;
+encode_int(I) when I < 4611686018427387904 ->
+	<<3:2, I:62>>.

+ 58 - 0
src/cow_http3_machine.erl

@@ -19,6 +19,7 @@
 -export([init_stream/5]).
 -export([set_unidi_remote_stream_type/3]).
 -export([frame/4]).
+-export([prepare_headers/5]).
 
 -record(stream, {
 	ref :: any(), %% @todo specs
@@ -320,6 +321,63 @@ headers_frame(Stream0, State0=#http3_machine{local_decoder_ref=DecoderRef},
 			{ok, {headers, IsFin, Headers, PseudoHeaders, Len}, {DecoderRef, DecData}, State}
 	end.
 
+%% Functions for sending a message header or body. Note that
+%% this module does not send data directly, instead it returns
+%% a value that can then be used to send the frames.
+
+%-spec prepare_headers(cow_http2:streamid(), State, idle | cow_http2:fin(),
+%	pseudo_headers(), cow_http:headers())
+%	-> {ok, cow_http2:fin(), iodata(), State} when State::http2_machine().
+-spec prepare_headers(_, _, _, _, _) -> todo.
+
+prepare_headers(StreamRef, State=#http3_machine{encode_state=EncodeState0},
+		IsFin0, PseudoHeaders, Headers0) ->
+	Stream = #stream{id=StreamID, method=Method, local=idle} = stream_get(StreamRef, State),
+	IsFin = case {IsFin0, Method} of
+		{idle, _} -> nofin;
+		{_, <<"HEAD">>} -> fin;
+		_ -> IsFin0
+	end,
+	Headers = merge_pseudo_headers(PseudoHeaders, remove_http11_headers(Headers0)),
+	{ok, HeaderBlock, EncData, EncodeState} = cow_qpack:encode_field_section(Headers, StreamID, EncodeState0),
+	{ok, IsFin, HeaderBlock, EncData, stream_store(Stream#stream{local=IsFin0},
+		State#http3_machine{encode_state=EncodeState})}.
+
+%% @todo Function copied from cow_http2_machine.
+remove_http11_headers(Headers) ->
+	RemoveHeaders0 = [
+		<<"keep-alive">>,
+		<<"proxy-connection">>,
+		<<"transfer-encoding">>,
+		<<"upgrade">>
+	],
+	RemoveHeaders = case lists:keyfind(<<"connection">>, 1, Headers) of
+		false ->
+			RemoveHeaders0;
+		{_, ConnHd} ->
+			%% We do not need to worry about any "close" header because
+			%% that header name is reserved.
+			Connection = cow_http_hd:parse_connection(ConnHd),
+			Connection ++ [<<"connection">>|RemoveHeaders0]
+	end,
+	lists:filter(fun({Name, _}) ->
+		not lists:member(Name, RemoveHeaders)
+	end, Headers).
+
+%% @todo Function copied from cow_http2_machine.
+merge_pseudo_headers(PseudoHeaders, Headers0) ->
+	lists:foldl(fun
+		({status, Status}, Acc) when is_integer(Status) ->
+			[{<<":status">>, integer_to_binary(Status)}|Acc];
+		({Name, Value}, Acc) ->
+			[{iolist_to_binary([$:, atom_to_binary(Name, latin1)]), Value}|Acc]
+		end, Headers0, maps:to_list(PseudoHeaders)).
+
+%% Stream-related functions.
+
+stream_get(StreamRef, #http3_machine{streams=Streams}) ->
+	maps:get(StreamRef, Streams, undefined).
+
 stream_store(#stream{ref=StreamRef, local=fin, remote=fin},
 		State=#http3_machine{streams=Streams0}) ->
 	Streams = maps:remove(StreamRef, Streams0),

+ 10 - 4
src/cowboy.erl

@@ -16,7 +16,7 @@
 
 -export([start_clear/3]).
 -export([start_tls/3]).
--export([start_quic/1]).
+-export([start_quic/2]).
 -export([start_quic_test/0]).
 -export([stop_listener/1]).
 -export([set_env/3]).
@@ -66,8 +66,8 @@ start_tls(Ref, TransOpts0, ProtoOpts0) ->
 %% @todo Experimental function to start a barebone QUIC listener.
 %%       This will need to be reworked to be closer to Ranch
 %%       listeners and provide equivalent features.
--spec start_quic(_) -> ok.
-start_quic(TransOpts) ->
+-spec start_quic(_, _) -> todo.
+start_quic(TransOpts, ProtoOpts) ->
 	{ok, _} = application:ensure_all_started(quicer),
 	Parent = self(),
 	Port = 4567,
@@ -83,7 +83,7 @@ start_quic(TransOpts) ->
 		{ok, Conn} = quicer:handshake(Conn),
 		Pid = spawn(fun() ->
 			receive go -> ok end,
-			cowboy_http3:init(Parent, Conn)
+			cowboy_http3:init(Parent, Conn, ProtoOpts)
 		end),
 		ok = quicer:controlling_process(Conn, Pid),
 		Pid ! go,
@@ -98,6 +98,12 @@ start_quic_test() ->
 			{cert, "deps/quicer/test/quicer_SUITE_data/cert.pem"},
 			{key, "deps/quicer/test/quicer_SUITE_data/key.pem"}
 		]
+	}, #{
+		env => #{dispatch => cowboy_router:compile([
+			{"localhost", [
+				{"/", quic_hello_h, []}
+			]}
+		])}
 	}).
 
 ensure_connection_type(TransOpts=#{connection_type := ConnectionType}) ->

+ 339 - 37
src/cowboy_http3.erl

@@ -12,14 +12,22 @@
 %% ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
 %% OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
 
+%% A key difference between cowboy_http2 and cowboy_http3
+%% is that HTTP/3 streams are QUIC streams and therefore
+%% much of the connection state is handled outside of
+%% Cowboy. The quicer library uses a reference for
+%% identifying streams, and we use that same reference
+%% for our StreamID internally. The real StreamID can
+%% be retrieved via quicer:get_stream_id(StreamRef).
+
 -module(cowboy_http3).
 
--export([init/2]).
+-export([init/3]).
 
 -include_lib("quicer/include/quicer.hrl").
 
 -record(stream, {
-	ref :: any(), %% @todo specs; is it useful in the record?
+	ref :: any(), %% @todo specs
 
 	%% Whether the stream is currently in a special state.
 	status :: header | normal | data | discard,
@@ -34,38 +42,62 @@
 -record(state, {
 	parent :: pid(),
 	conn :: any(), %% @todo specs
+	opts = #{} :: any(), %% @todo opts(),
+
+	%% Remote address and port for the connection.
+	peer = undefined :: {inet:ip_address(), inet:port_number()},
+
+	%% Local address and port for the connection.
+	sock = undefined :: {inet:ip_address(), inet:port_number()},
 
 	%% HTTP/3 state machine.
 	http3_machine :: cow_http3_machine:http3_machine(),
 
 	%% Bidirectional streams are used for requests and responses.
-	streams = #{} :: map() %% @todo specs
+	streams = #{} :: map(), %% @todo specs
+
+	%% Streams can spawn zero or more children which are then managed
+	%% by this module if operating as a supervisor.
+	children = cowboy_children:init() :: cowboy_children:children()
 }).
 
--spec init(_, _) -> no_return().
-init(Parent, Conn) ->
-	Opts = #{}, %% @todo
+-spec init(_, _, _) -> no_return().
+init(Parent, Conn, Opts) ->
 	{ok, SettingsBin, HTTP3Machine0} = cow_http3_machine:init(server, Opts),
 	{ok, Conn} = quicer:async_accept_stream(Conn, []),
 	%% Immediately open a control, encoder and decoder stream.
 	{ok, ControlRef} = quicer:start_stream(Conn,
 		#{open_flag => ?QUIC_STREAM_OPEN_FLAG_UNIDIRECTIONAL}),
-	quicer:send(ControlRef, [<<0>>, SettingsBin]),
+	{ok, _} = quicer:send(ControlRef, [<<0>>, SettingsBin]),
 	{ok, ControlID} = quicer:get_stream_id(ControlRef),
 	{ok, EncoderRef} = quicer:start_stream(Conn,
 		#{open_flag => ?QUIC_STREAM_OPEN_FLAG_UNIDIRECTIONAL}),
-	quicer:send(EncoderRef, <<2>>),
+	{ok, _} = quicer:send(EncoderRef, <<2>>),
 	{ok, EncoderID} = quicer:get_stream_id(EncoderRef),
 	{ok, DecoderRef} = quicer:start_stream(Conn,
 		#{open_flag => ?QUIC_STREAM_OPEN_FLAG_UNIDIRECTIONAL}),
-	quicer:send(DecoderRef, <<3>>),
+	{ok, _} = quicer:send(DecoderRef, <<3>>),
 	{ok, DecoderID} = quicer:get_stream_id(DecoderRef),
 	%% Set the control, encoder and decoder streams in the machine.
 	HTTP3Machine = cow_http3_machine:init_unidi_local_streams(
 		ControlRef, ControlID, EncoderRef, EncoderID, DecoderRef, DecoderID,
 		HTTP3Machine0),
-	%% Quick! Let's go!
-	loop(#state{parent=Parent, conn=Conn, http3_machine=HTTP3Machine}).
+	%% Get the peername/sockname.
+	Peer0 = quicer:peername(Conn),
+	Sock0 = quicer:sockname(Conn),
+	%% @todo Get the peer certificate here if it makes sense.
+	case {Peer0, Sock0} of
+		{{ok, Peer}, {ok, Sock}} ->
+			%% Quick! Let's go!
+			loop(#state{parent=Parent, conn=Conn, opts=Opts,
+				peer=Peer, sock=Sock, http3_machine=HTTP3Machine});
+		{{error, Reason}, _} ->
+			terminate(undefined, {socket_error, Reason,
+				'A socket error occurred when retrieving the peer name.'});
+		{_, {error, Reason}} ->
+			terminate(undefined, {socket_error, Reason,
+				'A socket error occurred when retrieving the sock name.'})
+	end.
 
 loop(State0=#state{conn=Conn}) ->
 	receive
@@ -106,6 +138,9 @@ loop(State0=#state{conn=Conn}) ->
 		%% QUIC_STREAM_EVENT_SEND_SHUTDOWN_COMPLETE
 		{quic, send_shutdown_complete, _StreamRef, _IsGraceful} ->
 			loop(State0);
+		%% Messages pertaining to a stream.
+		{{Pid, StreamRef}, Msg} when Pid =:= self() ->
+			loop(info(State0, StreamRef, Msg));
 		Msg ->
 			logger:error("msg ~p", [Msg]),
 			loop(State0)
@@ -117,8 +152,11 @@ parse(State=#state{streams=Streams}, Data, StreamRef, Props) ->
 		#stream{buffer= <<>>} ->
 			parse1(State, Data, Stream, Props);
 		#stream{buffer=Buffer} ->
-			parse1(State, <<Buffer/binary, Data/binary>>,
-				Stream#stream{buffer= <<>>}, Props)
+			%% @todo OK we should only keep the StreamRef forward
+			%%       and update the stream in the state here.
+			Stream1 = Stream#stream{buffer= <<>>},
+			parse1(stream_update(State, Stream1),
+				<<Buffer/binary, Data/binary>>, Stream1, Props)
 	end.
 
 %% @todo Swap Data and Stream/StreamRef.
@@ -126,15 +164,15 @@ parse1(State, Data, Stream=#stream{status=header}, Props) ->
 	parse_unidirectional_stream_header(State, Data, Stream, Props);
 %% @todo Continuation clause for data frames.
 %% @todo Clause that discards receiving data for aborted streams.
-parse1(State, Data, Stream, Props) ->
+parse1(State, Data, Stream=#stream{ref=StreamRef}, Props) ->
 	case cow_http3:parse(Data) of
 		{ok, Frame, Rest} ->
-			parse1(frame(State, Stream, Frame, Props), Rest, Stream, Props);
+			parse(frame(State, Stream, Frame, Props), Rest, StreamRef, Props);
 		{more, Frame, _Len} ->
 			%% @todo Change state of stream to expect more data frames.
 			loop(frame(State, Stream, Frame, Props));
 		{ignore, Rest} ->
-			parse1(ignored_frame(State, Stream), Rest, Stream, Props);
+			parse(ignored_frame(State, Stream), Rest, Stream, Props);
 		Error = {connection_error, _, _} ->
 			terminate(State, Error);
 		more ->
@@ -149,7 +187,7 @@ parse_unidirectional_stream_header(State0=#state{http3_machine=HTTP3Machine0},
 				StreamRef, Type, HTTP3Machine0),
 			State = State0#state{http3_machine=HTTP3Machine},
 			Stream = Stream0#stream{status=normal},
-			parse1(stream_update(State, Stream), Rest, Stream, Props);
+			parse(stream_update(State, Stream), Rest, StreamRef, Props);
 		{ok, push, _} ->
 			terminate(State0, {connection_error, h3_stream_creation_error,
 				'Only servers can push. (RFC9114 6.2.2)'});
@@ -170,32 +208,296 @@ frame(State=#state{http3_machine=HTTP3Machine0}, Stream=#stream{ref=StreamRef},
 			State#state{http3_machine=HTTP3Machine};
 		{ok, {headers, IsFin, Headers, PseudoHeaders, BodyLen}, HTTP3Machine} ->
 			headers_frame(State#state{http3_machine=HTTP3Machine},
-				Stream, Headers, PseudoHeaders, BodyLen);
+				Stream, IsFin, Headers, PseudoHeaders, BodyLen);
 		{ok, {headers, IsFin, Headers, PseudoHeaders, BodyLen},
 				{DecoderRef, DecData}, HTTP3Machine} ->
 			%% Send the decoder data.
-			quicer:send(DecoderRef, DecData),
+			{ok, _} = quicer:send(DecoderRef, DecData),
 			headers_frame(State#state{http3_machine=HTTP3Machine},
-				Stream, Headers, PseudoHeaders, BodyLen)
+				Stream, IsFin, Headers, PseudoHeaders, BodyLen)
 	end.
 
-headers_frame(State, Stream=#stream{ref=StreamRef}, Headers, PseudoHeaders, BodyLen) ->
-	logger:error("~p~n~p~n~p~n~p~n~p", [State, Stream, Headers, PseudoHeaders, BodyLen]),
-	{ok, StreamID} = quicer:get_stream_id(StreamRef),
-	{ok, Data, EncData, _} = cow_qpack:encode_field_section([
-		{<<":status">>, <<"200">>},
-		{<<"content-length">>, <<"12">>},
-		{<<"content-type">>, <<"text/plain">>}
-	], StreamID, cow_qpack:init()),
-%	%% Send the encoder data.
-%	quicer:send(EncoderRef, EncData),
-	%% Then the response data.
-	DataLen = iolist_size(Data),
-	quicer:send(StreamRef, [<<1, DataLen>>, Data]),
-	quicer:send(StreamRef, <<0,12,"Hello world!">>, ?QUIC_SEND_FLAG_FIN),
-%	quicer:shutdown_stream(StreamRef),
-	logger:error("sent response ~p~nenc data ~p", [iolist_to_binary([<<1, DataLen>>, Data]), EncData]),
-	State.
+%% @todo CONNECT, TRACE and possibly HTTP/1.1 conversion like HTTP/2 has.
+headers_frame(State, Stream, IsFin, Headers, PseudoHeaders=#{authority := Authority}, BodyLen) ->
+	headers_frame_parse_host(State, Stream, IsFin, Headers, PseudoHeaders, BodyLen, Authority).
+
+headers_frame_parse_host(State=#state{peer=Peer, sock=Sock},
+		Stream=#stream{ref=StreamRef}, IsFin, Headers,
+		#{method := Method, scheme := Scheme, path := PathWithQs},
+		BodyLen, Authority) ->
+	try cow_http_hd:parse_host(Authority) of
+		{Host, Port0} ->
+			Port = ensure_port(Scheme, Port0),
+			try cow_http:parse_fullpath(PathWithQs) of
+				{<<>>, _} ->
+					reset_stream(State, Stream, {stream_error, protocol_error,
+						'The path component must not be empty. (RFC7540 8.1.2.3)'});
+				{Path, Qs} ->
+					Req = #{
+						ref => quic, %% @todo Ref,
+						pid => self(),
+						streamid => StreamRef,
+						peer => Peer,
+						sock => Sock,
+%						cert => Cert, %% @todo
+						method => Method,
+						scheme => Scheme,
+						host => Host,
+						port => Port,
+						path => Path,
+						qs => Qs,
+						version => 'HTTP/3',
+						headers => headers_to_map(Headers, #{}),
+						has_body => IsFin =:= nofin,
+						body_length => BodyLen
+					},
+					%% We add the protocol information for extended CONNECTs. @todo
+%					Req = case PseudoHeaders of
+%						#{protocol := Protocol} -> Req1#{protocol => Protocol};
+%						_ -> Req1
+%					end,
+					headers_frame(State, Stream, Req)
+			catch _:_ ->
+				reset_stream(State, Stream, {stream_error, protocol_error,
+					'The :path pseudo-header is invalid. (RFC7540 8.1.2.3)'})
+			end
+	catch _:_ ->
+		reset_stream(State, Stream, {stream_error, protocol_error,
+			'The :authority pseudo-header is invalid. (RFC7540 8.1.2.3)'})
+	end.
+
+%% @todo Copied from cowboy_http2.
+ensure_port(<<"http">>, undefined) -> 80;
+ensure_port(<<"https">>, undefined) -> 443;
+ensure_port(_, Port) -> Port.
+
+%% @todo Copied from cowboy_http2.
+%% This function is necessary to properly handle duplicate headers
+%% and the special-case cookie header.
+headers_to_map([], Acc) ->
+	Acc;
+headers_to_map([{Name, Value}|Tail], Acc0) ->
+	Acc = case Acc0 of
+		%% The cookie header does not use proper HTTP header lists.
+		#{Name := Value0} when Name =:= <<"cookie">> ->
+			Acc0#{Name => << Value0/binary, "; ", Value/binary >>};
+		#{Name := Value0} ->
+			Acc0#{Name => << Value0/binary, ", ", Value/binary >>};
+		_ ->
+			Acc0#{Name => Value}
+	end,
+	headers_to_map(Tail, Acc).
+
+headers_frame(State=#state{opts=Opts, streams=Streams},
+		Stream=#stream{ref=StreamRef}, Req) ->
+	try cowboy_stream:init(StreamRef, Req, Opts) of
+		{Commands, StreamState} ->
+logger:error("~p", [Commands]),
+logger:error("~p", [StreamState]),
+			commands(State#state{
+				streams=Streams#{StreamRef => Stream#stream{state=StreamState}}},
+				StreamRef, Commands)
+	catch Class:Exception:Stacktrace ->
+		cowboy:log(cowboy_stream:make_error_log(init,
+			[StreamRef, Req, Opts],
+			Class, Exception, Stacktrace), Opts),
+		reset_stream(State, StreamRef, {internal_error, {Class, Exception},
+			'Unhandled exception in cowboy_stream:init/3.'})
+	end.
+
+info(State=#state{opts=Opts, http3_machine=_HTTP3Machine, streams=Streams}, StreamRef, Msg) ->
+logger:error("~p", [Msg]),
+	case Streams of
+		#{StreamRef := Stream=#stream{state=StreamState0}} ->
+			try cowboy_stream:info(StreamRef, Msg, StreamState0) of
+				{Commands, StreamState} ->
+logger:error("~p", [Commands]),
+logger:error("~p ~p", [StreamRef, Streams]),
+					commands(State#state{streams=Streams#{StreamRef => Stream#stream{state=StreamState}}},
+						StreamRef, Commands)
+			catch Class:Exception:Stacktrace ->
+				cowboy:log(cowboy_stream:make_error_log(info,
+					[StreamRef, Msg, StreamState0],
+					Class, Exception, Stacktrace), Opts),
+				reset_stream(State, StreamRef, {internal_error, {Class, Exception},
+					'Unhandled exception in cowboy_stream:info/3.'})
+			end;
+		_ ->
+%			case cow_http2_machine:is_lingering_stream(StreamID, HTTP2Machine) of
+%				true ->
+%					ok;
+%				false ->
+					cowboy:log(warning, "Received message ~p for unknown stream ~p.",
+						[Msg, StreamRef], Opts),
+%			end,
+			State
+	end.
+
+%% Stream handler commands.
+
+commands(State, _, []) ->
+	State;
+%% Error responses are sent only if a response wasn't sent already.
+%commands(State=#state{http3_machine=HTTP3Machine}, StreamRef,
+%		[{error_response, StatusCode, Headers, Body}|Tail]) ->
+%	%% @todo
+%	case cow_http2_machine:get_stream_local_state(StreamRef, HTTP2Machine) of
+%		{ok, idle, _} ->
+%			commands(State, StreamRef, [{response, StatusCode, Headers, Body}|Tail]);
+%		_ ->
+%			commands(State, StreamRef, Tail)
+%	end;
+%% Send an informational response.
+%commands(State0, StreamRef, [{inform, StatusCode, Headers}|Tail]) ->
+%	State = send_headers(State0, StreamRef, idle, StatusCode, Headers),
+%	commands(State, StreamRef, Tail);
+%% Send response headers.
+commands(State0, StreamRef, [{response, StatusCode, Headers, Body}|Tail]) ->
+	State = send_response(State0, StreamRef, StatusCode, Headers, Body),
+	commands(State, StreamRef, Tail);
+%% Send response headers.
+%commands(State0, StreamRef, [{headers, StatusCode, Headers}|Tail]) ->
+%	State = send_headers(State0, StreamRef, nofin, StatusCode, Headers),
+%	commands(State, StreamRef, Tail);
+%%% Send a response body chunk.
+%commands(State0, StreamRef, [{data, IsFin, Data}|Tail]) ->
+%	State = maybe_send_data(State0, StreamRef, IsFin, Data, []),
+%	commands(State, StreamRef, Tail);
+%%% Send trailers.
+%commands(State0, StreamRef, [{trailers, Trailers}|Tail]) ->
+%	State = maybe_send_data(State0, StreamRef, fin, {trailers, maps:to_list(Trailers)}, []),
+%	commands(State, StreamRef, Tail);
+%% Send a push promise.
+%%
+%% @todo Responses sent as a result of a push_promise request
+%% must not send push_promise frames themselves.
+%%
+%% @todo We should not send push_promise frames when we are
+%% in the closing http2_status.
+%commands(State0=#state{socket=Socket, transport=Transport, http3_machine=HTTP3Machine0},
+%		StreamRef, [{push, Method, Scheme, Host, Port, Path, Qs, Headers0}|Tail]) ->
+%	Authority = case {Scheme, Port} of
+%		{<<"http">>, 80} -> Host;
+%		{<<"https">>, 443} -> Host;
+%		_ -> iolist_to_binary([Host, $:, integer_to_binary(Port)])
+%	end,
+%	PathWithQs = iolist_to_binary(case Qs of
+%		<<>> -> Path;
+%		_ -> [Path, $?, Qs]
+%	end),
+%	PseudoHeaders = #{
+%		method => Method,
+%		scheme => Scheme,
+%		authority => Authority,
+%		path => PathWithQs
+%	},
+%	%% We need to make sure the header value is binary before we can
+%	%% create the Req object, as it expects them to be flat.
+%	Headers = maps:to_list(maps:map(fun(_, V) -> iolist_to_binary(V) end, Headers0)),
+%	%% @todo
+%	State = case cow_http2_machine:prepare_push_promise(StreamRef, HTTP3Machine0,
+%			PseudoHeaders, Headers) of
+%		{ok, PromisedStreamRef, HeaderBlock, HTTP3Machine} ->
+%			Transport:send(Socket, cow_http2:push_promise(
+%				StreamRef, PromisedStreamRef, HeaderBlock)),
+%			headers_frame(State0#state{http3_machine=HTTP2Machine},
+%				PromisedStreamRef, fin, Headers, PseudoHeaders, 0);
+%		{error, no_push} ->
+%			State0
+%	end,
+%	commands(State, StreamRef, Tail);
+%%% Read the request body.
+%commands(State0=#state{flow=Flow, streams=Streams}, StreamRef, [{flow, Size}|Tail]) ->
+%	#{StreamRef := Stream=#stream{flow=StreamFlow}} = Streams,
+%	State = update_window(State0#state{flow=Flow + Size,
+%		streams=Streams#{StreamRef => Stream#stream{flow=StreamFlow + Size}}},
+%		StreamRef),
+%	commands(State, StreamRef, Tail);
+%% Supervise a child process.
+commands(State=#state{children=Children}, StreamRef, [{spawn, Pid, Shutdown}|Tail]) ->
+	 commands(State#state{children=cowboy_children:up(Children, Pid, StreamRef, Shutdown)},
+		StreamRef, Tail);
+%% Error handling.
+%commands(State, StreamRef, [Error = {internal_error, _, _}|_Tail]) ->
+%	%% @todo Do we want to run the commands after an internal_error?
+%	%% @todo Do we even allow commands after?
+%	%% @todo Only reset when the stream still exists.
+%	reset_stream(State, StreamRef, Error);
+%% Upgrade to HTTP/2. This is triggered by cowboy_http2 itself.
+%commands(State=#state{socket=Socket, transport=Transport, http2_status=upgrade},
+%		StreamRef, [{switch_protocol, Headers, ?MODULE, _}|Tail]) ->
+%	%% @todo This 101 response needs to be passed through stream handlers.
+%	Transport:send(Socket, cow_http:response(101, 'HTTP/1.1', maps:to_list(Headers))),
+%	commands(State, StreamRef, Tail);
+%% Use a different protocol within the stream (CONNECT :protocol).
+%% @todo Make sure we error out when the feature is disabled.
+%commands(State0, StreamRef, [{switch_protocol, Headers, _Mod, _ModState}|Tail]) ->
+%	State = info(State0, StreamRef, {headers, 200, Headers}),
+%	commands(State, StreamRef, Tail);
+%% Set options dynamically.
+commands(State, StreamRef, [{set_options, _Opts}|Tail]) ->
+	commands(State, StreamRef, Tail);
+commands(State, StreamRef, [stop|_Tail]) ->
+	%% @todo Do we want to run the commands after a stop?
+	%% @todo Do we even allow commands after?
+	stop_stream(State, StreamRef);
+%% Log event.
+commands(State=#state{opts=Opts}, StreamRef, [Log={log, _, _, _}|Tail]) ->
+	cowboy:log(Log, Opts),
+	commands(State, StreamRef, Tail).
+
+send_response(State0=#state{http3_machine=HTTP3Machine0}, StreamRef, StatusCode, Headers, Body) ->
+	Size = case Body of
+		{sendfile, _, Bytes, _} -> Bytes;
+		_ -> iolist_size(Body)
+	end,
+	case Size of
+		0 ->
+			State = send_headers(State0, StreamRef, fin, StatusCode, Headers),
+			maybe_terminate_stream(State, StreamRef, fin);
+		_ ->
+			%% @todo Add a test for HEAD to make sure we don't send the body when
+			%% returning {response...} from a stream handler (or {headers...} then {data...}).
+			{ok, _IsFin, HeaderBlock, _EncData, HTTP3Machine}
+				= cow_http3_machine:prepare_headers(StreamRef, HTTP3Machine0, nofin,
+					#{status => cow_http:status_to_integer(StatusCode)},
+					headers_to_list(Headers)),
+			%% @todo It might be better to do async sends.
+			{ok, _} = quicer:send(StreamRef, [
+				cow_http3:headers(HeaderBlock),
+				cow_http3:data(Body)
+			], send_flag(fin)),
+			State0#state{http3_machine=HTTP3Machine}
+			%% @todo maybe_terminate_stream (see maybe_send_data for how to handle it)
+	end.
+
+send_headers(State=#state{http3_machine=HTTP3Machine0},
+		StreamRef, IsFin0, StatusCode, Headers) ->
+	{ok, IsFin, HeaderBlock, HTTP3Machine}
+		= cow_http3_machine:prepare_headers(StreamRef, HTTP3Machine0, IsFin0,
+			#{status => cow_http:status_to_integer(StatusCode)},
+			headers_to_list(Headers)),
+	{ok, _} = quicer:send(StreamRef, cow_http3:headers(HeaderBlock), send_flag(IsFin)),
+	State#state{http3_machine=HTTP3Machine}.
+
+%% The set-cookie header is special; we can only send one cookie per header.
+headers_to_list(Headers0=#{<<"set-cookie">> := SetCookies}) ->
+	Headers = maps:to_list(maps:remove(<<"set-cookie">>, Headers0)),
+	Headers ++ [{<<"set-cookie">>, Value} || Value <- SetCookies];
+headers_to_list(Headers) ->
+	maps:to_list(Headers).
+
+send_flag(nofin) -> ?QUIC_SEND_FLAG_NONE;
+send_flag(fin) -> ?QUIC_SEND_FLAG_FIN.
+
+reset_stream(_, _, _) ->
+	todo.
+
+stop_stream(_, _) ->
+	todo.
+
+maybe_terminate_stream(_, _, _) ->
+	todo.
 
 %% @todo In ignored_frame we must check for example that the frame
 %%       we received wasn't the first frame in a control stream

+ 9 - 0
src/quic_hello_h.erl

@@ -0,0 +1,9 @@
+%% This module sends a hello world response.
+
+-module(quic_hello_h).
+
+-export([init/2]).
+
+-spec init(_, _) -> _.
+init(Req, Opts) ->
+	{ok, cowboy_req:reply(200, #{}, <<"Hello world!">>, Req), Opts}.