You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2020/10/06 17:38:03 UTC

[GitHub] [hudi] bvaradar commented on issue #2149: Help with Reading Kafka topic written using Debezium Connector - Deltastreamer

bvaradar commented on issue #2149:
URL: https://github.com/apache/hudi/issues/2149#issuecomment-704436559


   @ashishmgofficial : You need to plugin a transformer class to only select the columns you need and record-payload to handle deletions. We are currently in the process of  adding the transformer to OSS Hudi but broadly here is how it will look like 
   (thanks to @joshk-kang). 
   
   gist : 
   
   ```
   package org.apache.hudi.utilities.transform;
   
   import org.apache.hudi.common.config.TypedProperties;
   import org.apache.spark.api.java.JavaSparkContext;
   import org.apache.spark.sql.Dataset;
   import org.apache.spark.sql.Row;
   import org.apache.spark.sql.SparkSession;
   
   public class DebeziumTransformer implements Transformer {
   
     @Override
     public Dataset apply(JavaSparkContext jsc, SparkSession sparkSession, Dataset<Row> rowDataset,
         TypedProperties properties) {
   
       Dataset<Row> insertedOrUpdatedData = rowDataset
           .select("op", "ts_ms", "after.*")
           .withColumnRenamed("op", "_op")
           .withColumnRenamed("ts_ms", "_ts_ms")
           .filter(rowDataset.col("op").notEqual("d"));
   
       Dataset<Row> deletedData = rowDataset
           .select("op", "ts_ms", "before.*")
           .withColumnRenamed("op", "_op")
           .withColumnRenamed("ts_ms", "_ts_ms")
           .filter(rowDataset.col("op").equalTo("d"));
   
       Dataset<Row> transformedData = insertedOrUpdatedData.union(deletedData);
   
       return transformedData;
     }
   }
   ```
   ```
   public class DebeziumAvroPayload extends OverwriteWithLatestAvroPayload {
   
     // Field is prefixed with a underscore by transformer to indicate metadata field
     public static final String OP_FIELD = "_op";
     public static final String DELETE_OP = "d";
   
     public DebeziumAvroPayload(GenericRecord record, Comparable orderingVal) {
       super(record, orderingVal);
     }
   
     public DebeziumAvroPayload(Option<GenericRecord> record) {
       this(record.get(), (record1) -> 0); // natural order
     }
   
     @Override
     protected boolean isDeleteRecord(GenericRecord genericRecord) {
       return genericRecord.get(OP_FIELD) != null && genericRecord.get(OP_FIELD).toString().equalsIgnoreCase(
           DELETE_OP);
     }
   }
   ```
   
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org