Skip to content

Commit d57236a

Browse files
committed
datalake: add additional system fields
Adds two more system fields: - partition: the kafka partition id - headers: a list of key-value pairs represented as iobufs
1 parent 5ded2df commit d57236a

File tree

8 files changed

+104
-32
lines changed

8 files changed

+104
-32
lines changed

src/v/datalake/record_multiplexer.cc

Lines changed: 29 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -50,6 +50,11 @@ record_multiplexer::operator()(model::record_batch batch) {
5050
first_timestamp + record.timestamp_delta()};
5151
kafka::offset offset{batch.base_offset()() + record.offset_delta()};
5252
int64_t estimated_size = key.size_bytes() + val.size_bytes();
53+
chunked_vector<std::pair<std::optional<iobuf>, std::optional<iobuf>>>
54+
header_kvs;
55+
for (auto& hdr : record.headers()) {
56+
header_kvs.emplace_back(hdr.share_key_opt(), hdr.share_value_opt());
57+
}
5358

5459
auto val_type_res = co_await _type_resolver.resolve_buf_type(
5560
std::move(val));
@@ -61,7 +66,11 @@ record_multiplexer::operator()(model::record_batch batch) {
6166
case type_resolver::errc::bad_input:
6267
case type_resolver::errc::translation_error:
6368
auto invalid_res = co_await handle_invalid_record(
64-
offset, record.share_key(), record.share_value(), timestamp);
69+
offset,
70+
record.share_key(),
71+
record.share_value(),
72+
timestamp,
73+
std::move(header_kvs));
6574
if (invalid_res.has_error()) {
6675
_error = invalid_res.error();
6776
co_return ss::stop_iteration::yes;
@@ -71,11 +80,13 @@ record_multiplexer::operator()(model::record_batch batch) {
7180
}
7281

7382
auto record_data_res = co_await record_translator::translate_data(
83+
_ntp.tp.partition,
7484
offset,
7585
std::move(key),
7686
val_type_res.value().type,
7787
std::move(val_type_res.value().parsable_buf),
78-
timestamp);
88+
timestamp,
89+
header_kvs);
7990
if (record_data_res.has_error()) {
8091
switch (record_data_res.error()) {
8192
case record_translator::errc::translation_error:
@@ -85,7 +96,11 @@ record_multiplexer::operator()(model::record_batch batch) {
8596
offset,
8697
record_data_res.error());
8798
auto invalid_res = co_await handle_invalid_record(
88-
offset, record.share_key(), record.share_value(), timestamp);
99+
offset,
100+
record.share_key(),
101+
record.share_value(),
102+
timestamp,
103+
std::move(header_kvs));
89104
if (invalid_res.has_error()) {
90105
_error = invalid_res.error();
91106
co_return ss::stop_iteration::yes;
@@ -107,7 +122,8 @@ record_multiplexer::operator()(model::record_batch batch) {
107122
offset,
108123
record.share_key(),
109124
record.share_value(),
110-
timestamp);
125+
timestamp,
126+
std::move(header_kvs));
111127
if (invalid_res.has_error()) {
112128
_error = invalid_res.error();
113129
co_return ss::stop_iteration::yes;
@@ -189,15 +205,22 @@ record_multiplexer::end_of_stream() {
189205

190206
ss::future<result<std::nullopt_t, writer_error>>
191207
record_multiplexer::handle_invalid_record(
192-
kafka::offset offset, iobuf key, iobuf val, model::timestamp ts) {
208+
kafka::offset offset,
209+
iobuf key,
210+
iobuf val,
211+
model::timestamp ts,
212+
chunked_vector<std::pair<std::optional<iobuf>, std::optional<iobuf>>>
213+
headers) {
193214
vlog(_log.debug, "Handling invalid record {}", offset);
194215
int64_t estimated_size = key.size_bytes() + val.size_bytes();
195216
auto record_data_res = co_await record_translator::translate_data(
217+
_ntp.tp.partition,
196218
offset,
197219
std::move(key),
198220
/*val_type*/ std::nullopt,
199221
std::move(val),
200-
ts);
222+
ts,
223+
headers);
201224
if (record_data_res.has_error()) {
202225
vlog(
203226
_log.error,

src/v/datalake/record_multiplexer.h

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -52,8 +52,12 @@ class record_multiplexer {
5252
// target table.
5353
// TODO: this just writes to the existing table, populating internal
5454
// columns. Consider a separate table entirely.
55-
ss::future<result<std::nullopt_t, writer_error>>
56-
handle_invalid_record(kafka::offset, iobuf, iobuf, model::timestamp);
55+
ss::future<result<std::nullopt_t, writer_error>> handle_invalid_record(
56+
kafka::offset,
57+
iobuf,
58+
iobuf,
59+
model::timestamp,
60+
chunked_vector<std::pair<std::optional<iobuf>, std::optional<iobuf>>>);
5761

5862
prefix_logger _log;
5963
const model::ntp& _ntp;

src/v/datalake/record_translator.cc

Lines changed: 27 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
#include "iceberg/avro_utils.h"
2020
#include "iceberg/datatypes.h"
2121
#include "iceberg/values.h"
22+
#include "model/fundamental.h"
2223

2324
#include <avro/Generic.hh>
2425
#include <avro/GenericDatum.hh>
@@ -78,7 +79,7 @@ record_translator::build_type(std::optional<resolved_type> val_type) {
7879
ret_type.fields[0]->type);
7980
// Use the next id of the system defaults.
8081
system_fields.fields.emplace_back(iceberg::nested_field::create(
81-
6, "data", field->required, std::move(field->type)));
82+
11, "data", field->required, std::move(field->type)));
8283
continue;
8384
}
8485
// Add the extra user-defined fields.
@@ -96,17 +97,41 @@ record_translator::build_type(std::optional<resolved_type> val_type) {
9697

9798
ss::future<checked<iceberg::struct_value, record_translator::errc>>
9899
record_translator::translate_data(
100+
model::partition_id pid,
99101
kafka::offset o,
100102
iobuf key,
101103
const std::optional<resolved_type>& val_type,
102104
iobuf parsable_val,
103-
model::timestamp ts) {
105+
model::timestamp ts,
106+
const chunked_vector<std::pair<std::optional<iobuf>, std::optional<iobuf>>>&
107+
headers) {
104108
auto ret_data = iceberg::struct_value{};
105109
auto system_data = std::make_unique<iceberg::struct_value>();
110+
system_data->fields.emplace_back(iceberg::int_value(pid));
106111
system_data->fields.emplace_back(iceberg::long_value(o));
107112
// NOTE: Kafka uses milliseconds, Iceberg uses microseconds.
108113
system_data->fields.emplace_back(
109114
iceberg::timestamp_value(ts.value() * 1000));
115+
116+
if (headers.empty()) {
117+
system_data->fields.emplace_back(std::nullopt);
118+
} else {
119+
auto headers_list = std::make_unique<iceberg::list_value>();
120+
for (const auto& [k, v] : headers) {
121+
auto header_kv_struct = std::make_unique<iceberg::struct_value>();
122+
header_kv_struct->fields.emplace_back(
123+
k ? std::make_optional<iceberg::value>(
124+
iceberg::binary_value(k->copy()))
125+
: std::nullopt);
126+
header_kv_struct->fields.emplace_back(
127+
v ? std::make_optional<iceberg::value>(
128+
iceberg::binary_value(v->copy()))
129+
: std::nullopt);
130+
headers_list->elements.emplace_back(std::move(header_kv_struct));
131+
}
132+
system_data->fields.emplace_back(std::move(headers_list));
133+
}
134+
110135
system_data->fields.emplace_back(iceberg::binary_value{std::move(key)});
111136
if (val_type.has_value()) {
112137
// Fill in the internal value field.

src/v/datalake/record_translator.h

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -33,11 +33,14 @@ class record_translator {
3333
friend std::ostream& operator<<(std::ostream&, const errc&);
3434
static record_type build_type(std::optional<resolved_type> val_type);
3535
static ss::future<checked<iceberg::struct_value, errc>> translate_data(
36+
model::partition_id pid,
3637
kafka::offset o,
3738
iobuf key,
3839
const std::optional<resolved_type>& val_type,
3940
iobuf parsable_val,
40-
model::timestamp ts);
41+
model::timestamp ts,
42+
const chunked_vector<
43+
std::pair<std::optional<iobuf>, std::optional<iobuf>>>& headers);
4144
};
4245

4346
} // namespace datalake

src/v/datalake/table_definition.cc

Lines changed: 19 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -17,13 +17,27 @@ struct_type schemaless_struct_type() {
1717
using namespace iceberg;
1818
struct_type system_fields;
1919
system_fields.fields.emplace_back(
20-
nested_field::create(2, "offset", field_required::yes, long_type{}));
20+
nested_field::create(2, "partition", field_required::yes, int_type{}));
21+
system_fields.fields.emplace_back(
22+
nested_field::create(3, "offset", field_required::yes, long_type{}));
23+
system_fields.fields.emplace_back(nested_field::create(
24+
4, "timestamp", field_required::yes, timestamp_type{}));
25+
26+
struct_type headers_kv;
27+
headers_kv.fields.emplace_back(
28+
nested_field::create(7, "key", field_required::no, binary_type{}));
29+
headers_kv.fields.emplace_back(
30+
nested_field::create(8, "value", field_required::no, binary_type{}));
2131
system_fields.fields.emplace_back(nested_field::create(
22-
3, "timestamp", field_required::yes, timestamp_type{}));
32+
5,
33+
"headers",
34+
field_required::no,
35+
list_type::create(6, field_required::yes, std::move(headers_kv))));
36+
2337
system_fields.fields.emplace_back(
24-
nested_field::create(4, "key", field_required::no, binary_type{}));
38+
nested_field::create(9, "key", field_required::no, binary_type{}));
2539
system_fields.fields.emplace_back(
26-
nested_field::create(5, "value", field_required::no, binary_type{}));
40+
nested_field::create(10, "value", field_required::no, binary_type{}));
2741
struct_type res;
2842
res.fields.emplace_back(nested_field::create(
2943
1,
@@ -45,7 +59,7 @@ schema default_schema() {
4559
partition_spec hour_partition_spec() {
4660
chunked_vector<partition_field> fields;
4761
fields.emplace_back(partition_field{
48-
.source_id = nested_field::id_t{3},
62+
.source_id = nested_field::id_t{4},
4963
.field_id = partition_field::id_t{1000},
5064
.name = "redpanda_timestamp_hour",
5165
.transform = hour_transform{},

src/v/datalake/tests/gtest_record_multiplexer_test.cc

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -281,7 +281,7 @@ TEST_F(RecordMultiplexerParquetTest, TestSimple) {
281281
// Default columns + a nested struct.
282282
EXPECT_EQ(table->num_columns(), 8);
283283
auto expected_type
284-
= R"(redpanda: struct<offset: int64 not null, timestamp: timestamp[us] not null, key: binary, value: binary> not null
284+
= R"(redpanda: struct<partition: int32 not null, offset: int64 not null, timestamp: timestamp[us] not null, headers: list<element: struct<key: binary, value: binary> not null>, key: binary, value: binary> not null
285285
mylong: int64 not null
286286
nestedrecord: struct<inval1: double not null, inval2: string not null, inval3: int32 not null> not null
287287
myarray: list<element: double not null> not null

src/v/datalake/tests/record_multiplexer_test.cc

Lines changed: 16 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -226,7 +226,7 @@ TEST_P(RecordMultiplexerParamTest, TestSimpleAvroRecords) {
226226

227227
// 4 default columns + RootRecord + mylong
228228
auto schema = get_current_schema();
229-
EXPECT_EQ(schema->highest_field_id(), 6);
229+
EXPECT_EQ(schema->highest_field_id(), 11);
230230
}
231231

232232
TEST_P(RecordMultiplexerParamTest, TestAvroRecordsMultipleSchemas) {
@@ -262,7 +262,7 @@ TEST_P(RecordMultiplexerParamTest, TestAvroRecordsMultipleSchemas) {
262262
}
263263
EXPECT_EQ(hrs.size(), GetParam().hrs);
264264
auto schema = get_current_schema();
265-
EXPECT_EQ(schema->highest_field_id(), 16);
265+
EXPECT_EQ(schema->highest_field_id(), 21);
266266
}
267267

268268
INSTANTIATE_TEST_SUITE_P(
@@ -297,10 +297,12 @@ TEST_F(RecordMultiplexerTest, TestAvroRecordsWithRedpandaField) {
297297

298298
// Add Avro records.
299299
auto start_offset = model::offset{0};
300-
auto res = mux(default_param, start_offset, [&gen](storage::record_batch_builder& b) {
301-
auto res = gen.add_random_avro_record(b, "avro_rp", std::nullopt).get();
302-
ASSERT_FALSE(res.has_error());
303-
});
300+
auto res = mux(
301+
default_param, start_offset, [&gen](storage::record_batch_builder& b) {
302+
auto res
303+
= gen.add_random_avro_record(b, "avro_rp", std::nullopt).get();
304+
ASSERT_FALSE(res.has_error());
305+
});
304306
ASSERT_TRUE(res.has_value());
305307
const auto& write_res = res.value();
306308
EXPECT_EQ(write_res.data_files.size(), default_param.hrs);
@@ -315,11 +317,12 @@ TEST_F(RecordMultiplexerTest, TestAvroRecordsWithRedpandaField) {
315317
// 1 nested redpanda column + 4 default columns + mylong + 1 user redpanda
316318
// column + 1 nested
317319
auto schema = get_current_schema();
318-
EXPECT_EQ(schema->highest_field_id(), 8);
320+
EXPECT_EQ(schema->highest_field_id(), 13);
319321

320322
// The redpanda system fields should include the 'data' column.
321-
const auto& rp_struct = std::get<iceberg::struct_type>(schema->schema_struct.fields[0]->type);
322-
EXPECT_EQ(5, rp_struct.fields.size());
323+
const auto& rp_struct = std::get<iceberg::struct_type>(
324+
schema->schema_struct.fields[0]->type);
325+
EXPECT_EQ(7, rp_struct.fields.size());
323326
EXPECT_EQ("data", rp_struct.fields.back()->name);
324327
}
325328

@@ -338,7 +341,7 @@ TEST_F(RecordMultiplexerTest, TestMissingSchema) {
338341
EXPECT_EQ(write_res.data_files.size(), default_param.hrs);
339342

340343
auto schema = get_current_schema();
341-
EXPECT_EQ(schema->highest_field_id(), 5);
344+
EXPECT_EQ(schema->highest_field_id(), 10);
342345
}
343346

344347
TEST_F(RecordMultiplexerTest, TestBadData) {
@@ -364,7 +367,7 @@ TEST_F(RecordMultiplexerTest, TestBadData) {
364367
// shouldn't register the Avro schema -- instead we should see the default
365368
// schema.
366369
auto schema = get_current_schema();
367-
EXPECT_EQ(schema->highest_field_id(), 5);
370+
EXPECT_EQ(schema->highest_field_id(), 10);
368371
}
369372

370373
TEST_F(RecordMultiplexerTest, TestBadSchemaChange) {
@@ -397,7 +400,7 @@ TEST_F(RecordMultiplexerTest, TestBadSchemaChange) {
397400

398401
// This should have registered the valid schema.
399402
auto schema = get_current_schema();
400-
EXPECT_EQ(schema->highest_field_id(), 6);
403+
EXPECT_EQ(schema->highest_field_id(), 11);
401404

402405
// Now try writing with an incompatible schema.
403406
res = mux(
@@ -413,5 +416,5 @@ TEST_F(RecordMultiplexerTest, TestBadSchemaChange) {
413416
const auto& write_res = res.value();
414417
EXPECT_EQ(write_res.data_files.size(), default_param.hrs);
415418
schema = get_current_schema();
416-
EXPECT_EQ(schema->highest_field_id(), 6);
419+
EXPECT_EQ(schema->highest_field_id(), 11);
417420
}

tests/rptest/tests/datalake/datalake_e2e_test.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -106,7 +106,7 @@ def test_avro_schema(self, storage_type, query_engine, use_serde_parquet):
106106
trino = dl.trino()
107107
trino_expected_out = [(
108108
'redpanda',
109-
'row(offset bigint, timestamp timestamp(6), key varbinary, value varbinary)',
109+
'row(partition integer, offset bigint, timestamp timestamp(6), headers array(row(key varbinary, value varbinary)), key varbinary, value varbinary)',
110110
'', ''), ('val', 'bigint', '', '')]
111111
trino_describe_out = trino.run_query_fetch_all(
112112
f"describe {table_name}")
@@ -116,7 +116,7 @@ def test_avro_schema(self, storage_type, query_engine, use_serde_parquet):
116116
spark = dl.spark()
117117
spark_expected_out = [(
118118
'redpanda',
119-
'struct<offset:bigint,timestamp:timestamp_ntz,key:binary,value:binary>',
119+
'struct<partition:int,offset:bigint,timestamp:timestamp_ntz,headers:array<struct<key:binary,value:binary>>,key:binary,value:binary>',
120120
None), ('val', 'bigint', None), ('', '', ''),
121121
('# Partitioning', '', ''),
122122
('Part 0', 'hours(redpanda.timestamp)',

0 commit comments

Comments
 (0)