BalaMahesh opened a new issue, #5893: URL: https://github.com/apache/hudi/issues/5893
**_Tips before filing an issue_** - Have you gone through our [FAQs](https://hudi.apache.org/learn/faq/)? - Join the mailing list to engage in conversations and get faster support at [email protected]. - If you have triaged this as a bug, then file an [issue](https://issues.apache.org/jira/projects/HUDI/issues) directly. **Describe the problem you faced** I am trying to consume data using PostgresDebeziumSource from the kafka topic which is produced from PostgresCdcSource (debezium) confluent kafka connector. Schema Registry is the schema provider. After records are fetched from kafka topic , PostgresDebeziumSource is transforming the data into new schema which is different from the source schema. ` @Override protected Dataset<Row> processDataset(Dataset<Row> rowDataset) { if (rowDataset.columns().length > 0) { // Pick selective debezium and postgres meta fields: pick the row values from before field for delete record // and row values from after field for insert or update records. Dataset<Row> insertedOrUpdatedData = rowDataset .selectExpr( String.format("%s as %s", DebeziumConstants.INCOMING_OP_FIELD, DebeziumConstants.FLATTENED_OP_COL_NAME), String.format("%s as %s", DebeziumConstants.INCOMING_TS_MS_FIELD, DebeziumConstants.UPSTREAM_PROCESSING_TS_COL_NAME), String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_NAME_FIELD, DebeziumConstants.FLATTENED_SHARD_NAME), String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_TS_MS_FIELD, DebeziumConstants.FLATTENED_TS_COL_NAME), String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_TXID_FIELD, DebeziumConstants.FLATTENED_TX_ID_COL_NAME), String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_LSN_FIELD, DebeziumConstants.FLATTENED_LSN_COL_NAME), String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_XMIN_FIELD, DebeziumConstants.FLATTENED_XMIN_COL_NAME), String.format("%s.*", DebeziumConstants.INCOMING_AFTER_FIELD) ) .filter(rowDataset.col(DebeziumConstants.INCOMING_OP_FIELD).notEqual(DebeziumConstants.DELETE_OP)); Dataset<Row> deletedData = rowDataset .selectExpr( String.format("%s as %s", DebeziumConstants.INCOMING_OP_FIELD, DebeziumConstants.FLATTENED_OP_COL_NAME), String.format("%s as %s", DebeziumConstants.INCOMING_TS_MS_FIELD, DebeziumConstants.UPSTREAM_PROCESSING_TS_COL_NAME), String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_NAME_FIELD, DebeziumConstants.FLATTENED_SHARD_NAME), String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_TS_MS_FIELD, DebeziumConstants.FLATTENED_TS_COL_NAME), String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_TXID_FIELD, DebeziumConstants.FLATTENED_TX_ID_COL_NAME), String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_LSN_FIELD, DebeziumConstants.FLATTENED_LSN_COL_NAME), String.format("%s as %s", DebeziumConstants.INCOMING_SOURCE_XMIN_FIELD, DebeziumConstants.FLATTENED_XMIN_COL_NAME), String.format("%s.*", DebeziumConstants.INCOMING_BEFORE_FIELD) ) .filter(rowDataset.col(DebeziumConstants.INCOMING_OP_FIELD).equalTo(DebeziumConstants.DELETE_OP)); return insertedOrUpdatedData.union(deletedData); } else { return rowDataset; } } ` After the data is transformed, a new schemaProvider is added to InputBatch using the below method in RowSource `@Override protected final InputBatch<Dataset<Row>> fetchNewData(Option<String> lastCkptStr, long sourceLimit) { Pair<Option<Dataset<Row>>, String> res = fetchNextBatch(lastCkptStr, sourceLimit); return res.getKey().map(dsr -> { SchemaProvider rowSchemaProvider = UtilHelpers.createRowBasedSchemaProvider(dsr.schema(), props, sparkContext); return new InputBatch<>(res.getKey(), res.getValue(), rowSchemaProvider); }).orElseGet(() -> new InputBatch<>(res.getKey(), res.getValue())); }` This the right schema of the data which is going to be inserted finally into the table. But again inside Source class, the below method is overiding the schemaProvider to the initial one which has raw schema(before the transformation). Here overriddenSchemaProvider is not null and is set to SchemaRegistryProvider `public final InputBatch<T> fetchNext(Option<String> lastCkptStr, long sourceLimit) { InputBatch<T> batch = fetchNewData(lastCkptStr, sourceLimit); // If overriddenSchemaProvider is passed in CLI, use it return overriddenSchemaProvider == null ? batch : new InputBatch<>(batch.getBatch(), batch.getCheckpointForNextBatch(), overriddenSchemaProvider); }` Now during the write in OverwriteWithLatestAvroPayload , `@Override public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException { if (recordBytes.length == 0) { return Option.empty(); } IndexedRecord indexedRecord = HoodieAvroUtils.bytesToAvro(recordBytes, schema); if (isDeleteRecord((GenericRecord) indexedRecord)) { return Option.empty(); } else { return Option.of(indexedRecord); } }` The above method is failing at HoodieAvroUtils.bytesToAvro(recordBytes, schema) , since the transformed recordBytes and untransformed schema are not matching and failing with exception. `java.lang.ArrayIndexOutOfBoundsException: 47548110 at org.apache.avro.io.parsing.Symbol$Alternative.getSymbol(Symbol.java:460) at org.apache.avro.io.ResolvingDecoder.readIndex(ResolvingDecoder.java:283) at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:187) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:160) at org.apache.avro.generic.GenericDatumReader.readField(GenericDatumReader.java:259) at org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:247) at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:179) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:160) at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:187) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:160) at org.apache.avro.generic.GenericDatumReader.readField(GenericDatumReader.java:259) at org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:247) at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:179) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:160) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153) at org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro(HoodieAvroUtils.java:156) at org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro(HoodieAvroUtils.java:146) at org.apache.hudi.common.model.OverwriteWithLatestAvroPayload.getInsertValue(OverwriteWithLatestAvroPayload.java:75) at org.apache.hudi.common.model.debezium.AbstractDebeziumAvroPayload.getInsertRecord(AbstractDebeziumAvroPayload.java:87) at org.apache.hudi.common.model.debezium.AbstractDebeziumAvroPayload.getInsertValue(AbstractDebeziumAvroPayload.java:58) at org.apache.hudi.common.model.HoodieRecordPayload.getInsertValue(HoodieRecordPayload.java:105) at org.apache.hudi.execution.HoodieLazyInsertIterable$HoodieInsertValueGenResult.<init>(HoodieLazyInsertIterable.java:90) at org.apache.hudi.execution.HoodieLazyInsertIterable.lambda$getTransformFunction$0(HoodieLazyInsertIterable.java:103) at org.apache.hudi.common.util.queue.BoundedInMemoryQueue.insertRecord(BoundedInMemoryQueue.java:190) at org.apache.hudi.common.util.queue.IteratorBasedQueueProducer.produce(IteratorBasedQueueProducer.java:46) at org.apache.hudi.common.util.queue.BoundedInMemoryExecutor.lambda$null$0(BoundedInMemoryExecutor.java:105) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:750) ` After changing `public final InputBatch<T> fetchNext(Option<String> lastCkptStr, long sourceLimit) { InputBatch<T> batch = fetchNewData(lastCkptStr, sourceLimit); // If overriddenSchemaProvider is passed in CLI, use it return overriddenSchemaProvider == null ? batch : new InputBatch<>(batch.getBatch(), batch.getCheckpointForNextBatch(), overriddenSchemaProvider); }` to `public final InputBatch<T> fetchNext(Option<String> lastCkptStr, long sourceLimit) { InputBatch<T> batch = fetchNewData(lastCkptStr, sourceLimit); // If overriddenSchemaProvider is passed in CLI, use it // overriddenSchemaProvider == null ? -- source code. return batch.getSchemaProvider().getSourceSchema() != null ? batch : new InputBatch<>(batch.getBatch(), batch.getCheckpointForNextBatch(), overriddenSchemaProvider); }` writes are working. **To Reproduce** Steps to reproduce the behavior: 1. Produce data into Kafka using postgres debezium connector 2. Run hudi with PostgresDebeziumSource, SchemaRegistryProvider and PostgresDebeziumAvroPayload 3. Writes will fail with this. **Expected behavior** A clear and concise description of what you expected to happen. Work as usual and commit data to tables. **Environment Description** * Hudi version : 0.11.0 * Spark version : 3.2.1 * Hive version : 3.x.x * Hadoop version : 3.x.x * Storage (HDFS/S3/GCS..) : GCS * Running on Docker? (yes/no) : no **Additional context** Add any other context about the problem here. **Stacktrace** ```java.lang.ArrayIndexOutOfBoundsException: 47548110 at org.apache.avro.io.parsing.Symbol$Alternative.getSymbol(Symbol.java:460) at org.apache.avro.io.ResolvingDecoder.readIndex(ResolvingDecoder.java:283) at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:187) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:160) at org.apache.avro.generic.GenericDatumReader.readField(GenericDatumReader.java:259) at org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:247) at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:179) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:160) at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:187) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:160) at org.apache.avro.generic.GenericDatumReader.readField(GenericDatumReader.java:259) at org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:247) at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:179) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:160) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153) at org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro(HoodieAvroUtils.java:156) at org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro(HoodieAvroUtils.java:146) at org.apache.hudi.common.model.OverwriteWithLatestAvroPayload.getInsertValue(OverwriteWithLatestAvroPayload.java:75) at org.apache.hudi.common.model.debezium.AbstractDebeziumAvroPayload.getInsertRecord(AbstractDebeziumAvroPayload.java:87) at org.apache.hudi.common.model.debezium.AbstractDebeziumAvroPayload.getInsertValue(AbstractDebeziumAvroPayload.java:58) at org.apache.hudi.common.model.HoodieRecordPayload.getInsertValue(HoodieRecordPayload.java:105) at org.apache.hudi.execution.HoodieLazyInsertIterable$HoodieInsertValueGenResult.<init>(HoodieLazyInsertIterable.java:90) at org.apache.hudi.execution.HoodieLazyInsertIterable.lambda$getTransformFunction$0(HoodieLazyInsertIterable.java:103) at org.apache.hudi.common.util.queue.BoundedInMemoryQueue.insertRecord(BoundedInMemoryQueue.java:190) at org.apache.hudi.common.util.queue.IteratorBasedQueueProducer.produce(IteratorBasedQueueProducer.java:46) at org.apache.hudi.common.util.queue.BoundedInMemoryExecutor.lambda$null$0(BoundedInMemoryExecutor.java:105) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:750) ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
