[SUPPORT] Issue with adding column while running deltastreamer with kafka source.
See original GitHub issueDescribe the problem you faced
Schema evolution is not working when using deltastreamer with a kafka source and avro schema registry.
In my usecase, I am trying to ingest cdc dumped to kafka by a debezium connector. The registry is maintained by debezium connector.
On making a simple schema change like alter table accounts add column description text default 'test description';
, The change is reflected in schema registry by debezium. The new column is visible in hive metastore as well.
I am using a transformer to make some modifications in fields ( which are running fine normally ) and overriding target schema to null so as to use DataSet’s schema while writing.
Results in warning
21/02/22 08:27:51 WARN TaskSetManager: Lost task 0.0 in stage 14.0 (TID 471, ip-172-31-4-18.ap-south-1.compute.internal, executor 1): org.apache.hudi.exception.HoodieException: Exception when reading log file
at org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.scan(AbstractHoodieLogRecordScanner.java:250)
at org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.performScan(HoodieMergedLogRecordScanner.java:100)
at org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:93)
at org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:75)
at org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner$Builder.build(HoodieMergedLogRecordScanner.java:230)
at org.apache.hudi.table.action.compact.HoodieSparkMergeOnReadTableCompactor.compact(HoodieSparkMergeOnReadTableCompactor.java:142)
at org.apache.hudi.table.action.compact.HoodieSparkMergeOnReadTableCompactor.lambda$compact$9ec9d4c7$1(HoodieSparkMergeOnReadTableCompactor.java:105)
at org.apache.spark.api.java.JavaPairRDD$.$anonfun$toScalaFunction$1(JavaPairRDD.scala:1041)
at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:484)
at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:490)
at org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221)
at org.apache.spark.storage.memory.MemoryStore.putIteratorAsBytes(MemoryStore.scala:349)
at org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1388)
at org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1298)
at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1362)
at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1186)
at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:360)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:311)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
at org.apache.spark.scheduler.Task.run(Task.scala:127)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:446)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:449)
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:748)
Caused by: org.apache.avro.AvroTypeException: Found hoodie.source.hoodie_source, expecting hoodie.source.hoodie_source, missing required field description
at org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:292)
at org.apache.avro.io.parsing.Parser.advance(Parser.java:88)
at org.apache.avro.io.ResolvingDecoder.readFieldOrder(ResolvingDecoder.java:130)
at org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:215)
at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:175)
at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153)
at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:145)
at org.apache.hudi.common.table.log.block.HoodieAvroDataBlock.deserializeRecords(HoodieAvroDataBlock.java:157)
at org.apache.hudi.common.table.log.block.HoodieDataBlock.createRecordsFromContentBytes(HoodieDataBlock.java:128)
at org.apache.hudi.common.table.log.block.HoodieDataBlock.getRecords(HoodieDataBlock.java:106)
at org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.processDataBlock(AbstractHoodieLogRecordScanner.java:278)
at org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.processQueuedBlocksForInstant(AbstractHoodieLogRecordScanner.java:313)
at org.apache.hudi.common.table.log.AbstractHoodieLogRecordScanner.scan(AbstractHoodieLogRecordScanner.java:156)
... 29 more
And eventually an error
21/02/22 08:28:13 WARN ExceptionMapper: Uncaught exception
java.lang.IllegalArgumentException: Last known instant from client was 20210222075752 but server has the following timeline [[20210222070142__clean__COMPLETED], [20210222070358__clean__COMPLETED], [20210222070407__deltacommit__COMPLETED], [20210222070426__commit__COMPLETED], [20210222070432__deltacommit__COMPLETED], [20210222070457__deltacommit__COMPLETED], [20210222070518__deltacommit__COMPLETED], [20210222070542__deltacommit__COMPLETED], [20210222070559__clean__COMPLETED], [20210222070605__deltacommit__COMPLETED], [20210222070635__commit__COMPLETED], [20210222070644__deltacommit__COMPLETED], [20210222070707__deltacommit__COMPLETED], [20210222070737__deltacommit__COMPLETED], [20210222070800__deltacommit__COMPLETED], [20210222070818__clean__COMPLETED], [20210222070825__deltacommit__COMPLETED], [20210222070843__commit__COMPLETED], [20210222070848__deltacommit__COMPLETED], [20210222070912__deltacommit__COMPLETED], [20210222070945__deltacommit__COMPLETED], [20210222071010__deltacommit__COMPLETED], [20210222071028__clean__COMPLETED], [20210222071036__deltacommit__COMPLETED], [==>20210222071054__compaction__INFLIGHT], [20210222071059__deltacommit__COMPLETED], [20210222071532__rollback__COMPLETED], [20210222071533__deltacommit__COMPLETED], [20210222072221__rollback__COMPLETED], [20210222072222__deltacommit__COMPLETED], [20210222072803__rollback__COMPLETED], [20210222072804__deltacommit__COMPLETED], [20210222072935__rollback__COMPLETED], [20210222072936__deltacommit__COMPLETED], [20210222073001__clean__COMPLETED], [20210222075751__rollback__COMPLETED], [20210222075752__deltacommit__COMPLETED], [20210222082746__rollback__COMPLETED]]
at org.apache.hudi.common.util.ValidationUtils.checkArgument(ValidationUtils.java:40)
at org.apache.hudi.timeline.service.FileSystemViewHandler$ViewHandler.handle(FileSystemViewHandler.java:372)
at io.javalin.security.SecurityUtil.noopAccessManager(SecurityUtil.kt:22)
at io.javalin.Javalin.lambda$addHandler$0(Javalin.java:606)
at io.javalin.core.JavalinServlet$service$2$1.invoke(JavalinServlet.kt:46)
at io.javalin.core.JavalinServlet$service$2$1.invoke(JavalinServlet.kt:17)
at io.javalin.core.JavalinServlet$service$1.invoke(JavalinServlet.kt:143)
at io.javalin.core.JavalinServlet$service$2.invoke(JavalinServlet.kt:41)
at io.javalin.core.JavalinServlet.service(JavalinServlet.kt:107)
at io.javalin.core.util.JettyServerUtil$initialize$httpHandler$1.doHandle(JettyServerUtil.kt:72)
at org.apache.hudi.org.eclipse.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:203)
at org.apache.hudi.org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:480)
at org.apache.hudi.org.eclipse.jetty.server.session.SessionHandler.doScope(SessionHandler.java:1668)
at org.apache.hudi.org.eclipse.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:201)
at org.apache.hudi.org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1247)
at org.apache.hudi.org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:144)
at org.apache.hudi.org.eclipse.jetty.server.handler.HandlerList.handle(HandlerList.java:61)
at org.apache.hudi.org.eclipse.jetty.server.handler.StatisticsHandler.handle(StatisticsHandler.java:174)
at org.apache.hudi.org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:132)
at org.apache.hudi.org.eclipse.jetty.server.Server.handle(Server.java:502)
at org.apache.hudi.org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:370)
at org.apache.hudi.org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:267)
at org.apache.hudi.org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:305)
at org.apache.hudi.org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:103)
at org.apache.hudi.org.eclipse.jetty.io.ChannelEndPoint$2.run(ChannelEndPoint.java:117)
at org.apache.hudi.org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.runTask(EatWhatYouKill.java:333)
at org.apache.hudi.org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.doProduce(EatWhatYouKill.java:310)
at org.apache.hudi.org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.tryProduce(EatWhatYouKill.java:168)
at org.apache.hudi.org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.run(EatWhatYouKill.java:126)
at org.apache.hudi.org.eclipse.jetty.util.thread.ReservedThreadExecutor$ReservedThread.run(ReservedThreadExecutor.java:366)
at org.apache.hudi.org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:765)
at org.apache.hudi.org.eclipse.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:683)
at java.lang.Thread.run(Thread.java:748)
Environment Description
-
Hudi version : 0.7.0
-
Spark version : 3.0.1
-
Hive version : 3.1.2
-
Storage (HDFS/S3/GCS…) : S3
-
Running on Docker? (yes/no) : no
Issue Analytics
- State:
- Created 3 years ago
- Comments:14 (14 by maintainers)
@t0il3ts0ap : yes, I am working on a potential fix for this issue. jira: https://issues.apache.org/jira/browse/HUDI-1687 PR: https://github.com/apache/hudi/pull/2654/
crux of the issue: In deltastreamer w/ a transformer, if schema provider is set, hudi takes it as is. Only if schema provider is not configured, hudi falls back to RowbasedSchemaProvider (which is the behavior you expect).
@t0il3ts0ap I’ve open a new project to deal with ddl by debezium which can identify the DDL SQL and transform the structural changes to the new Kafka message. If you have time, maybe you can try it. In debezium, the change of structure will not trigger the data push. This leads to the fact that if I change the field from a to b, no new data will appear in Kafka unless DML operation is generated.