3030import java .util .Set ;
3131
3232import org .apache .kafka .common .header .Header ;
33- import org .apache .kafka .common .record .FileLogInputStream ;
3433import org .apache .kafka .common .record .Record ;
3534import org .apache .kafka .common .record .RecordBatch ;
3635import org .apache .kafka .common .utils .ByteBufferInputStream ;
6160import io .aiven .kafka .tieredstorage .storage .BytesRange ;
6261import io .aiven .kafka .tieredstorage .storage .ObjectKey ;
6362
64- import io .confluent .kafka .schemaregistry .ParsedSchema ;
65- import io .confluent .kafka .schemaregistry .client .rest .exceptions .RestClientException ;
6663import org .apache .avro .Schema ;
6764import org .apache .avro .generic .GenericData ;
6865import org .apache .iceberg .AppendFiles ;
8279import org .slf4j .Logger ;
8380import org .slf4j .LoggerFactory ;
8481
82+ import static io .aiven .kafka .tieredstorage .iceberg .StructureProvider .SchemaAndId ;
83+
8584public class IcebergRemoteStorageManager extends InternalRemoteStorageManager {
8685 private static final Logger LOG = LoggerFactory .getLogger (IcebergRemoteStorageManager .class );
8786
@@ -198,10 +197,10 @@ private LogSegmentProcessingResult processLogSegment(
198197
199198 for (final var batch : segment .log ().batches ()) {
200199 for (final Record record : batch ) {
201- final ParsedRecord parsedRecord = extractRecordData (batch , record , topicName );
200+ final ParsedRecord parsedRecord = extractRecordData (record , topicName );
202201
203- keySchemaId = parsedRecord .keySchemaId ();
204- valueSchemaId = parsedRecord .valueSchemaId ();
202+ keySchemaId = parsedRecord .key (). schemaId ();
203+ valueSchemaId = parsedRecord .value (). schemaId ();
205204
206205 if (writer == null ) {
207206 final TableIdentifier tableIdentifier = TableIdentifier .of (icebergNamespace , topicName );
@@ -226,71 +225,67 @@ private LogSegmentProcessingResult processLogSegment(
226225 }
227226
228227 private ParsedRecord extractRecordData (
229- final FileLogInputStream .FileChannelRecordBatch batch ,
230228 final Record record ,
231229 final String topicName ) throws Exception {
232230
233- final byte [] rawKey = new byte [record .keySize ()];
234- record .key ().get (rawKey );
235- final byte [] rawValue = new byte [record .valueSize ()];
236- record .value ().get (rawValue );
237-
238- final SchemaAndId keySchema = getSchema (rawKey );
239- final SchemaAndId valueSchema = getSchema (rawValue );
240-
241- final Object keyRecord = structureProvider .deserializeKey (topicName , null , rawKey );
242- final Object valueRecord = structureProvider .deserializeValue (topicName , null , rawValue );
231+ final Deserialized deserializedKey = getDeserializedKey (record , topicName );
232+ final Deserialized deserializedValue = getDeserializedValue (record , topicName );
243233
244- final Schema recordSchema = RowSchema .createRowSchema (keySchema .schema , valueSchema .schema );
234+ final Schema recordSchema = RowSchema .createRowSchema (
235+ deserializedKey .schema ,
236+ deserializedValue .schema );
245237
246238 return new ParsedRecord (
247239 record .offset (),
248240 record .timestamp (),
249- keySchema .schemaId ,
250- rawKey ,
251- keyRecord ,
252- valueSchema .schemaId ,
253- rawValue ,
254- valueRecord ,
241+ deserializedKey ,
242+ deserializedValue ,
255243 recordSchema ,
256244 record .headers ()
257245 );
258246 }
259247
248+ private Deserialized getDeserializedKey (final Record record , final String topicName ) throws IOException {
249+ if (record .hasKey ()) {
250+ final byte [] rawKey = new byte [record .keySize ()];
251+ record .key ().get (rawKey );
252+ final Integer schemaId = getSchemaId (rawKey );
253+ final SchemaAndId <Schema > schema = structureProvider .getSchemaById (schemaId );
254+ final Object keyRecord = structureProvider .deserializeKey (topicName , null , rawKey );
255+ return new Deserialized (rawKey , keyRecord , schema .schemaId (), schema .schema ());
256+ } else {
257+ final SchemaAndId <Schema > schema = structureProvider .getSchemaById (null );
258+ return new Deserialized (null , null , null , schema .schema ());
259+ }
260+ }
261+
262+ private Deserialized getDeserializedValue (final Record record , final String topicName ) throws IOException {
263+ if (record .hasValue ()) {
264+ final byte [] rawValue = new byte [record .valueSize ()];
265+ record .value ().get (rawValue );
266+ final Integer schemaId = getSchemaId (rawValue );
267+ final SchemaAndId <Schema > schema = structureProvider .getSchemaById (schemaId );
268+ final Object valueRecord = structureProvider .deserializeValue (topicName , null , rawValue );
269+ return new Deserialized (rawValue , valueRecord , schema .schemaId (), schema .schema ());
270+ } else {
271+ final SchemaAndId <Schema > schema = structureProvider .getSchemaById (null );
272+ return new Deserialized (null , null , null , schema .schema ());
273+ }
274+ }
275+
276+ private record Deserialized (byte [] raw , Object record , Integer schemaId , Schema schema ) {
277+ }
278+
260279 private record ParsedRecord (
261280 long recordOffset ,
262281 long recordTimestamp ,
263- Integer keySchemaId ,
264- byte [] rawKey ,
265- Object keyRecord ,
266- Integer valueSchemaId ,
267- byte [] rawValue ,
268- Object valueRecord ,
282+ Deserialized key ,
283+ Deserialized value ,
269284 Schema recordSchema ,
270285 Header [] headers
271286 ) {
272287 }
273288
274- private SchemaAndId getSchema (final byte [] value ) throws IOException {
275- final int schemaId = getSchemaId (value );
276- ParsedSchema schema = null ;
277- try {
278- schema = structureProvider .getSchemaById (schemaId );
279- } catch (final RestClientException ignore ) {
280- //writing raw value if schema is not found
281- }
282-
283- if (schema == null ) {
284- return new SchemaAndId (Schema .createUnion (Schema .create (Schema .Type .BYTES ),
285- Schema .create (Schema .Type .NULL )), schemaId );
286- } else {
287- return new SchemaAndId (Schema .createUnion ((Schema ) schema .rawSchema (), Schema .create (Schema .Type .NULL )),
288- schemaId );
289- }
290- }
291-
292- private record SchemaAndId (Schema schema , Integer schemaId ) { }
293-
294289 private void writeRecordToIceberg (
295290 final IcebergWriter writer ,
296291 final RemoteLogSegmentMetadata remoteLogSegmentMetadata ,
@@ -316,15 +311,15 @@ private void writeRecordToIceberg(
316311
317312 final GenericData .Record finalRecord = new GenericData .Record (parsedRecord .recordSchema ());
318313 finalRecord .put ("kafka" , kafkaPart );
319- if (parsedRecord .keyRecord () != null ) {
320- finalRecord .put ("key" , parsedRecord .keyRecord ());
314+ if (parsedRecord .key (). record () != null ) {
315+ finalRecord .put ("key" , parsedRecord .key (). record ());
321316 } else {
322- finalRecord .put ("key_raw" , parsedRecord .rawKey );
317+ finalRecord .put ("key_raw" , parsedRecord .key (). raw () );
323318 }
324- if (parsedRecord .keyRecord () != null ) {
325- finalRecord .put ("value" , parsedRecord .valueRecord ());
319+ if (parsedRecord .value (). record () != null ) {
320+ finalRecord .put ("value" , parsedRecord .value (). record ());
326321 } else {
327- finalRecord .put ("value_raw" , parsedRecord .rawValue );
322+ finalRecord .put ("value_raw" , parsedRecord .value (). raw () );
328323 }
329324 finalRecord .put ("headers" , Arrays .asList (parsedRecord .headers ()));
330325
@@ -374,19 +369,12 @@ public InputStream fetchLogSegment(final RemoteLogSegmentMetadata remoteLogSegme
374369 new RecordBatchGrouper (new MultiFileReader (remoteFilePaths , dataFileMetadata -> {
375370 final FileIO io = table .io ();
376371
377- final Schema valueSchema ;
378- final Schema keySchema ;
379- try {
380- keySchema =
381- (Schema ) structureProvider .getSchemaById (dataFileMetadata .keySchemaId ()).rawSchema ();
382- valueSchema =
383- (Schema ) structureProvider .getSchemaById (dataFileMetadata .valueSchemaId ()).rawSchema ();
384-
385- } catch (final RestClientException e ) {
386- throw new RuntimeException (e );
387- }
372+ final SchemaAndId <Schema > keySchema =
373+ structureProvider .getSchemaById (dataFileMetadata .keySchemaId ());
374+ final SchemaAndId <Schema > valueSchema =
375+ structureProvider .getSchemaById (dataFileMetadata .valueSchemaId ());
388376
389- final Schema recordSchema = RowSchema .createRowSchema (keySchema , valueSchema );
377+ final Schema recordSchema = RowSchema .createRowSchema (keySchema . schema () , valueSchema . schema () );
390378
391379 return Parquet .read (io .newInputFile (dataFileMetadata .location ()))
392380 .project (table .schema ())
0 commit comments