Skip to content

Commit dbf848a

Browse files
authored
Merge pull request #21468 from BenPope/backport-pr-21323-v24.1.x
[v24.1.x] [CORE-5670] Pandaproxy: Avoid large allocations
2 parents 7937fab + 8b68ac5 commit dbf848a

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

46 files changed

+720
-308
lines changed

src/v/bytes/include/bytes/iobuf.h

+6
Original file line numberDiff line numberDiff line change
@@ -76,6 +76,12 @@ class iobuf {
7676
using byte_iterator = details::io_byte_iterator;
7777
using placeholder = details::io_placeholder;
7878

79+
static iobuf from(std::string_view view) {
80+
iobuf i;
81+
i.append(view.data(), view.size());
82+
return i;
83+
}
84+
7985
// NOLINTNEXTLINE
8086
iobuf() noexcept {
8187
// nothing allocates memory, but boost intrusive list is not marked as

src/v/bytes/include/bytes/streambuf.h

+40
Original file line numberDiff line numberDiff line change
@@ -91,3 +91,43 @@ class iobuf_ostreambuf final : public std::streambuf {
9191
private:
9292
iobuf* _buf;
9393
};
94+
95+
///\brief Wrap a std::istream around an iobuf
96+
///
97+
/// iobuf buf;
98+
/// iobuf_istream is(std::move(buf));
99+
/// std::string out;
100+
/// is.istream() >> out;
101+
class iobuf_istream {
102+
public:
103+
explicit iobuf_istream(iobuf buf)
104+
: _buf(std::move(buf))
105+
, _isb(_buf)
106+
, _sis{&_isb} {}
107+
std::istream& istream() { return _sis; }
108+
109+
private:
110+
iobuf _buf;
111+
iobuf_istreambuf _isb;
112+
std::istream _sis;
113+
};
114+
115+
///\brief Wrap a std::ostream around an iobuf
116+
///
117+
/// iobuf_ostream os;
118+
/// os.ostream() << "Hello World";
119+
/// iobuf buf = std::move(os).buf();
120+
class iobuf_ostream {
121+
public:
122+
iobuf_ostream()
123+
: _buf()
124+
, _osb(_buf)
125+
, _sos{&_osb} {}
126+
std::ostream& ostream() { return _sos; }
127+
iobuf buf() && { return std::move(_buf); }
128+
129+
private:
130+
iobuf _buf;
131+
iobuf_ostreambuf _osb;
132+
std::ostream _sos;
133+
};

src/v/json/chunked_buffer.h

+25-1
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,13 @@
1414

1515
namespace json {
1616

17+
template<
18+
typename OutputStream,
19+
typename SourceEncoding,
20+
typename TargetEncoding,
21+
unsigned writeFlags>
22+
class generic_iobuf_writer;
23+
1724
namespace impl {
1825

1926
/**
@@ -37,15 +44,32 @@ struct generic_chunked_buffer {
3744
//! Get the length of string in Ch in the string buffer.
3845
size_t GetLength() const { return _impl.size_bytes() / sizeof(Ch); }
3946

40-
void Reserve(size_t s) { _impl.reserve(s); }
47+
void Reserve(size_t s) { _impl.reserve_memory(s); }
4148

4249
void Clear() { _impl.clear(); }
4350

4451
/**@}*/
4552

53+
/**
54+
* Append a fragment to this chunked_buffer. This takes ownership of the
55+
* fragment and is a zero-copy operation.
56+
*/
57+
void append(std::unique_ptr<iobuf::fragment> frag) {
58+
_impl.append(std::move(frag));
59+
}
60+
61+
/**
62+
* Return the underlying iobuf, this is destructive and zero-copy.
63+
*/
4664
iobuf as_iobuf() && { return std::move(_impl); }
4765

4866
private:
67+
template<
68+
typename OutputStream,
69+
typename SourceEncoding,
70+
typename TargetEncoding,
71+
unsigned writeFlags>
72+
friend class json::generic_iobuf_writer;
4973
iobuf _impl;
5074
};
5175

src/v/json/iobuf_writer.h

+117
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,117 @@
1+
// Copyright 2024 Redpanda Data, Inc.
2+
//
3+
// Use of this software is governed by the Business Source License
4+
// included in the file licenses/BSL.md
5+
//
6+
// As of the Change Date specified in that file, in accordance with
7+
// the Business Source License, use of this software will be governed
8+
// by the Apache License, Version 2.0
9+
10+
#pragma once
11+
12+
#include "bytes/iobuf.h"
13+
#include "bytes/iobuf_parser.h"
14+
#include "json/chunked_buffer.h"
15+
#include "json/writer.h"
16+
17+
#include <rapidjson/rapidjson.h>
18+
19+
namespace json {
20+
21+
///\brief a json::Writer that can accept an iobuf as a String payload.
22+
template<
23+
typename OutputStream,
24+
typename SourceEncoding = json::UTF8<>,
25+
typename TargetEncoding = json::UTF8<>,
26+
unsigned writeFlags = rapidjson::kWriteDefaultFlags>
27+
class generic_iobuf_writer
28+
: public Writer<OutputStream, SourceEncoding, TargetEncoding, writeFlags> {
29+
using Base
30+
= Writer<OutputStream, SourceEncoding, TargetEncoding, writeFlags>;
31+
32+
public:
33+
explicit generic_iobuf_writer(OutputStream& os)
34+
: Base{os} {}
35+
36+
using Base::String;
37+
bool String(const iobuf& buf) {
38+
constexpr bool buffer_is_chunked
39+
= std::same_as<OutputStream, json::chunked_buffer>;
40+
if constexpr (buffer_is_chunked) {
41+
return write_chunked_string(buf);
42+
} else {
43+
iobuf_const_parser p{buf};
44+
auto str = p.read_string(p.bytes_left());
45+
return this->String(str.data(), str.size(), true);
46+
}
47+
}
48+
49+
private:
50+
bool write_chunked_string(const iobuf& buf) {
51+
const auto last_frag = [this]() {
52+
return std::prev(this->os_->_impl.end());
53+
};
54+
using Ch = Base::Ch;
55+
this->Prefix(rapidjson::kStringType);
56+
const auto beg = buf.begin();
57+
const auto end = buf.end();
58+
const auto last = std::prev(end);
59+
Ch stashed{};
60+
Ch* stash_pos{};
61+
// Base::WriteString is used to JSON encode the string, and requires a
62+
// contiguous range (pointer, len), so we pass it each fragment.
63+
//
64+
// Unfortunately it also encloses the encoded fragment with double
65+
// quotes:
66+
// R"("A string made of ""fragments will need ""fixing")"
67+
//
68+
// This algorithm efficiently removes the extra quotes without
69+
// additional copying:
70+
// For each encoded fragment that is written (except the last one):
71+
// 1. Trim the suffix quote
72+
// 2. Stash the final character, and where it is to be written
73+
// 3. Drop the final character
74+
// For each encoded fragment that is written (except the first one):
75+
// 4. Restore the stashed character over the prefix-quote
76+
for (auto i = beg; i != end; ++i) {
77+
if (!Base::WriteString(i->get(), i->size())) {
78+
return false;
79+
}
80+
if (i != beg) {
81+
// 4. Restore the stashed character over the prefix-quote
82+
*stash_pos = stashed;
83+
}
84+
if (i != last) {
85+
// 1. Trim the suffix quote
86+
this->os_->_impl.trim_back(1);
87+
88+
// 2. Stash the final character, ...
89+
auto last = last_frag();
90+
stashed = *std::prev(last->get_current());
91+
// 3. Drop the final character
92+
this->os_->_impl.trim_back(1);
93+
94+
// Ensure a stable address to restore the stashed character
95+
if (last != last_frag()) {
96+
this->os_->_impl.reserve_memory(1);
97+
}
98+
// 2. ...and where it is to be written.
99+
stash_pos = last_frag()->get_current();
100+
}
101+
}
102+
return this->EndValue(true);
103+
}
104+
};
105+
106+
template<
107+
typename OutputStream,
108+
typename SourceEncoding = json::UTF8<>,
109+
typename TargetEncoding = json::UTF8<>,
110+
unsigned writeFlags = rapidjson::kWriteDefaultFlags>
111+
using iobuf_writer = generic_iobuf_writer<
112+
OutputStream,
113+
SourceEncoding,
114+
TargetEncoding,
115+
writeFlags>;
116+
117+
} // namespace json

src/v/json/json.cc

+10
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@
1010
#include "json/json.h"
1111

1212
#include "json/chunked_buffer.h"
13+
#include "json/chunked_input_stream.h"
1314
#include "json/stringbuffer.h"
1415

1516
namespace json {
@@ -128,6 +129,15 @@ ss::sstring minify(std::string_view json) {
128129
return ss::sstring(out.GetString(), out.GetSize());
129130
}
130131

132+
iobuf minify(iobuf json) {
133+
json::Reader r;
134+
json::chunked_input_stream in(std::move(json));
135+
json::chunked_buffer out;
136+
json::Writer<json::chunked_buffer> w{out};
137+
r.Parse(in, w);
138+
return std::move(out).as_iobuf();
139+
}
140+
131141
ss::sstring prettify(std::string_view json) {
132142
json::Reader r;
133143
json::StringStream in(json.data());

src/v/json/json.h

+2
Original file line numberDiff line numberDiff line change
@@ -11,6 +11,7 @@
1111

1212
#pragma once
1313

14+
#include "bytes/iobuf.h"
1415
#include "json/_include_first.h"
1516
#include "json/prettywriter.h"
1617
#include "json/reader.h"
@@ -132,6 +133,7 @@ void rjson_serialize(
132133
}
133134

134135
ss::sstring minify(std::string_view json);
136+
iobuf minify(iobuf json);
135137

136138
ss::sstring prettify(std::string_view json);
137139

src/v/json/tests/json_serialization_test.cc

+57
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,11 @@
88
// by the Apache License, Version 2.0
99

1010
#include "base/seastarx.h"
11+
#include "bytes/iobuf_parser.h"
12+
#include "json/chunked_buffer.h"
13+
#include "json/chunked_input_stream.h"
1114
#include "json/document.h"
15+
#include "json/iobuf_writer.h"
1216
#include "json/json.h"
1317
#include "json/stringbuffer.h"
1418
#include "json/writer.h"
@@ -134,3 +138,56 @@ SEASTAR_THREAD_TEST_CASE(json_serialization_test) {
134138

135139
BOOST_TEST(res_doc["obj"].IsObject());
136140
}
141+
142+
static constexpr std::string_view input_string{
143+
R"(The quick brown fox jumps over the lazy dog)"};
144+
145+
static constexpr auto make_chunked_str = []() {
146+
constexpr auto half = input_string.size() / 2;
147+
iobuf in;
148+
in.append_fragments(iobuf::from(input_string.substr(0, half)));
149+
in.append_fragments(iobuf::from(input_string.substr(half)));
150+
BOOST_REQUIRE_EQUAL(std::distance(in.begin(), in.end()), 2);
151+
return in;
152+
};
153+
154+
static constexpr auto make_chunked_json = []() {
155+
iobuf in;
156+
in.append_fragments(iobuf::from("\""));
157+
in.append_fragments(make_chunked_str());
158+
in.append_fragments(iobuf::from("\""));
159+
BOOST_REQUIRE_EQUAL(std::distance(in.begin(), in.end()), 4);
160+
return in;
161+
};
162+
163+
SEASTAR_THREAD_TEST_CASE(json_chunked_input_stream_test) {
164+
{
165+
json::chunked_input_stream is{make_chunked_json()};
166+
json::Document doc;
167+
doc.ParseStream(is);
168+
BOOST_REQUIRE(!doc.HasParseError());
169+
170+
BOOST_REQUIRE(doc.IsString());
171+
auto out_str = std::string_view{doc.GetString(), doc.GetStringLength()};
172+
BOOST_REQUIRE_EQUAL(out_str, input_string);
173+
}
174+
}
175+
176+
SEASTAR_THREAD_TEST_CASE(json_iobuf_writer_test) {
177+
constexpr auto to_string = [](const iobuf& buf) {
178+
iobuf_const_parser p{std::move(buf)};
179+
auto b = p.read_bytes(p.bytes_left());
180+
return std::string{b.begin(), b.end()};
181+
};
182+
183+
{
184+
json::chunked_buffer out;
185+
json::iobuf_writer<json::chunked_buffer> os{out};
186+
auto buf = make_chunked_str();
187+
os.String(buf);
188+
auto out_buf = std::move(out).as_iobuf();
189+
auto expected = make_chunked_json();
190+
BOOST_CHECK_EQUAL(out_buf, expected);
191+
BOOST_CHECK_EQUAL(to_string(out_buf), to_string(expected));
192+
}
193+
}

src/v/pandaproxy/json/iobuf.h

+4-8
Original file line numberDiff line numberDiff line change
@@ -12,9 +12,8 @@
1212
#pragma once
1313

1414
#include "bytes/iobuf.h"
15-
#include "bytes/iobuf_parser.h"
15+
#include "json/iobuf_writer.h"
1616
#include "json/reader.h"
17-
#include "json/stream.h"
1817
#include "json/writer.h"
1918
#include "pandaproxy/json/rjson_util.h"
2019
#include "utils/base64.h"
@@ -65,7 +64,7 @@ class rjson_serialize_impl<iobuf> {
6564
: _fmt(fmt) {}
6665

6766
template<typename Buffer>
68-
bool operator()(::json::Writer<Buffer>& w, iobuf buf) {
67+
bool operator()(::json::iobuf_writer<Buffer>& w, iobuf buf) {
6968
switch (_fmt) {
7069
case serialization_format::none:
7170
[[fallthrough]];
@@ -81,7 +80,7 @@ class rjson_serialize_impl<iobuf> {
8180
}
8281

8382
template<typename Buffer>
84-
bool encode_base64(::json::Writer<Buffer>& w, iobuf buf) {
83+
bool encode_base64(::json::iobuf_writer<Buffer>& w, iobuf buf) {
8584
if (buf.empty()) {
8685
return w.Null();
8786
}
@@ -94,11 +93,8 @@ class rjson_serialize_impl<iobuf> {
9493
if (buf.empty()) {
9594
return w.Null();
9695
}
97-
iobuf_parser p{std::move(buf)};
98-
auto str = p.read_string(p.bytes_left());
99-
static_assert(str.padding(), "StringStream requires null termination");
96+
::json::chunked_input_stream ss{std::move(buf)};
10097
::json::Reader reader;
101-
::json::StringStream ss{str.c_str()};
10298
return reader.Parse(ss, w);
10399
};
104100

src/v/pandaproxy/json/requests/fetch.h

+3-2
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,7 @@ class rjson_serialize_impl<model::record> {
3939
, _base_offset(base_offset) {}
4040

4141
template<typename Buffer>
42-
bool operator()(::json::Writer<Buffer>& w, model::record record) {
42+
bool operator()(::json::iobuf_writer<Buffer>& w, model::record record) {
4343
auto offset = _base_offset() + record.offset_delta();
4444

4545
w.StartObject();
@@ -93,7 +93,8 @@ class rjson_serialize_impl<kafka::fetch_response> {
9393
: _fmt(fmt) {}
9494

9595
template<typename Buffer>
96-
bool operator()(::json::Writer<Buffer>& w, kafka::fetch_response&& res) {
96+
bool
97+
operator()(::json::iobuf_writer<Buffer>& w, kafka::fetch_response&& res) {
9798
// Eager check for errors
9899
for (auto& v : res) {
99100
if (v.partition_response->error_code != kafka::error_code::none) {

0 commit comments

Comments
 (0)