├── imgs ├── emr_performance.png └── emr_cluster_autoscaling.png ├── src └── main │ ├── protobuf │ ├── Employee.desc │ └── Employee.proto │ ├── java │ └── com │ │ └── aws │ │ └── emr │ │ ├── proto │ │ ├── SparkProtoRegistry.java │ │ ├── SparkProtoUDF.java │ │ ├── kakfa │ │ │ └── consumer │ │ │ │ ├── ProtoConsumer.java │ │ │ │ └── ProtoConsumerSchemaRegistry.java │ │ ├── kafka │ │ │ └── producer │ │ │ │ ├── ProtoProducerSchemaRegistry.java │ │ │ │ └── ProtoProducer.java │ │ ├── SparkNativeIcebergIngestProto.java │ │ └── SparkCustomIcebergIngestProtoHex.java │ │ ├── json │ │ └── kafka │ │ │ ├── Employee.java │ │ │ ├── consumer │ │ │ └── JsonConsumerSchemaRegistry.java │ │ │ └── producer │ │ │ └── JsonProducerSchemaRegistry.java │ │ ├── avro │ │ └── kafka │ │ │ ├── consumer │ │ │ ├── AvroConsumer.java │ │ │ └── AvroConsumerSchemaRegistry.java │ │ │ ├── producer │ │ │ ├── AvroProducerSchemaRegistry.java │ │ │ └── AvroProducer.java │ │ │ └── SparkNativeIcebergIngestAvro.java │ │ └── spark │ │ ├── cdc │ │ ├── simulator │ │ │ └── KafkaCDCSimulator.java │ │ ├── SparkIncrementalPipeline.java │ │ ├── SparkCDCMirror.java │ │ └── SparkLogChange.java │ │ └── iot │ │ ├── SparkIcebergUtils.java │ │ ├── SparkCustomIcebergIngest.java │ │ ├── SparkCustomIcebergIngestMoR.java │ │ └── SparkCustomIcebergIngestMoRS3BucketsAutoAvro.java │ ├── avro │ └── Employee.avsc │ └── resources │ └── log4j2.properties ├── CODE_OF_CONDUCT.md ├── docker-compose.yml ├── .gitignore ├── LICENSE ├── CONTRIBUTING.md ├── pom.xml └── README.md /imgs/emr_performance.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/aws-samples/iceberg-streaming-examples/HEAD/imgs/emr_performance.png -------------------------------------------------------------------------------- /imgs/emr_cluster_autoscaling.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/aws-samples/iceberg-streaming-examples/HEAD/imgs/emr_cluster_autoscaling.png -------------------------------------------------------------------------------- /src/main/protobuf/Employee.desc: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/aws-samples/iceberg-streaming-examples/HEAD/src/main/protobuf/Employee.desc -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/proto/SparkProtoRegistry.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.proto; 2 | 3 | public class SparkProtoRegistry { 4 | // ideally we would like to have this integrated with the Glue schema registry.... to be done 5 | 6 | 7 | } 8 | -------------------------------------------------------------------------------- /CODE_OF_CONDUCT.md: -------------------------------------------------------------------------------- 1 | ## Code of Conduct 2 | This project has adopted the [Amazon Open Source Code of Conduct](https://aws.github.io/code-of-conduct). 3 | For more information see the [Code of Conduct FAQ](https://aws.github.io/code-of-conduct-faq) or contact 4 | opensource-codeofconduct@amazon.com with any additional questions or comments. 5 | -------------------------------------------------------------------------------- /src/main/avro/Employee.avsc: -------------------------------------------------------------------------------- 1 | {"namespace": "gsr.avro.post", 2 | "type": "record", 3 | "name": "Employee", 4 | "fields": [ 5 | {"name": "employee_id", "type": "long"}, 6 | {"name": "age", "type": "int"}, 7 | {"name": "start_date", "type": "long"}, 8 | {"name": "team", "type": "string"}, 9 | {"name": "role", "type": "string"}, 10 | {"name": "address", "type": "string"}, 11 | {"name": "name", "type": "string"}] 12 | } 13 | 14 | 15 | -------------------------------------------------------------------------------- /src/main/protobuf/Employee.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | package gsr.proto.post; 3 | 4 | import "google/protobuf/wrappers.proto"; 5 | import "google/protobuf/timestamp.proto"; 6 | 7 | message Employee { 8 | int32 id = 1; 9 | string name = 2; 10 | string address = 3; 11 | google.protobuf.Int32Value employee_age = 4; 12 | google.protobuf.Timestamp start_date = 5; 13 | Team team = 6; 14 | Role role = 7; 15 | 16 | } 17 | message Team { 18 | string name = 1; 19 | string location = 2; 20 | } 21 | enum Role { 22 | MANAGER = 0; 23 | DEVELOPER = 1; 24 | ARCHITECT = 2; 25 | } -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/json/kafka/Employee.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.json.kafka; 2 | 3 | import com.fasterxml.jackson.annotation.JsonProperty; 4 | 5 | public class Employee { 6 | @JsonProperty 7 | private int employeeId; 8 | 9 | @JsonProperty 10 | private String name; 11 | 12 | public Employee() {} 13 | 14 | public Employee(int employeeId, String name) { 15 | this.setEmployeeId(employeeId); 16 | this.setName(name); 17 | } 18 | 19 | public int getEmployeeId() { 20 | return employeeId; 21 | } 22 | 23 | public void setEmployeeId(int employeeId) { 24 | this.employeeId = employeeId; 25 | } 26 | 27 | public String getName() { 28 | return name; 29 | } 30 | 31 | public void setName(String name) { 32 | this.name = name; 33 | } 34 | } 35 | 36 | -------------------------------------------------------------------------------- /docker-compose.yml: -------------------------------------------------------------------------------- 1 | version: "3" 2 | services: 3 | kafka: 4 | image: 'bitnami/kafka:latest' 5 | ports: 6 | - '9092:9092' 7 | environment: 8 | - KAFKA_CFG_NODE_ID=0 9 | - KAFKA_CFG_PROCESS_ROLES=controller,broker 10 | - KAFKA_CFG_LISTENERS=PLAINTEXT://:9094,CONTROLLER://:9093,EXTERNAL://:9092 11 | - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://kafka:9094,EXTERNAL://localhost:9092 12 | - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT,EXTERNAL:PLAINTEXT 13 | - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=0@kafka:9093 14 | - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER 15 | - KAFKA_CFG_NUM_PARTITIONS=8 # remember that the number of partitions will be mapped to consumers in Spark to read in parallel 16 | - KAFKA_CFG_AUTO_CREATE_TOPICS_ENABLE=true 17 | - KAFKA_CFG_LOG_RETENTION_MS=1200000 18 | - KAFKA_CFG_LOG_RETENTION_CHECK_INTERVAL_MS=1800000 19 | 20 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # Compiled class files 2 | *.class 3 | 4 | # Log files 5 | *.log 6 | # Package Files 7 | *.jar 8 | *.war 9 | *.nar 10 | *.ear 11 | *.zip 12 | *.tar.gz 13 | *.rar 14 | 15 | # Virtual machine crash logs 16 | hs_err_pid* 17 | replay_pid* 18 | 19 | # Maven 20 | target/ 21 | pom.xml.tag 22 | pom.xml.releaseBackup 23 | pom.xml.versionsBackup 24 | release.properties 25 | .flattened-pom.xml 26 | dependency-reduced-pom.xml 27 | buildNumber.properties 28 | .mvn/timing.properties 29 | 30 | # Eclipse 31 | .metadata 32 | .classpath 33 | .project 34 | .settings/ 35 | bin/ 36 | 37 | # IntelliJ 38 | *.iml 39 | .idea/ 40 | *.ipr 41 | *.iws 42 | 43 | # NetBeans 44 | nb-configuration.xml 45 | 46 | # Visual Studio Code 47 | .vscode/ 48 | 49 | # OSX 50 | .DS_Store 51 | 52 | # Compiled Output 53 | dist/ 54 | tmp/ 55 | out-tsc/ 56 | bazel-out/ 57 | spark-warehouse/ 58 | warehouse/ 59 | # Node 60 | node_modules/ 61 | npm-debug.log 62 | yarn-error.log 63 | yarn.lock 64 | 65 | # Angular 66 | .angular/ 67 | 68 | # Sass 69 | .sass-cache/ 70 | *.css.map 71 | *.sass.map 72 | *.scss.map -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2024 Amazon Web Services 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/proto/SparkProtoUDF.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.proto; 2 | 3 | import gsr.proto.post.EmployeeOuterClass; 4 | import org.apache.spark.sql.*; 5 | import org.apache.spark.sql.api.java.UDF1; 6 | import org.apache.spark.sql.streaming.StreamingQuery; 7 | import org.apache.spark.sql.streaming.StreamingQueryException; 8 | import org.apache.spark.sql.types.DataTypes; 9 | 10 | import static org.apache.spark.sql.functions.col; 11 | 12 | import java.io.IOException; 13 | import java.util.concurrent.TimeoutException; 14 | 15 | /** 16 | * 17 | * An example of consuming messages from Kafka using Protocol Buffers and writing them to Iceberg using a Spark UDF 18 | * to make the "translation" 19 | * 20 | * @author acmanjon@amazon.com 21 | */ 22 | 23 | public class SparkProtoUDF { 24 | 25 | /** 26 | * The entry point of application. 27 | * 28 | * @param args the input arguments 29 | * @throws IOException the io exception 30 | * @throws TimeoutException the timeout exception 31 | * @throws StreamingQueryException the streaming query exception 32 | */ 33 | public static void main(String[] args) 34 | throws IOException, TimeoutException, StreamingQueryException { 35 | SparkSession spark = 36 | SparkSession.builder() 37 | .master("local[*]") 38 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 39 | .getOrCreate(); 40 | 41 | spark 42 | .udf() 43 | .register( 44 | "proto", 45 | new UDF1() { 46 | @Override 47 | public String call(byte[] messageValue) throws Exception { 48 | // String[] strArr = messageValue.split(","); 49 | EmployeeOuterClass.Employee emp = 50 | EmployeeOuterClass.Employee.parseFrom(messageValue); 51 | return emp.getName(); 52 | } 53 | }, 54 | DataTypes.StringType); 55 | 56 | Dataset df = 57 | spark 58 | .readStream() 59 | .format("kafka") 60 | .option("kafka.bootstrap.servers", "localhost:9092") 61 | .option("subscribe", "protobuf-demo-topic-pure") 62 | // .option("startingOffsets","latest") 63 | .load(); 64 | 65 | Dataset output = df.select(col("value")); 66 | 67 | output.createOrReplaceTempView("employee"); 68 | 69 | Dataset result = spark.sql("SELECT proto(value) as name FROM employee"); 70 | 71 | StreamingQuery query = result.writeStream().format("console").outputMode("append").start(); 72 | 73 | query.awaitTermination(); 74 | // wait for user input 75 | System.in.read(); 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/main/resources/log4j2.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | # Set everything to be logged to the console 19 | # this file comes from Spark log4j2 template ;) 20 | rootLogger.level = info 21 | rootLogger.appenderRef.stdout.ref = console 22 | 23 | appender.console.type = Console 24 | appender.console.name = console 25 | appender.console.target = SYSTEM_ERR 26 | 27 | 28 | logger.thriftserver.name = org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver 29 | logger.thriftserver.level = warn 30 | 31 | # Settings to quiet third party logs that are too verbose 32 | logger.jetty1.name = org.sparkproject.jetty 33 | logger.jetty1.level = warn 34 | logger.jetty2.name = org.sparkproject.jetty.util.component.AbstractLifeCycle 35 | logger.jetty2.level = error 36 | logger.compress.name = org.apache.hadoop.io.compress.CodecPool 37 | logger.compress.level = error 38 | logger.replexprTyper.name = org.apache.spark.repl.SparkIMain$exprTyper 39 | logger.replexprTyper.level = info 40 | logger.replSparkILoopInterpreter.name = org.apache.spark.repl.SparkILoop$SparkILoopInterpreter 41 | logger.replSparkILoopInterpreter.level = info 42 | logger.parquet1.name = org.apache.parquet 43 | logger.parquet1.level = error 44 | logger.parquet2.name = parquet 45 | logger.parquet2.level = error 46 | 47 | # SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support 48 | logger.RetryingHMSHandler.name = org.apache.hadoop.hive.metastore.RetryingHMSHandler 49 | logger.RetryingHMSHandler.level = fatal 50 | logger.FunctionRegistry.name = org.apache.hadoop.hive.ql.exec.FunctionRegistry 51 | logger.FunctionRegistry.level = error 52 | 53 | # Kafka 54 | logger.kafka-org.name = org.apache.kafka 55 | logger.kafka-org.level = warn 56 | logger.kafka.name = kafka 57 | logger.kafka.level = warn 58 | logger.kafka-state.name = state.change.logger 59 | logger.kafka-state.level = warn 60 | logger.zookeeper.name = org.apache.zookeeper 61 | logger.zookeeper.level = warn 62 | logger.curator.name = org.apache.curator 63 | logger.curator.level = warn 64 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/json/kafka/consumer/JsonConsumerSchemaRegistry.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.json.kafka.consumer; 2 | 3 | import com.amazonaws.services.schemaregistry.deserializers.GlueSchemaRegistryKafkaDeserializer; 4 | import com.amazonaws.services.schemaregistry.serializers.json.JsonDataWithSchema; 5 | import com.amazonaws.services.schemaregistry.utils.AWSSchemaRegistryConstants; 6 | import com.aws.emr.json.kafka.Employee; 7 | import java.time.Duration; 8 | import java.util.Collections; 9 | import java.util.Properties; 10 | import org.apache.kafka.clients.consumer.ConsumerConfig; 11 | import org.apache.kafka.clients.consumer.ConsumerRecord; 12 | import org.apache.kafka.clients.consumer.ConsumerRecords; 13 | import org.apache.kafka.clients.consumer.KafkaConsumer; 14 | import org.apache.kafka.clients.producer.ProducerConfig; 15 | import org.apache.kafka.common.serialization.StringDeserializer; 16 | import org.apache.logging.log4j.LogManager; 17 | 18 | public class JsonConsumerSchemaRegistry { 19 | 20 | private static final org.apache.logging.log4j.Logger logger = LogManager.getLogger(JsonConsumerSchemaRegistry.class); 21 | 22 | private String bootstrapServers="localhost:9092"; 23 | 24 | public static void main(String args[]){ 25 | JsonConsumerSchemaRegistry consumer = new JsonConsumerSchemaRegistry(); 26 | consumer.startConsumer(); 27 | } 28 | 29 | private Properties getConsumerConfig() { 30 | Properties props = new Properties(); 31 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapServers); 32 | props.put(ConsumerConfig.GROUP_ID_CONFIG, "json"); 33 | props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,"earliest"); 34 | props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); 35 | props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, GlueSchemaRegistryKafkaDeserializer.class.getName()); 36 | props.put(AWSSchemaRegistryConstants.AWS_REGION,"eu-west-1"); 37 | return props; 38 | } 39 | 40 | public void startConsumer() { 41 | logger.info("starting consumer..."); 42 | String topic = "json-demo-topic"; 43 | try (KafkaConsumer consumer = new KafkaConsumer<>(getConsumerConfig())){ 44 | consumer.subscribe(Collections.singletonList(topic)); 45 | while (true) { 46 | final ConsumerRecords records = consumer.poll(Duration.ofMillis(1000)); 47 | for (final ConsumerRecord record : records) { 48 | JsonDataWithSchema value = record.value(); 49 | logger.warn("Employee: " + value.getPayload()); 50 | } 51 | } 52 | }catch (Exception e) { 53 | e.printStackTrace(); 54 | } 55 | } 56 | 57 | 58 | } 59 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/avro/kafka/consumer/AvroConsumer.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.avro.kafka.consumer; 2 | 3 | import com.google.protobuf.InvalidProtocolBufferException; 4 | import gsr.avro.post.Employee; 5 | import gsr.proto.post.EmployeeOuterClass; 6 | import java.time.Duration; 7 | import java.util.Collections; 8 | import java.util.Properties; 9 | import org.apache.kafka.clients.consumer.ConsumerConfig; 10 | import org.apache.kafka.clients.consumer.ConsumerRecord; 11 | import org.apache.kafka.clients.consumer.ConsumerRecords; 12 | import org.apache.kafka.clients.consumer.KafkaConsumer; 13 | import org.apache.kafka.clients.producer.ProducerConfig; 14 | import org.apache.kafka.common.serialization.ByteArrayDeserializer; 15 | import org.apache.kafka.common.serialization.StringDeserializer; 16 | import org.apache.logging.log4j.LogManager; 17 | 18 | public class AvroConsumer { 19 | private static final org.apache.logging.log4j.Logger logger = LogManager.getLogger(AvroConsumer.class); 20 | 21 | 22 | private String bootstrapServers="localhost:9092"; 23 | 24 | public static void main(String args[]) throws InvalidProtocolBufferException { 25 | AvroConsumer consumer = new AvroConsumer(); 26 | consumer.startConsumer(); 27 | } 28 | 29 | private Properties getConsumerConfig() { 30 | Properties props = new Properties(); 31 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapServers); 32 | props.put(ConsumerConfig.GROUP_ID_CONFIG, "avro-pure"); 33 | props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,"earliest"); 34 | props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); 35 | props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); 36 | return props; 37 | } 38 | 39 | public void startConsumer() { 40 | logger.info("starting consumer..."); 41 | String topic = "avro-demo-topic-pure"; 42 | try(KafkaConsumer consumer = new KafkaConsumer<>(getConsumerConfig())){ 43 | consumer.subscribe(Collections.singletonList(topic)); 44 | 45 | while (true) { 46 | final ConsumerRecords records = consumer.poll(Duration.ofMillis(1000)); 47 | for (final ConsumerRecord record : records) { 48 | final Employee employee = Employee.getDecoder().decode(record.value()); 49 | logger.warn("Employee Id: " + employee.getEmployeeId() + " | Name: " + employee.getName() + " | Address: " + employee.getAddress() + 50 | " | Age: " + employee.getAge() + " | Startdate: " + employee.getStartDate() + 51 | " | Team: " + employee.getTeam() + " | Role: " + employee.getRole()); 52 | } 53 | }}catch (Exception e) { 54 | e.printStackTrace(); 55 | } 56 | } 57 | 58 | 59 | } 60 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/avro/kafka/consumer/AvroConsumerSchemaRegistry.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.avro.kafka.consumer; 2 | 3 | import com.amazonaws.services.schemaregistry.deserializers.GlueSchemaRegistryKafkaDeserializer; 4 | import com.amazonaws.services.schemaregistry.utils.AWSSchemaRegistryConstants; 5 | import gsr.avro.post.Employee; 6 | import java.time.Duration; 7 | import java.util.Collections; 8 | import java.util.Properties; 9 | import org.apache.kafka.clients.consumer.ConsumerConfig; 10 | import org.apache.kafka.clients.consumer.ConsumerRecord; 11 | import org.apache.kafka.clients.consumer.ConsumerRecords; 12 | import org.apache.kafka.clients.consumer.KafkaConsumer; 13 | import org.apache.kafka.clients.producer.ProducerConfig; 14 | import org.apache.kafka.common.serialization.StringDeserializer; 15 | import org.apache.logging.log4j.LogManager; 16 | 17 | 18 | /** 19 | * A Kafka consumer implemented in Java using the Glue Schema Registry consuming Avro 20 | * 21 | * @author acmanjon@amazon.com 22 | */ 23 | 24 | public class AvroConsumerSchemaRegistry { 25 | 26 | private static final org.apache.logging.log4j.Logger logger = LogManager.getLogger(AvroConsumerSchemaRegistry.class); 27 | 28 | private String bootstrapServers="localhost:9092"; 29 | 30 | public static void main(String args[]){ 31 | AvroConsumerSchemaRegistry consumer = new AvroConsumerSchemaRegistry(); 32 | consumer.startConsumer(); 33 | } 34 | 35 | private Properties getConsumerConfig() { 36 | Properties props = new Properties(); 37 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapServers); 38 | props.put(ConsumerConfig.GROUP_ID_CONFIG, "avro"); 39 | props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,"earliest"); 40 | props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); 41 | props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, GlueSchemaRegistryKafkaDeserializer.class.getName()); 42 | props.put(AWSSchemaRegistryConstants.AWS_REGION,"us-east-1"); 43 | return props; 44 | } 45 | 46 | public void startConsumer() { 47 | logger.info("starting consumer..."); 48 | String topic = "avro-demo-topic"; 49 | try (KafkaConsumer consumer = new KafkaConsumer<>(getConsumerConfig())){ 50 | consumer.subscribe(Collections.singletonList(topic)); 51 | while (true) { 52 | final ConsumerRecords records = consumer.poll(Duration.ofMillis(1000)); 53 | for (final ConsumerRecord record : records) { 54 | final Employee employee = record.value(); 55 | logger.warn("Employee Id: " + employee.getEmployeeId() + " | Name: " + employee.getName() + " | Address: " + employee.getAddress() + 56 | " | Age: " + employee.getAge() + " | Startdate: " + employee.getStartDate()); 57 | } 58 | } 59 | }catch (Exception e) { 60 | e.printStackTrace(); 61 | } 62 | } 63 | 64 | 65 | } 66 | -------------------------------------------------------------------------------- /CONTRIBUTING.md: -------------------------------------------------------------------------------- 1 | # Contributing Guidelines 2 | 3 | Thank you for your interest in contributing to our project. Whether it's a bug report, new feature, correction, or additional 4 | documentation, we greatly value feedback and contributions from our community. 5 | 6 | Please read through this document before submitting any issues or pull requests to ensure we have all the necessary 7 | information to effectively respond to your bug report or contribution. 8 | 9 | 10 | ## Reporting Bugs/Feature Requests 11 | 12 | We welcome you to use the GitHub issue tracker to report bugs or suggest features. 13 | 14 | When filing an issue, please check existing open, or recently closed, issues to make sure somebody else hasn't already 15 | reported the issue. Please try to include as much information as you can. Details like these are incredibly useful: 16 | 17 | * A reproducible test case or series of steps 18 | * The version of our code being used 19 | * Any modifications you've made relevant to the bug 20 | * Anything unusual about your environment or deployment 21 | 22 | 23 | ## Contributing via Pull Requests 24 | Contributions via pull requests are much appreciated. Before sending us a pull request, please ensure that: 25 | 26 | 1. You are working against the latest source on the *main* branch. 27 | 2. You check existing open, and recently merged, pull requests to make sure someone else hasn't addressed the problem already. 28 | 3. You open an issue to discuss any significant work - we would hate for your time to be wasted. 29 | 30 | To send us a pull request, please: 31 | 32 | 1. Fork the repository. 33 | 2. Modify the source; please focus on the specific change you are contributing. If you also reformat all the code, it will be hard for us to focus on your change. 34 | 3. Ensure local tests pass. 35 | 4. Commit to your fork using clear commit messages. 36 | 5. Send us a pull request, answering any default questions in the pull request interface. 37 | 6. Pay attention to any automated CI failures reported in the pull request, and stay involved in the conversation. 38 | 39 | GitHub provides additional document on [forking a repository](https://help.github.com/articles/fork-a-repo/) and 40 | [creating a pull request](https://help.github.com/articles/creating-a-pull-request/). 41 | 42 | 43 | ## Finding contributions to work on 44 | Looking at the existing issues is a great way to find something to contribute on. As our projects, by default, use the default GitHub issue labels (enhancement/bug/duplicate/help wanted/invalid/question/wontfix), looking at any 'help wanted' issues is a great place to start. 45 | 46 | 47 | ## Code of Conduct 48 | This project has adopted the [Amazon Open Source Code of Conduct](https://aws.github.io/code-of-conduct). 49 | For more information see the [Code of Conduct FAQ](https://aws.github.io/code-of-conduct-faq) or contact 50 | opensource-codeofconduct@amazon.com with any additional questions or comments. 51 | 52 | 53 | ## Security issue notifications 54 | If you discover a potential security issue in this project we ask that you notify AWS/Amazon Security via our [vulnerability reporting page](http://aws.amazon.com/security/vulnerability-reporting/). Please do **not** create a public github issue. 55 | 56 | 57 | ## Licensing 58 | 59 | See the [LICENSE](LICENSE) file for our project's licensing. We will ask you to confirm the licensing of your contribution. -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/proto/kakfa/consumer/ProtoConsumer.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.proto.kakfa.consumer; 2 | 3 | import com.google.protobuf.InvalidProtocolBufferException; 4 | import gsr.proto.post.EmployeeOuterClass; 5 | import java.time.Duration; 6 | import java.util.Collections; 7 | import java.util.Properties; 8 | import org.apache.kafka.clients.consumer.ConsumerConfig; 9 | import org.apache.kafka.clients.consumer.ConsumerRecord; 10 | import org.apache.kafka.clients.consumer.ConsumerRecords; 11 | import org.apache.kafka.clients.consumer.KafkaConsumer; 12 | import org.apache.kafka.clients.producer.ProducerConfig; 13 | import org.apache.kafka.common.serialization.ByteArrayDeserializer; 14 | import org.apache.kafka.common.serialization.StringDeserializer; 15 | import org.apache.logging.log4j.LogManager; 16 | 17 | /** 18 | * A Kafka consumer implemented in pure Java consuming Protocol Buffers 19 | * 20 | * @author acmanjon@amazon.com 21 | * 22 | */ 23 | 24 | public class ProtoConsumer { 25 | 26 | private static final org.apache.logging.log4j.Logger log = 27 | LogManager.getLogger(ProtoConsumer.class); 28 | 29 | private static String bootstrapServers = "localhost:9092"; 30 | 31 | /** 32 | * 33 | * The entry point of application. 34 | * 35 | * @param args the kafkaBootstrapString -- optional defaults to localhost:9092 36 | * @throws InvalidProtocolBufferException the invalid protocol buffer exception 37 | */ 38 | 39 | public static void main(String[] args) throws InvalidProtocolBufferException { 40 | if(args.length == 1) { 41 | bootstrapServers=args[0]; 42 | } 43 | ProtoConsumer consumer = new ProtoConsumer(); 44 | consumer.startConsumer(); 45 | } 46 | 47 | private Properties getConsumerConfig() { 48 | Properties props = new Properties(); 49 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapServers); 50 | props.put(ConsumerConfig.GROUP_ID_CONFIG, "protobuf-pure"); 51 | props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); 52 | props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); 53 | props.put( 54 | ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); 55 | return props; 56 | } 57 | 58 | /** Start kafka consumer. */ 59 | public void startConsumer() { 60 | log.info("starting consumer..."); 61 | String topic = "protobuf-demo-topic-pure"; 62 | try (KafkaConsumer consumer = new KafkaConsumer<>(getConsumerConfig())) { 63 | consumer.subscribe(Collections.singletonList(topic)); 64 | 65 | while (true) { 66 | final ConsumerRecords records = consumer.poll(Duration.ofMillis(1000)); 67 | for (final ConsumerRecord record : records) { 68 | final EmployeeOuterClass.Employee employee = 69 | EmployeeOuterClass.Employee.parseFrom(record.value()); 70 | 71 | log.warn( 72 | "Employee Id: " 73 | + employee.getId() 74 | + " | Name: " 75 | + employee.getName() 76 | + " | Address: " 77 | + employee.getAddress() 78 | + " | Age: " 79 | + employee.getEmployeeAge().getValue() 80 | + " | Startdate: " 81 | + employee.getStartDate().getSeconds() 82 | + " | Team: " 83 | + employee.getTeam().getName() 84 | + " | Role: " 85 | + employee.getRole().name()); 86 | } 87 | } 88 | } catch (Exception e) { 89 | e.printStackTrace(); 90 | } 91 | } 92 | } 93 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/proto/kakfa/consumer/ProtoConsumerSchemaRegistry.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.proto.kakfa.consumer; 2 | 3 | import com.amazonaws.services.schemaregistry.deserializers.GlueSchemaRegistryKafkaDeserializer; 4 | import com.amazonaws.services.schemaregistry.utils.AWSSchemaRegistryConstants; 5 | import com.amazonaws.services.schemaregistry.utils.ProtobufMessageType; 6 | 7 | import java.time.Duration; 8 | import java.util.Collections; 9 | import java.util.Properties; 10 | 11 | import gsr.proto.post.EmployeeOuterClass; 12 | import org.apache.kafka.clients.consumer.ConsumerConfig; 13 | import org.apache.kafka.clients.consumer.ConsumerRecord; 14 | import org.apache.kafka.clients.consumer.ConsumerRecords; 15 | import org.apache.kafka.clients.consumer.KafkaConsumer; 16 | import org.apache.kafka.clients.producer.ProducerConfig; 17 | import org.apache.kafka.common.serialization.StringDeserializer; 18 | import org.apache.logging.log4j.LogManager; 19 | 20 | /** 21 | * 22 | * 23 | * A Kafka consumer implemented in Java using the Glue Schema Registry consuming Protocol Buffers 24 | * 25 | * @author acmanjon@amazon.com 26 | */ 27 | 28 | public class ProtoConsumerSchemaRegistry { 29 | 30 | private static final org.apache.logging.log4j.Logger log = LogManager.getLogger(ProtoConsumerSchemaRegistry.class); 31 | 32 | private static String bootstrapServers="localhost:9092"; 33 | 34 | /** 35 | * 36 | * The entry point of application. 37 | * 38 | * @param args the kafkaBootstrapString -- optional defaults to localhost:9092 39 | */ 40 | 41 | public static void main(String args[]){ 42 | if(args.length == 1) { 43 | bootstrapServers=args[0]; 44 | } 45 | ProtoConsumerSchemaRegistry consumer = new ProtoConsumerSchemaRegistry(); 46 | consumer.startConsumer(); 47 | } 48 | 49 | private Properties getConsumerConfig() { 50 | Properties props = new Properties(); 51 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapServers); 52 | props.put(ConsumerConfig.GROUP_ID_CONFIG, "protobuf"); 53 | props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,"latest"); 54 | props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); 55 | props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, GlueSchemaRegistryKafkaDeserializer.class.getName()); 56 | props.put(AWSSchemaRegistryConstants.AWS_REGION,"eu-west-1"); 57 | props.put(AWSSchemaRegistryConstants.PROTOBUF_MESSAGE_TYPE, ProtobufMessageType.POJO.getName()); 58 | return props; 59 | } 60 | 61 | /** Start consumer. */ 62 | public void startConsumer() { 63 | log.info("starting consumer..."); 64 | String topic = "protobuf-demo-topic"; 65 | try (KafkaConsumer consumer = new KafkaConsumer<>(getConsumerConfig())){ 66 | consumer.subscribe(Collections.singletonList(topic)); 67 | while (true) { 68 | final ConsumerRecords records = consumer.poll(Duration.ofMillis(1000)); 69 | for (final ConsumerRecord record : records) { 70 | final EmployeeOuterClass.Employee employee = record.value(); 71 | log.warn("Employee Id: " + employee.getId() + " | Name: " + employee.getName() + " | Address: " + employee.getAddress() + 72 | " | Age: " + employee.getEmployeeAge().getValue() + " | Startdate: " + employee.getStartDate().getSeconds()); 73 | } 74 | } 75 | }catch (Exception e) { 76 | e.printStackTrace(); 77 | } 78 | } 79 | 80 | 81 | } 82 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/json/kafka/producer/JsonProducerSchemaRegistry.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.json.kafka.producer; 2 | 3 | import com.amazonaws.services.schemaregistry.serializers.GlueSchemaRegistryKafkaSerializer; 4 | import com.amazonaws.services.schemaregistry.utils.AWSSchemaRegistryConstants; 5 | import com.aws.emr.json.kafka.Employee; 6 | import java.util.Properties; 7 | import org.apache.kafka.clients.producer.Callback; 8 | import org.apache.kafka.clients.producer.KafkaProducer; 9 | import org.apache.kafka.clients.producer.ProducerConfig; 10 | import org.apache.kafka.clients.producer.ProducerRecord; 11 | import org.apache.kafka.clients.producer.RecordMetadata; 12 | import org.apache.kafka.common.serialization.StringSerializer; 13 | import org.apache.logging.log4j.LogManager; 14 | import software.amazon.awssdk.services.glue.model.DataFormat; 15 | 16 | /** 17 | * 18 | * A Kafka Java Producer implemented in Java producing Json messages using Glue Schema Registry 19 | * It uses a SplittableRandom as it is a lot faster than the default implementation, and we are not using it for 20 | * cryptographic functions 21 | * 22 | * @author acmanjon@amazon.com 23 | */ 24 | 25 | public class JsonProducerSchemaRegistry { 26 | 27 | private static final org.apache.logging.log4j.Logger logger = LogManager.getLogger(JsonProducerSchemaRegistry.class); 28 | 29 | private String bootstrapServers="localhost:9092"; 30 | 31 | private Properties getProducerConfig() { 32 | Properties props = new Properties(); 33 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapServers); 34 | props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); 35 | props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, GlueSchemaRegistryKafkaSerializer.class.getName()); 36 | props.put(AWSSchemaRegistryConstants.DATA_FORMAT, DataFormat.JSON.name()); 37 | props.put(AWSSchemaRegistryConstants.AWS_REGION, "eu-west-1"); 38 | props.put(AWSSchemaRegistryConstants.REGISTRY_NAME, "employee-schema-registry"); 39 | props.put(AWSSchemaRegistryConstants.SCHEMA_NAME, "Employee.json"); 40 | return props; 41 | } 42 | 43 | public Employee createEmployeeRecord(int employeeId) { 44 | Employee employee 45 | = new Employee(); 46 | employee.setEmployeeId(employeeId); 47 | employee.setName("Dummy"); 48 | return employee; 49 | } 50 | 51 | public void startProducer() throws InterruptedException { 52 | String topic = "json-demo-topic"; 53 | KafkaProducer producer = new KafkaProducer<>(getProducerConfig()); 54 | logger.warn("Starting to send records..."); 55 | int employeeId = 0; 56 | while (employeeId < 1000) { 57 | Employee person = createEmployeeRecord(employeeId); 58 | String key = "key-" + employeeId; 59 | ProducerRecord record = new ProducerRecord<>(topic, key, person); 60 | producer.send(record, new ProducerCallback()); 61 | employeeId++; 62 | } 63 | } 64 | 65 | private class ProducerCallback implements Callback { 66 | 67 | @Override 68 | public void onCompletion(RecordMetadata recordMetaData, Exception e) { 69 | if (e == null) { 70 | logger.warn("Received new metadata. \n" 71 | + "Topic:" + recordMetaData.topic() + "\n" 72 | + "Partition: " + recordMetaData.partition() + "\n" 73 | + "Offset: " + recordMetaData.offset() + "\n" 74 | + "Timestamp: " + recordMetaData.timestamp()); 75 | } else { 76 | logger.warn("There's been an error from the Producer side"); 77 | e.printStackTrace(); 78 | } 79 | } 80 | } 81 | 82 | public static void main(String args[]) throws InterruptedException { 83 | JsonProducerSchemaRegistry producer = new JsonProducerSchemaRegistry(); 84 | producer.startProducer(); 85 | } 86 | 87 | } 88 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/spark/cdc/simulator/KafkaCDCSimulator.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.spark.cdc.simulator; 2 | 3 | import java.io.IOException; 4 | import java.time.Instant; 5 | import java.util.Properties; 6 | import java.util.SplittableRandom; 7 | import java.util.concurrent.TimeUnit; 8 | 9 | import gsr.avro.post.Employee; 10 | import org.apache.kafka.clients.producer.Callback; 11 | import org.apache.kafka.clients.producer.KafkaProducer; 12 | import org.apache.kafka.clients.producer.ProducerConfig; 13 | import org.apache.kafka.clients.producer.ProducerRecord; 14 | import org.apache.kafka.clients.producer.RecordMetadata; 15 | import org.apache.kafka.common.serialization.ByteArraySerializer; 16 | import org.apache.kafka.common.serialization.StringSerializer; 17 | import org.apache.logging.log4j.LogManager; 18 | 19 | /** 20 | * 21 | * A Kafka Java Producer implemented in Java producing DMS messages. 22 | * It uses a SplittableRandom as it is a lot faster than the default implementation, and we are not using it for 23 | * cryptographic functions 24 | * 25 | * @author acmanjon @amazon.com 26 | */ 27 | 28 | public class KafkaCDCSimulator{ 29 | 30 | private static final org.apache.logging.log4j.Logger log = LogManager.getLogger(KafkaCDCSimulator.class); 31 | 32 | private static final SplittableRandom sr = new SplittableRandom(); 33 | /** 34 | * The constant bootstrapServers. 35 | */ 36 | protected static String bootstrapServers="localhost:9092"; // by default localhost 37 | 38 | /** 39 | * Main entry point. 40 | * 41 | * @param args the kafkaBootstrapString -- optional defaults to localhost:9092 42 | * @throws InterruptedException the interrupted exception 43 | */ 44 | public static void main(String args[]) throws InterruptedException { 45 | if(args.length == 1) { 46 | bootstrapServers=args[0]; 47 | } 48 | log.warn("Kafka bootstrap servers are set to "+bootstrapServers); 49 | KafkaCDCSimulator producer = new KafkaCDCSimulator(); 50 | producer.startProducer(); 51 | } 52 | 53 | private Properties getProducerConfig() { 54 | Properties props = new Properties(); 55 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapServers); 56 | props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); 57 | props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); 58 | return props; 59 | } 60 | 61 | /** 62 | * Create employee record employee outer class . employee. 63 | * 64 | * @return the employee outer class . employee 65 | */ 66 | 67 | public String createCDCRecord() { 68 | 69 | // simulate 70 | int id=sr.nextInt(10000); 71 | int balance=sr.nextInt(1000,10000); 72 | Instant instant = Instant.now(); 73 | return "U,"+Integer.toString(id)+","+Integer.toString(balance)+","+Long.toString(instant.toEpochMilli()); 74 | } 75 | 76 | /** 77 | * Start producer. 78 | * 79 | * @throws InterruptedException the interrupted exception 80 | */ 81 | public void startProducer() throws InterruptedException { 82 | String topic = "streaming-cdc-log-ingest"; 83 | 84 | try (KafkaProducer producer = new KafkaProducer<>(getProducerConfig())){ 85 | log.warn("Starting to send records..."); 86 | int count = 1; 87 | int throttle = 0; 88 | while (true) { 89 | if (count % 100000000 == 0) { 90 | log.warn("100 million messages produced... "); 91 | } 92 | String cdc = createCDCRecord(); 93 | var array= cdc.getBytes(); 94 | ProducerRecord record = new ProducerRecord<>(topic, array); 95 | producer.send(record, new KafkaCDCSimulator.ProducerCallback()); 96 | count++; 97 | throttle++; 98 | // if you want to really push just un-comment this block 99 | 100 | if (throttle % 70000 == 0) { 101 | TimeUnit.MILLISECONDS.sleep(400); //about 20.000 msg/seg 102 | } 103 | } 104 | } 105 | } 106 | 107 | private class ProducerCallback implements Callback { 108 | 109 | @Override 110 | public void onCompletion(RecordMetadata recordMetaData, Exception e) { 111 | if (e == null) { 112 | log.debug("Received new metadata. \n" 113 | + "Topic:" + recordMetaData.topic() + "\n" 114 | + "Partition: " + recordMetaData.partition() + "\n" 115 | + "Offset: " + recordMetaData.offset() + "\n" 116 | + "Timestamp: " + recordMetaData.timestamp()); 117 | } else { 118 | log.warn("There's been an error from the Producer side"); 119 | e.printStackTrace(); 120 | } 121 | } 122 | } 123 | 124 | } 125 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/avro/kafka/producer/AvroProducerSchemaRegistry.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.avro.kafka.producer; 2 | 3 | import com.amazonaws.services.schemaregistry.serializers.GlueSchemaRegistryKafkaSerializer; 4 | import com.amazonaws.services.schemaregistry.utils.AWSSchemaRegistryConstants; 5 | import com.amazonaws.services.schemaregistry.utils.ProtobufMessageType; 6 | import gsr.avro.post.Employee; 7 | 8 | import java.time.Instant; 9 | import java.util.Properties; 10 | import java.util.SplittableRandom; 11 | import org.apache.kafka.clients.producer.Callback; 12 | import org.apache.kafka.clients.producer.KafkaProducer; 13 | import org.apache.kafka.clients.producer.ProducerConfig; 14 | import org.apache.kafka.clients.producer.ProducerRecord; 15 | import org.apache.kafka.clients.producer.RecordMetadata; 16 | import org.apache.kafka.common.serialization.StringSerializer; 17 | import org.apache.logging.log4j.LogManager; 18 | import software.amazon.awssdk.services.glue.model.DataFormat; 19 | 20 | /** 21 | * 22 | * A Kafka Java Producer implemented in Java producing Avro messages using Glue Schema Registry 23 | * It uses a SplittableRandom as it is a lot faster than the default implementation, and we are not using it for 24 | * cryptographic functions 25 | * 26 | * @author acmanjon @amazon.com 27 | */ 28 | 29 | public class AvroProducerSchemaRegistry { 30 | 31 | 32 | private static final org.apache.logging.log4j.Logger logger = LogManager.getLogger(AvroProducerSchemaRegistry.class); 33 | 34 | private static String bootstrapServers = "localhost:9092"; // by default localhost 35 | 36 | private static final SplittableRandom sr = new SplittableRandom(); 37 | 38 | private Properties getProducerConfig() { 39 | Properties props = new Properties(); 40 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapServers); 41 | props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); 42 | props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, GlueSchemaRegistryKafkaSerializer.class.getName()); 43 | props.put(AWSSchemaRegistryConstants.DATA_FORMAT, DataFormat.AVRO.name()); 44 | props.put(AWSSchemaRegistryConstants.AWS_REGION, "eu-west-1"); 45 | props.put(AWSSchemaRegistryConstants.REGISTRY_NAME, "employee-schema-registry"); 46 | props.put(AWSSchemaRegistryConstants.SCHEMA_NAME, "Employee.avsc"); 47 | props.put(AWSSchemaRegistryConstants.PROTOBUF_MESSAGE_TYPE, ProtobufMessageType.POJO.getName()); 48 | return props; 49 | } 50 | 51 | /** 52 | * Create employee record employee outer class . employee. 53 | * 54 | * @param employeeId the employee id 55 | * @return the employee outer class . employee 56 | */ 57 | public Employee createEmployeeRecord(int employeeId) { 58 | Instant instant = Instant.now(); 59 | Employee emp=Employee.newBuilder() 60 | .setEmployeeId(employeeId) 61 | .setName("Dummy"+sr.nextInt(100)) 62 | .setAddress("Melbourne, Australia") 63 | .setAge(sr.nextInt(99)) 64 | .setStartDate(instant.toEpochMilli()) 65 | .setRole("ARCHITECT") 66 | .setTeam("Solutions Architects") 67 | .build(); 68 | return emp; 69 | } 70 | 71 | /** 72 | * Start producer. 73 | * 74 | * @throws InterruptedException the interrupted exception 75 | */ 76 | public void startProducer() throws InterruptedException { 77 | String topic = "avro-demo-topic"; 78 | try(KafkaProducer producer = new KafkaProducer<>(getProducerConfig())){ 79 | logger.warn("Starting to send records..."); 80 | int employeeId = 0; 81 | while (employeeId < 1000) { 82 | Employee person = createEmployeeRecord(employeeId); 83 | String key = "key-" + employeeId; 84 | ProducerRecord record = new ProducerRecord<>(topic, key, person); 85 | producer.send(record, new ProducerCallback()); 86 | employeeId++; 87 | } 88 | } 89 | } 90 | 91 | private class ProducerCallback implements Callback { 92 | 93 | @Override 94 | public void onCompletion(RecordMetadata recordMetaData, Exception e) { 95 | if (e == null) { 96 | logger.warn("Received new metadata. \n" 97 | + "Topic:" + recordMetaData.topic() + "\n" 98 | + "Partition: " + recordMetaData.partition() + "\n" 99 | + "Offset: " + recordMetaData.offset() + "\n" 100 | + "Timestamp: " + recordMetaData.timestamp()); 101 | } else { 102 | logger.warn("There's been an error from the Producer side"); 103 | e.printStackTrace(); 104 | } 105 | } 106 | } 107 | 108 | /** 109 | * Main entry point 110 | * 111 | * @param args the kafkaBootstrapString -- optional defaults to localhost:9092 112 | * @throws InterruptedException the interrupted exception 113 | */ 114 | public static void main(String args[]) throws InterruptedException { 115 | if(args.length == 1) { 116 | bootstrapServers=args[0]; 117 | } 118 | AvroProducerSchemaRegistry producer = new AvroProducerSchemaRegistry(); 119 | producer.startProducer(); 120 | } 121 | 122 | } 123 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/proto/kafka/producer/ProtoProducerSchemaRegistry.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.proto.kafka.producer; 2 | 3 | import com.amazonaws.services.schemaregistry.serializers.GlueSchemaRegistryKafkaSerializer; 4 | import com.amazonaws.services.schemaregistry.utils.AWSSchemaRegistryConstants; 5 | import com.amazonaws.services.schemaregistry.utils.ProtobufMessageType; 6 | import java.util.Properties; 7 | import java.util.SplittableRandom; 8 | 9 | import com.google.protobuf.Int32Value; 10 | import com.google.protobuf.Timestamp; 11 | import gsr.proto.post.EmployeeOuterClass; 12 | import org.apache.kafka.clients.producer.Callback; 13 | import org.apache.kafka.clients.producer.KafkaProducer; 14 | import org.apache.kafka.clients.producer.ProducerConfig; 15 | import org.apache.kafka.clients.producer.ProducerRecord; 16 | import org.apache.kafka.clients.producer.RecordMetadata; 17 | import org.apache.kafka.common.serialization.StringSerializer; 18 | import org.slf4j.Logger; 19 | import org.slf4j.LoggerFactory; 20 | import software.amazon.awssdk.services.glue.model.DataFormat; 21 | 22 | import static com.google.protobuf.util.Timestamps.fromMillis; 23 | import static java.lang.System.currentTimeMillis; 24 | 25 | /** 26 | * 27 | * A Kafka Java Producer implemented in Java producing Proto messages using Glue Schema Registry 28 | * It uses a SplittableRandom as it is a lot faster than the default implementation, and we are not using it for 29 | * cryptographic functions 30 | * 31 | * @author acmanjon @amazon.com 32 | */ 33 | public class ProtoProducerSchemaRegistry { 34 | 35 | 36 | protected final Logger logger = LoggerFactory.getLogger(getClass()); 37 | private static String bootstrapServers = "localhost:9092"; // by default localhost 38 | 39 | private static final SplittableRandom sr = new SplittableRandom(); 40 | 41 | private Properties getProducerConfig() { 42 | Properties props = new Properties(); 43 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapServers); 44 | props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); 45 | props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, GlueSchemaRegistryKafkaSerializer.class.getName()); 46 | props.put(AWSSchemaRegistryConstants.DATA_FORMAT, DataFormat.PROTOBUF.name()); 47 | props.put(AWSSchemaRegistryConstants.AWS_REGION, "eu-west-1"); 48 | props.put(AWSSchemaRegistryConstants.REGISTRY_NAME, "employee-schema-registry"); 49 | props.put(AWSSchemaRegistryConstants.SCHEMA_NAME, "Employee.proto"); 50 | props.put(AWSSchemaRegistryConstants.PROTOBUF_MESSAGE_TYPE, ProtobufMessageType.POJO.getName()); 51 | return props; 52 | } 53 | 54 | /** 55 | * Create employee record employee outer class . employee. 56 | * 57 | * @param employeeId the employee id 58 | * @return the employee outer class . employee 59 | */ 60 | public EmployeeOuterClass.Employee createEmployeeRecord(int employeeId) { 61 | Timestamp ts = fromMillis(currentTimeMillis()); 62 | EmployeeOuterClass.Employee employee 63 | = EmployeeOuterClass.Employee.newBuilder() 64 | .setId((sr.nextInt(100000))) 65 | .setName("Dummy"+sr.nextInt(100)) 66 | .setAddress("Melbourne, Australia") 67 | .setEmployeeAge(Int32Value.newBuilder().setValue(sr.nextInt(99)).build()) 68 | .setStartDate((ts)) 69 | .setRole(EmployeeOuterClass.Role.ARCHITECT) 70 | .setTeam(EmployeeOuterClass.Team.newBuilder() 71 | .setName("Solutions Architects") 72 | .setLocation("Australia").build()).build(); 73 | 74 | return employee; 75 | } 76 | 77 | /** 78 | * Start producer. 79 | * 80 | * @throws InterruptedException the interrupted exception 81 | */ 82 | public void startProducer() throws InterruptedException { 83 | String topic = "protobuf-demo-topic"; 84 | try(KafkaProducer producer = new KafkaProducer<>(getProducerConfig())){ 85 | logger.warn("Starting to send records..."); 86 | int employeeId = 0; 87 | while (employeeId < 1000) { 88 | EmployeeOuterClass.Employee person = createEmployeeRecord(employeeId); 89 | String key = "key-" + employeeId; 90 | ProducerRecord record = new ProducerRecord<>(topic, key, person); 91 | producer.send(record, new ProducerCallback()); 92 | employeeId++; 93 | } 94 | } 95 | } 96 | 97 | private class ProducerCallback implements Callback { 98 | 99 | @Override 100 | public void onCompletion(RecordMetadata recordMetaData, Exception e) { 101 | if (e == null) { 102 | logger.warn("Received new metadata. \n" 103 | + "Topic:" + recordMetaData.topic() + "\n" 104 | + "Partition: " + recordMetaData.partition() + "\n" 105 | + "Offset: " + recordMetaData.offset() + "\n" 106 | + "Timestamp: " + recordMetaData.timestamp()); 107 | } else { 108 | logger.warn("There's been an error from the Producer side"); 109 | e.printStackTrace(); 110 | } 111 | } 112 | } 113 | 114 | /** 115 | * Main entry point 116 | * 117 | * @param args the kafkaBootstrapString -- optional defaults to localhost:9092 118 | * @throws InterruptedException the interrupted exception 119 | */ 120 | public static void main(String args[]) throws InterruptedException { 121 | if(args.length == 1) { 122 | bootstrapServers=args[0]; 123 | } 124 | ProtoProducerSchemaRegistry producer = new ProtoProducerSchemaRegistry(); 125 | producer.startProducer(); 126 | } 127 | 128 | } 129 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/avro/kafka/producer/AvroProducer.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.avro.kafka.producer; 2 | 3 | import java.io.IOException; 4 | import java.time.Instant; 5 | import java.util.Properties; 6 | import java.util.SplittableRandom; 7 | 8 | import com.aws.emr.proto.kafka.producer.ProtoProducer; 9 | import gsr.avro.post.Employee; 10 | import org.apache.kafka.clients.producer.Callback; 11 | import org.apache.kafka.clients.producer.KafkaProducer; 12 | import org.apache.kafka.clients.producer.ProducerConfig; 13 | import org.apache.kafka.clients.producer.ProducerRecord; 14 | import org.apache.kafka.clients.producer.RecordMetadata; 15 | import org.apache.kafka.common.serialization.ByteArraySerializer; 16 | import org.apache.kafka.common.serialization.StringSerializer; 17 | import org.apache.logging.log4j.LogManager; 18 | import static com.google.protobuf.util.Timestamps.fromMillis; 19 | import static java.lang.System.currentTimeMillis; 20 | 21 | /** 22 | * 23 | * A Kafka Java Producer implemented in Java producing avro messages. 24 | * It uses a SplittableRandom as it is a lot faster than the default implementation, and we are not using it for 25 | * cryptographic functions 26 | * 27 | * @author acmanjon @amazon.com 28 | */ 29 | 30 | public class AvroProducer { 31 | 32 | private static final org.apache.logging.log4j.Logger log = LogManager.getLogger(AvroProducer.class); 33 | 34 | private static final SplittableRandom sr = new SplittableRandom(); 35 | private static boolean lateEvent=true; 36 | private static boolean duplicates=true; 37 | 38 | /** 39 | * The constant bootstrapServers. 40 | */ 41 | protected static String bootstrapServers="localhost:9092"; // by default localhost 42 | 43 | /** 44 | * Main entry point. 45 | * 46 | * @param args the kafkaBootstrapString -- optional defaults to localhost:9092 47 | * @throws InterruptedException the interrupted exception 48 | */ 49 | public static void main(String args[]) throws InterruptedException { 50 | if(args.length == 1) { 51 | bootstrapServers=args[0]; 52 | } 53 | log.warn("Kafka bootstrap servers are set to "+bootstrapServers); 54 | AvroProducer producer = new AvroProducer(); 55 | producer.startProducer(); 56 | } 57 | 58 | private Properties getProducerConfig() { 59 | Properties props = new Properties(); 60 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapServers); 61 | props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); 62 | props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); 63 | return props; 64 | } 65 | 66 | /** 67 | * Create employee record employee outer class . employee. 68 | * 69 | * @return the employee outer class . employee 70 | */ 71 | public Employee createEmployeeRecord() { 72 | Instant instant = Instant.now(); 73 | var time=instant.toEpochMilli(); 74 | if(AvroProducer.lateEvent){ 75 | // 0.001% we will have a "late" event touching the hour before 76 | if (sr.nextInt(1000) == 0) { 77 | time= time - 3600000; 78 | } 79 | } 80 | Employee emp=Employee.newBuilder() 81 | .setEmployeeId(sr.nextInt(100000)) 82 | .setName("Dummy"+sr.nextInt(100)) 83 | .setAddress("Melbourne, Australia") 84 | .setAge(sr.nextInt(99)) 85 | .setStartDate(time) 86 | .setRole("ARCHITECT") 87 | .setTeam("Solutions Architects") 88 | .build(); 89 | 90 | return emp; 91 | 92 | } 93 | 94 | /** 95 | * Start producer. 96 | * 97 | * @throws InterruptedException the interrupted exception 98 | */ 99 | public void startProducer() throws InterruptedException { 100 | String topic = "avro-demo-topic-pure"; 101 | 102 | try (KafkaProducer producer = new KafkaProducer<>(getProducerConfig())){ 103 | log.warn("Starting to send records..."); 104 | int count = 1; 105 | int throttle = 0; 106 | while (true) { 107 | if (count % 100000000 == 0) { 108 | log.warn("100 million messages produced... "); 109 | } 110 | Employee person = createEmployeeRecord(); 111 | // for kafka key specification, not used in this example 112 | // String key = "key-" + employeeId; 113 | 114 | var buf= person.toByteBuffer(); 115 | var array=new byte[buf.remaining()]; 116 | buf.get(array); 117 | ProducerRecord record = new ProducerRecord<>(topic, array); 118 | producer.send(record, new ProducerCallback()); 119 | if(AvroProducer.duplicates){ 120 | // 0.005% we will have a "duplicate" event 121 | if (sr.nextInt(500) == 0) { 122 | producer.send(record, new ProducerCallback()); 123 | } 124 | } 125 | count++; 126 | throttle++; 127 | // if you want to really push just un-comment this block 128 | 129 | /* if (throttle % 70000 == 0) { 130 | TimeUnit.MILLISECONDS.sleep(400); //about 20.000 msg/seg 131 | }*/ 132 | } 133 | } catch (IOException e) { 134 | throw new RuntimeException(e); 135 | } 136 | } 137 | 138 | private class ProducerCallback implements Callback { 139 | 140 | @Override 141 | public void onCompletion(RecordMetadata recordMetaData, Exception e) { 142 | if (e == null) { 143 | log.debug("Received new metadata. \n" 144 | + "Topic:" + recordMetaData.topic() + "\n" 145 | + "Partition: " + recordMetaData.partition() + "\n" 146 | + "Offset: " + recordMetaData.offset() + "\n" 147 | + "Timestamp: " + recordMetaData.timestamp()); 148 | } else { 149 | log.warn("There's been an error from the Producer side"); 150 | e.printStackTrace(); 151 | } 152 | } 153 | } 154 | 155 | } 156 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/spark/iot/SparkIcebergUtils.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.spark.iot; 2 | import java.io.IOException; 3 | import java.util.concurrent.TimeoutException; 4 | 5 | import org.apache.logging.log4j.LogManager; 6 | import org.apache.logging.log4j.Logger; 7 | import org.apache.spark.sql.*; 8 | import org.apache.spark.sql.streaming.StreamingQueryException; 9 | 10 | /** 11 | * 12 | * An example compaction and snapshot expiration on Apache Iceberg on the local environment mode. 13 | * 14 | * Implementation for Glue Catalog up to the user 15 | * 16 | * @author acmanjon@amazon.com 17 | */ 18 | 19 | public class SparkIcebergUtils { 20 | 21 | private static final Logger log = LogManager.getLogger(SparkIcebergUtils.class); 22 | private static boolean snapshotExpiration = true; 23 | private static boolean compactionEnabled = true; 24 | private static boolean removeDuplicates = true; 25 | 26 | public static void main(String[] args) throws IOException, TimeoutException, StreamingQueryException { 27 | SparkSession spark = SparkSession 28 | .builder() 29 | .master("local[*]") 30 | .appName("") 31 | .config("spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 32 | .config("spark.sql.catalog.spark_catalog","org.apache.iceberg.spark.SparkSessionCatalog") 33 | .config("spark.sql.catalog.spark_catalog.type","hive") 34 | .config("spark.sql.catalog.local","org.apache.iceberg.spark.SparkCatalog") 35 | .config("spark.sql.catalog.local.type","hadoop") 36 | .config("spark.sql.shuffle.partitions","50") //we are not using AQE then we need to tune this 37 | .config("spark.sql.catalog.local.warehouse","warehouse") 38 | .config("spark.sql.defaultCatalog","local") 39 | .getOrCreate(); 40 | 41 | spark.sql(""" 42 | CREATE DATABASE IF NOT EXISTS bigdata; 43 | """); 44 | 45 | spark.sql(""" 46 | USE bigdata; 47 | """); 48 | 49 | spark.sql( 50 | """ 51 | CREATE TABLE IF NOT EXISTS employee 52 | (employee_id bigint, 53 | age int, 54 | start_date timestamp, 55 | team string, 56 | role string, 57 | address string, 58 | name string 59 | ) 60 | PARTITIONED BY (bucket(8, employee_id), hours(start_date), team) 61 | TBLPROPERTIES ( 62 | 'table_type'='ICEBERG', 63 | 'write.parquet.compression-level'='7', 64 | 'format'='parquet', 65 | 'commit.retry.num-retries'='10', --Number of times to retry a commit before failing 66 | 'commit.retry.min-wait-ms'='250', --Minimum time in milliseconds to wait before retrying a commit 67 | 'commit.retry.max-wait-ms'='60000', -- (1 min) Maximum time in milliseconds to wait before retrying a commit 68 | 'write.parquet.compression-codec'='zstd', 69 | -- if you have a huge number of columns remember to tune dict-size and page-size 70 | 'compatibility.snapshot-id-inheritance.enabled'='true' ); 71 | """); 72 | 73 | if (snapshotExpiration) { 74 | // remember to config the tables or look the defaults to see what is going to be deleted 75 | spark 76 | .sql( 77 | """ 78 | CALL system.expire_snapshots( 79 | table => 'employee' 80 | ) 81 | """) 82 | .show(); 83 | } 84 | if(compactionEnabled){ 85 | spark 86 | .sql( 87 | """ 88 | CALL system.rewrite_data_files( 89 | table => 'employee', 90 | strategy => 'sort', 91 | sort_order => 'start_date', 92 | where => 'start_date >= (current_timestamp() - INTERVAL 2 HOURS) AND start_date <= (current_timestamp() - INTERVAL 1 HOURS)', -- this sql needs to be adapted to only compact older partitions 93 | options => map( 94 | 'rewrite-job-order','bytes-asc', 95 | 'target-file-size-bytes','273741824', 96 | 'max-file-group-size-bytes','10737418240', 97 | 'partial-progress.enabled', 'true', 98 | 'max-concurrent-file-group-rewrites', '10000', 99 | 'partial-progress.max-commits', '10' 100 | )) 101 | """) 102 | .show(); 103 | 104 | if(removeDuplicates){ 105 | //iceberg prefer dynamic overwrite, just set it 106 | spark.sparkContext().conf().set("spark.sql.sources.partitionOverwriteMode","dynamic"); 107 | //remove duplicates from a partition or a set of partitions, this query needs to be tested 108 | spark 109 | .sql(""" 110 | INSERT OVERWRITE employee 111 | SELECT employee_id, start_date, first(team),first(role),first(address),first(name) 112 | FROM employee 113 | WHERE cast(start_date as date) = '2020-07-01' -- here we remove from a predefined day 114 | GROUP BY employee_id, start_date 115 | """ 116 | ) 117 | .show(); 118 | } 119 | } 120 | } 121 | 122 | } 123 | 124 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/proto/kafka/producer/ProtoProducer.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.proto.kafka.producer; 2 | 3 | import java.util.Properties; 4 | import java.util.SplittableRandom; 5 | import java.util.concurrent.TimeUnit; 6 | 7 | import com.google.protobuf.Int32Value; 8 | import com.google.protobuf.Timestamp; 9 | import gsr.proto.post.EmployeeOuterClass; 10 | import org.apache.kafka.clients.producer.Callback; 11 | import org.apache.kafka.clients.producer.KafkaProducer; 12 | import org.apache.kafka.clients.producer.ProducerConfig; 13 | import org.apache.kafka.clients.producer.ProducerRecord; 14 | import org.apache.kafka.clients.producer.RecordMetadata; 15 | import org.apache.kafka.common.serialization.ByteArraySerializer; 16 | import org.apache.kafka.common.serialization.StringSerializer; 17 | import org.apache.logging.log4j.LogManager; 18 | 19 | import static com.google.protobuf.util.Timestamps.fromMillis; 20 | import static java.lang.System.currentTimeMillis; 21 | 22 | /** 23 | * 24 | * A Kafka Java Producer implemented in Java producing Proto messages. 25 | * It uses a SplittableRandom as it is a lot faster than the default implementation, and we are not using it for 26 | * cryptographic functions 27 | * 28 | * @author acmanjon@amazon.com 29 | */ 30 | public class ProtoProducer { 31 | 32 | private static final org.apache.logging.log4j.Logger log = LogManager.getLogger(ProtoProducer.class); 33 | 34 | private static final SplittableRandom sr = new SplittableRandom(); 35 | private static boolean lateEvent=true; 36 | private static boolean duplicates=true; 37 | 38 | /** 39 | * The constant bootstrapServers. 40 | */ 41 | protected static String bootstrapServers="localhost:9092"; // by default localhost 42 | 43 | 44 | /** 45 | * Main entry point. 46 | * 47 | * @param args the kafkaBootstrapString -- optional defaults to localhost:9092 48 | * @throws InterruptedException the interrupted exception 49 | */ 50 | public static void main(String args[]) throws InterruptedException { 51 | if(args.length == 1) { 52 | bootstrapServers=args[0]; 53 | } else if (args.length ==2){ 54 | lateEvent=Boolean.parseBoolean(args[1]); 55 | }else if (args.length ==3){ 56 | duplicates=Boolean.parseBoolean(args[2]); 57 | } 58 | log.warn("Kafka bootstrap servers are set to "+bootstrapServers); 59 | ProtoProducer producer = new ProtoProducer(); 60 | producer.startProducer(); 61 | } 62 | 63 | private Properties getProducerConfig() { 64 | Properties props = new Properties(); 65 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapServers); 66 | props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); 67 | props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); 68 | return props; 69 | } 70 | 71 | /** 72 | * Create employee record employee outer class . employee. 73 | * 74 | * @return the employee outer class . employee 75 | */ 76 | public EmployeeOuterClass.Employee createEmployeeRecord() { 77 | Timestamp ts; 78 | if(ProtoProducer.lateEvent){ 79 | // 0.001% we will have a "late" event touching the hour before 80 | if (sr.nextInt(1000) == 0) { 81 | ts = fromMillis(currentTimeMillis() - 3600000); 82 | } else{ 83 | ts = fromMillis(currentTimeMillis()); 84 | }}else{ 85 | ts = fromMillis(currentTimeMillis()); 86 | } 87 | EmployeeOuterClass.Employee employee 88 | = EmployeeOuterClass.Employee.newBuilder() 89 | .setId((sr.nextInt(100000))) 90 | .setName("Dummy"+sr.nextInt(100)) 91 | .setAddress("Melbourne, Australia") 92 | .setEmployeeAge(Int32Value.newBuilder().setValue(sr.nextInt(99)).build()) 93 | .setStartDate((ts)) 94 | .setRole(EmployeeOuterClass.Role.ARCHITECT) 95 | .setTeam(EmployeeOuterClass.Team.newBuilder() 96 | .setName("Solutions Architects") 97 | .setLocation("Australia").build()).build(); 98 | 99 | return employee; 100 | } 101 | 102 | /** 103 | * Start producer. 104 | * 105 | * @throws InterruptedException the interrupted exception 106 | */ 107 | public void startProducer() throws InterruptedException { 108 | String topic = "protobuf-demo-topic-pure"; 109 | 110 | try (KafkaProducer producer = new KafkaProducer<>(getProducerConfig())){ 111 | log.warn("Starting to send records..."); 112 | int count = 1; 113 | int throttle = 0; 114 | while (true) { 115 | if (count % 100000000 == 0) { 116 | log.warn("100 million messages produced... "); 117 | } 118 | EmployeeOuterClass.Employee person = createEmployeeRecord(); 119 | // for kafka key specification, not used in this example 120 | // String key = "key-" + employeeId; 121 | ProducerRecord record = new ProducerRecord<>(topic, person.toByteArray()); 122 | producer.send(record, new ProducerCallback()); 123 | if(ProtoProducer.duplicates){ 124 | // 0.005% we will have a "duplicate" event 125 | if (sr.nextInt(500) == 0) { 126 | record = new ProducerRecord<>(topic, person.toByteArray()); 127 | producer.send(record, new ProducerCallback()); 128 | } 129 | } 130 | count++; 131 | throttle++; 132 | // if you want to really push just un-comment this block 133 | /*if (throttle % 50000 == 0) { 134 | TimeUnit.MILLISECONDS.sleep(200); 135 | }*/ 136 | } 137 | } 138 | } 139 | 140 | private class ProducerCallback implements Callback { 141 | 142 | @Override 143 | public void onCompletion(RecordMetadata recordMetaData, Exception e) { 144 | if (e == null) { 145 | log.debug("Received new metadata. \n" 146 | + "Topic:" + recordMetaData.topic() + "\n" 147 | + "Partition: " + recordMetaData.partition() + "\n" 148 | + "Offset: " + recordMetaData.offset() + "\n" 149 | + "Timestamp: " + recordMetaData.timestamp()); 150 | } else { 151 | log.warn("There's been an error from the Producer side"); 152 | e.printStackTrace(); 153 | } 154 | } 155 | } 156 | 157 | } 158 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/spark/cdc/SparkIncrementalPipeline.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.spark.cdc; 2 | 3 | import org.apache.iceberg.Snapshot; 4 | import org.apache.iceberg.Table; 5 | import org.apache.iceberg.spark.CommitMetadata; 6 | import org.apache.iceberg.spark.Spark3Util; 7 | import org.apache.iceberg.util.SnapshotUtil; 8 | import org.apache.logging.log4j.LogManager; 9 | import org.apache.logging.log4j.Logger; 10 | import org.apache.spark.sql.Dataset; 11 | import org.apache.spark.sql.Row; 12 | import org.apache.spark.sql.SparkSession; 13 | import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; 14 | import org.apache.spark.sql.catalyst.parser.ParseException; 15 | 16 | import java.util.Map; 17 | 18 | public class SparkIncrementalPipeline { 19 | 20 | private static final Logger log = LogManager.getLogger(SparkLogChange.class); 21 | private static String master = ""; 22 | private static String icebergWarehouse = "warehouse/"; 23 | public static void main(String[] args) throws NoSuchTableException, ParseException { 24 | 25 | SparkSession spark; 26 | //local environment 27 | if (args.length < 1) { 28 | master = "local[*]"; 29 | log.warn( 30 | "No arguments provided, running using local default settings: master={} and Iceberg hadoop based file catalog ", 31 | master); 32 | log.warn( 33 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 34 | + " this mode is for local based execution and development'"); 35 | spark = 36 | SparkSession.builder() 37 | .master(master) 38 | .appName("CDCLogChangeWriter") 39 | .config("spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 40 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 41 | .config("spark.sql.catalog.spark_catalog.type", "hive") 42 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 43 | .config("spark.sql.catalog.local.type", "hadoop") 44 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 45 | .config("spark.sql.catalog.local.warehouse", "warehouse") 46 | .config("spark.sql.defaultCatalog", "local") 47 | .getOrCreate(); 48 | //local environment with deduplication via watermarking 49 | } else if (args.length == 1) { 50 | master = "local[*]"; 51 | log.warn( 52 | "Running with local master: {} and Iceberg hadoop based file catalog", 53 | master 54 | ); 55 | log.warn( 56 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 57 | + " this mode is for local based execution. Kafka broker in this case will also be 'localhost:9092'."); 58 | 59 | spark = 60 | SparkSession.builder() 61 | .master(master) 62 | .appName("CDCLogChangeWriter") 63 | .config("spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 64 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 65 | .config("spark.sql.catalog.spark_catalog.type", "hive") 66 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 67 | .config("spark.sql.catalog.local.type", "hadoop") 68 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 69 | .config("spark.sql.catalog.local.warehouse", "warehouse") 70 | .config("spark.sql.defaultCatalog", "local") 71 | .getOrCreate(); 72 | } else if (args.length == 6) { 73 | icebergWarehouse = args[1]; 74 | log.warn( 75 | "Master will be inferred from the environment Iceberg Glue catalog will be used, with the warehouse being: {} \n " 76 | ,icebergWarehouse 77 | ); 78 | spark = 79 | SparkSession.builder() 80 | .appName("CDCLogChangeWriter") 81 | .config("spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 82 | .config("spark.sql.catalog.glue_catalog.warehouse", "org.apache.iceberg.spark.SparkCatalog") 83 | .config("spark.sql.catalog.glue_catalog.warehouse", icebergWarehouse) 84 | .config("spark.sql.catalog.glue_catalog.catalog-impl","org.apache.iceberg.aws.glue.GlueCatalog") 85 | .config("spark.sql.catalog.glue_catalog.io-impl", "org.apache.iceberg.aws.s3.S3FileIO") 86 | .config("spark.hadoop.fs.s3.impl","org.apache.hadoop.fs.s3a.S3AFileSystem") 87 | .config("spark.sql.iceberg.data-prefetch.enabled","true") 88 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 89 | .config("spark.sql.defaultCatalog", "glue_catalog") 90 | .getOrCreate(); 91 | } else { 92 | spark = null; 93 | log.error( 94 | "Invalid number of arguments provided, please check the readme for the correct usage"); 95 | System.exit(1); 96 | } 97 | spark.sql( 98 | """ 99 | CREATE DATABASE IF NOT EXISTS bigdata; 100 | """); 101 | 102 | spark.sql( 103 | """ 104 | USE bigdata; 105 | """); 106 | // load the logs table (this is the target table that has been deduplicated) 107 | Table logsTable = Spark3Util.loadIcebergTable(spark, "accounts_mirror"); 108 | // Retrieve last processed snapshot ID from the table 109 | // Find the last processed ID when the last incremental run isn't the current snapshot 110 | String lastProcessedId = null; 111 | for (Snapshot snap : SnapshotUtil.currentAncestors(logsTable)) { 112 | lastProcessedId = snap.summary().get("watermark:accounts_changelog"); 113 | if (lastProcessedId != null) { 114 | break; 115 | } 116 | } 117 | 118 | // load the source table, log_source 119 | Table logSourceTable = Spark3Util.loadIcebergTable(spark, "accounts_changelog"); 120 | String toProcessId = Long.toString(logSourceTable.currentSnapshot().snapshotId()); 121 | 122 | log.warn("last processed id was {} and the to process id is {}", lastProcessedId, toProcessId); 123 | // do the incremental read from the source table 124 | Dataset newLogs = spark.read() 125 | .format("iceberg") 126 | .option("start-snapshot-id", lastProcessedId) 127 | .option("end-snapshot-id", toProcessId) 128 | .table("raw.log_source"); 129 | // create a temp view from the result so we can use it in the MERGE INTO sql 130 | newLogs.createOrReplaceTempView("accounts_source"); 131 | // Update the target table and set the watermark in the same commit 132 | CommitMetadata.withCommitProperties( 133 | Map.of("watermark:accounts_source", toProcessId), 134 | () -> { 135 | spark.sql("MERGE INTO ..."); 136 | return 0; 137 | }, RuntimeException.class); 138 | 139 | // Rollback & replay 140 | // CALL spark.rollback_to_snapshot('accounts_mirror', ); 141 | 142 | } 143 | } 144 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/spark/cdc/SparkCDCMirror.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.spark.cdc; 2 | 3 | import java.io.IOException; 4 | import java.util.concurrent.TimeoutException; 5 | 6 | import org.apache.logging.log4j.LogManager; 7 | import org.apache.logging.log4j.Logger; 8 | import org.apache.spark.sql.*; 9 | import org.apache.spark.sql.streaming.StreamingQueryException; 10 | 11 | /** 12 | * An example of MERGE INTO in CDC scenario via Mirror table 13 | * 14 | * @author acmanjon@amazon.com 15 | */ 16 | 17 | public class SparkCDCMirror { 18 | 19 | private static final Logger log = LogManager.getLogger(SparkCDCMirror.class); 20 | private static String master = ""; 21 | private static String icebergWarehouse = "warehouse/"; 22 | private static String checkpointDir = "tmp/"; 23 | private static String bootstrapServers = "localhost:9092"; 24 | 25 | public static void main(String[] args) 26 | throws IOException, TimeoutException, StreamingQueryException { 27 | 28 | SparkSession spark = null; 29 | //local environment 30 | if (args.length < 1) { 31 | master = "local[*]"; 32 | log.warn( 33 | "No arguments provided, running using local default settings: master={} and Iceberg hadoop based file catalog ", 34 | master); 35 | log.warn( 36 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 37 | + " this mode is for local based execution and development. Kafka broker in this case will also be 'localhost:9092'." 38 | + " Remember to clean the checkpoint dir for any changes or if you want to start 'clean'"); 39 | spark = 40 | SparkSession.builder() 41 | .master(master) 42 | .appName("CDCLogChangeWriter") 43 | .config("spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 44 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 45 | .config("spark.sql.catalog.spark_catalog.type", "hive") 46 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 47 | .config("spark.sql.catalog.local.type", "hadoop") 48 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 49 | .config("spark.sql.catalog.local.warehouse", "warehouse") 50 | .config("spark.sql.defaultCatalog", "local") 51 | .getOrCreate(); 52 | //local environment with deduplication via watermarking 53 | } else if (args.length == 1) { 54 | master = "local[*]"; 55 | log.warn( 56 | "Running with local master: {} and Iceberg hadoop based file catalog", 57 | master 58 | ); 59 | log.warn( 60 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 61 | + " this mode is for local based execution. Kafka broker in this case will also be 'localhost:9092'."); 62 | 63 | spark = 64 | SparkSession.builder() 65 | .master(master) 66 | .appName("CDCLogChangeWriter") 67 | .config("spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 68 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 69 | .config("spark.sql.catalog.spark_catalog.type", "hive") 70 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 71 | .config("spark.sql.catalog.local.type", "hadoop") 72 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 73 | .config("spark.sql.catalog.local.warehouse", "warehouse") 74 | .config("spark.sql.defaultCatalog", "local") 75 | .getOrCreate(); 76 | } else if (args.length == 6) { 77 | icebergWarehouse = args[1]; 78 | checkpointDir = args[3]; 79 | bootstrapServers = args[4]; 80 | log.warn( 81 | "Master will be inferred from the environment Iceberg Glue catalog will be used, with the warehouse being: {} \n " 82 | + ", the checkpoint is at: {}\n " 83 | + "and Kafka bootstrap is: {}", 84 | icebergWarehouse, 85 | checkpointDir, 86 | bootstrapServers 87 | ); 88 | spark = 89 | SparkSession.builder() 90 | .appName("CDCLogChangeWriter") 91 | .config("spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 92 | .config("spark.sql.catalog.glue_catalog.warehouse", "org.apache.iceberg.spark.SparkCatalog") 93 | .config("spark.sql.catalog.glue_catalog.warehouse", icebergWarehouse) 94 | .config("spark.sql.catalog.glue_catalog.catalog-impl","org.apache.iceberg.aws.glue.GlueCatalog") 95 | .config("spark.sql.catalog.glue_catalog.io-impl", "org.apache.iceberg.aws.s3.S3FileIO") 96 | .config("spark.hadoop.fs.s3.impl","org.apache.hadoop.fs.s3a.S3AFileSystem") 97 | .config("spark.sql.iceberg.data-prefetch.enabled","true") 98 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 99 | .config("spark.sql.defaultCatalog", "glue_catalog") 100 | .getOrCreate(); 101 | } else { 102 | log.error( 103 | "Invalid number of arguments provided, please check the readme for the correct usage"); 104 | System.exit(1); 105 | } 106 | spark.sql( 107 | """ 108 | CREATE DATABASE IF NOT EXISTS bigdata; 109 | """); 110 | 111 | spark.sql( 112 | """ 113 | USE bigdata; 114 | """); 115 | spark.sql( 116 | """ 117 | CREATE TABLE IF NOT EXISTS accounts_mirror 118 | (account_id bigint, 119 | balance float, 120 | last_updated timestamp 121 | ) 122 | PARTITIONED BY (bucket(8, account_id)) 123 | TBLPROPERTIES ( 124 | 'table_type'='ICEBERG', 125 | 'write.parquet.compression-level'='7', 126 | 'format'='parquet', 127 | 'commit.retry.num-retries'='10', --Number of times to retry a commit before failing 128 | 'commit.retry.min-wait-ms'='250', --Minimum time in milliseconds to wait before retrying a commit 129 | 'commit.retry.max-wait-ms'='60000', -- (1 min) Maximum time in milliseconds to wait before retrying a commit 130 | 'write.parquet.compression-codec'='zstd', 131 | -- if you have a huge number of columns remember to tune dict-size and page-size 132 | 'compatibility.snapshot-id-inheritance.enabled'='true' ); 133 | """); 134 | 135 | // we just filter changes from the last day as we don't want to scan for latest change while deduplication 136 | // on a huge dataset, I filter by timestamp, but it would be great to use advanced techniques such as the ones 137 | // presented on https://tabular.io/apache-iceberg-cookbook/data-engineering-incremental-processing/ 138 | 139 | spark.sql(""" 140 | WITH windowed_changes AS ( 141 | SELECT 142 | account_id, 143 | balance, 144 | last_updated, 145 | operation, 146 | row_number() OVER ( 147 | PARTITION BY account_id 148 | ORDER BY last_updated DESC) AS row_num 149 | FROM accounts_changelog where last_updated > current_timestamp() - INTERVAL 1 DAY 150 | ), 151 | accounts_changes AS ( 152 | SELECT * FROM windowed_changes WHERE row_num = 1 153 | ) 154 | MERGE INTO accounts_mirror a USING accounts_changes c 155 | ON a.account_id = c.account_id 156 | WHEN MATCHED AND c.operation = 'D' THEN DELETE 157 | WHEN MATCHED THEN UPDATE 158 | SET a.balance = c.balance, 159 | a.last_updated = c.last_updated 160 | WHEN NOT MATCHED AND c.operation != 'D' THEN 161 | INSERT (account_id, balance, last_updated) 162 | VALUES (c.account_id, c.balance, c.last_updated); 163 | 164 | """); 165 | } 166 | } 167 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/proto/SparkNativeIcebergIngestProto.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.proto; 2 | 3 | import static org.apache.spark.sql.functions.col; 4 | import static org.apache.spark.sql.protobuf.functions.*; 5 | 6 | import java.io.IOException; 7 | import java.util.concurrent.TimeUnit; 8 | import java.util.concurrent.TimeoutException; 9 | 10 | import org.apache.logging.log4j.LogManager; 11 | import org.apache.logging.log4j.Logger; 12 | import org.apache.spark.sql.*; 13 | import org.apache.spark.sql.streaming.StreamingQuery; 14 | import org.apache.spark.sql.streaming.StreamingQueryException; 15 | import org.apache.spark.sql.streaming.Trigger; 16 | 17 | /** 18 | * 19 | * An example of consuming messages from Kafka using Protocol Buffers and writing them to Iceberg using the native 20 | * data source and writing via native Spark/Iceberg writing mechanism 21 | * 22 | * @author acmanjon@amazon.com 23 | */ 24 | 25 | public class SparkNativeIcebergIngestProto { 26 | 27 | private static final Logger log = LogManager.getLogger(SparkNativeIcebergIngestProto.class); private static String master = ""; 28 | 29 | private static boolean removeDuplicates = false; 30 | private static String protoDescFile = "Employee.desc"; 31 | private static String icebergWarehouse = "warehouse/"; 32 | private static String checkpointDir = "tmp/"; 33 | private static String bootstrapServers = "localhost:9092"; 34 | public static void main(String[] args) 35 | throws IOException, TimeoutException, StreamingQueryException { 36 | 37 | SparkSession spark = null; 38 | if (args.length < 1 ){ 39 | master="local[*]"; 40 | log.warn("No arguments provided, running using local default settings: master={} and Iceberg hadoop based file catalog ", master); 41 | log.warn( "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" + 42 | " this mode is for local based execution and development. Kafka broker in this case will also be 'localhost:9092'." + 43 | " Remember to clean the checkpoint dir for any changes or if you want to start 'clean'"); 44 | removeDuplicates = false; 45 | spark= SparkSession.builder() 46 | .master(master) 47 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 48 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 49 | .config("spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 50 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 51 | .config("spark.sql.catalog.spark_catalog.type", "hive") 52 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 53 | .config("spark.sql.catalog.local.type", "hadoop") 54 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 55 | .config("spark.sql.catalog.local.warehouse", "warehouse") 56 | .config("spark.sql.defaultCatalog", "local") 57 | .getOrCreate(); 58 | }else if (args.length==1){ 59 | removeDuplicates = Boolean.parseBoolean(args[0]); 60 | master = "local[*]"; 61 | log.warn("Running with local master: {} and Iceberg hadoop based file catalog " + 62 | "removing duplicates within the watermark is {}", master,removeDuplicates); 63 | log.warn( "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" + 64 | " this mode is for local based execution. Kafka broker in this case will also be 'localhost:9092'."); 65 | 66 | spark =SparkSession.builder() 67 | .master(master) 68 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 69 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 70 | .config("spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 71 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 72 | .config("spark.sql.catalog.spark_catalog.type", "hive") 73 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 74 | .config("spark.sql.catalog.local.type", "hadoop") 75 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 76 | .config("spark.sql.catalog.local.warehouse", "warehouse") 77 | .config("spark.sql.defaultCatalog", "local") 78 | .getOrCreate(); 79 | }else if (args.length ==5){ 80 | removeDuplicates = Boolean.parseBoolean(args[0]); 81 | icebergWarehouse = args[1]; 82 | protoDescFile = args[2]; 83 | checkpointDir = args[3]; 84 | bootstrapServers= args[4]; 85 | log.warn( 86 | "Master will be inferred from the environment Iceberg Glue catalog will be used, with the warehouse being: {} \n " 87 | + "removing duplicates within the watermark is {}, the descriptor file is at: {} and the checkpoint is at: {}\n " + 88 | "Kafka bootstrap is: {}", 89 | icebergWarehouse, 90 | removeDuplicates, 91 | protoDescFile, 92 | checkpointDir,bootstrapServers); 93 | spark =SparkSession.builder() 94 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 95 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 96 | .config("spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 97 | .config("spark.sql.catalog.glue_catalog.warehouse", "org.apache.iceberg.spark.SparkCatalog") 98 | .config("spark.sql.catalog.glue_catalog.warehouse", icebergWarehouse) 99 | .config("spark.sql.catalog.glue_catalog.catalog-impl", "org.apache.iceberg.aws.glue.GlueCatalog") 100 | .config("spark.hadoop.fs.s3.impl","org.apache.hadoop.fs.s3a.S3AFileSystem") 101 | .config("spark.sql.iceberg.data-prefetch.enabled","true") 102 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 103 | .config("spark.sql.defaultCatalog", "glue_catalog") 104 | .getOrCreate(); 105 | }else{ 106 | log.error("Invalid number of arguments provided, please check the readme for the correct usage"); 107 | System.exit(1); 108 | } 109 | spark.sql( 110 | """ 111 | CREATE DATABASE IF NOT EXISTS bigdata; 112 | """); 113 | 114 | spark.sql( 115 | """ 116 | USE bigdata; 117 | """); 118 | 119 | 120 | spark.sql( 121 | """ 122 | CREATE TABLE IF NOT EXISTS employee 123 | (employee_id bigint, 124 | age int, 125 | start_date timestamp, 126 | team string, 127 | role string, 128 | address string, 129 | name string 130 | ) 131 | PARTITIONED BY (bucket(32, employee_id), hours(start_date), team) 132 | TBLPROPERTIES ( 133 | 'table_type'='ICEBERG', 134 | 'write.parquet.compression-level'='7', 135 | 'format'='parquet', 136 | 'commit.retry.num-retries'='20', --Number of times to retry a commit before failing 137 | 'commit.retry.min-wait-ms'='250', --Minimum time in milliseconds to wait before retrying a commit 138 | 'commit.retry.max-wait-ms'='60000', -- (1 min) Maximum time in milliseconds to wait before retrying a commit 139 | 'write.parquet.compression-codec'='zstd', 140 | -- if you have a huge number of columns remember to tune dict-size and page-size 141 | 'compatibility.snapshot-id-inheritance.enabled'='true' ); 142 | """); 143 | 144 | Dataset df = 145 | spark 146 | .readStream() 147 | .format("kafka") 148 | .option("kafka.bootstrap.servers", bootstrapServers) 149 | .option("subscribe", "protobuf-demo-topic-pure") 150 | .load(); 151 | 152 | Dataset output = 153 | df.select(from_protobuf(col("value"), "Employee", protoDescFile).as("Employee")) 154 | .select(col("Employee.*")) 155 | .select( 156 | col("id").as("employee_id"), 157 | col("employee_age.value").as("age"), 158 | col("start_date"), 159 | col("team.name").as("team"), 160 | col("role"), 161 | col("address"), 162 | col("name")); 163 | 164 | if(removeDuplicates){ 165 | output=output 166 | .withWatermark("start_date", "120 seconds") 167 | .dropDuplicatesWithinWatermark("start_date","employee_id"); 168 | } 169 | // output.printSchema(); 170 | StreamingQuery query = 171 | output 172 | .writeStream() 173 | .queryName("streaming-protobuf-ingest") 174 | .format("iceberg") 175 | .trigger(Trigger.ProcessingTime(5, TimeUnit.MINUTES)) 176 | .outputMode("append") 177 | .option("checkpointLocation", "tmp/") //iceberg native writing requires this to be enabled 178 | .option("fanout-enabled", "true") // disable ordering for low latency writes 179 | .toTable("employee"); 180 | query.awaitTermination(); 181 | } 182 | } 183 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/spark/cdc/SparkLogChange.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.spark.cdc; 2 | 3 | import static org.apache.spark.sql.functions.*; 4 | 5 | 6 | import java.io.IOException; 7 | import java.util.Arrays; 8 | import java.util.List; 9 | import java.util.concurrent.TimeUnit; 10 | import java.util.concurrent.TimeoutException; 11 | 12 | import org.apache.logging.log4j.LogManager; 13 | import org.apache.logging.log4j.Logger; 14 | import org.apache.spark.sql.*; 15 | import org.apache.spark.sql.streaming.StreamingQuery; 16 | import org.apache.spark.sql.streaming.StreamingQueryException; 17 | import org.apache.spark.sql.streaming.Trigger; 18 | 19 | /** 20 | * An example of consuming messages from Kafka using a CDC like String format and writing them to Iceberg 21 | * via native Spark/Iceberg writing mechanism 22 | * 23 | * @author acmanjon@amazon.com 24 | */ 25 | 26 | public class SparkLogChange { 27 | 28 | private static final Logger log = LogManager.getLogger(SparkLogChange.class); 29 | private static String master = ""; 30 | private static String icebergWarehouse = "warehouse/"; 31 | private static String checkpointDir = "tmp/"; 32 | private static String bootstrapServers = "localhost:9092"; 33 | 34 | public static void main(String[] args) 35 | throws IOException, TimeoutException, StreamingQueryException { 36 | 37 | SparkSession spark = null; 38 | //local environment 39 | if (args.length < 1) { 40 | master = "local[*]"; 41 | log.warn( 42 | "No arguments provided, running using local default settings: master={} and Iceberg hadoop based file catalog ", 43 | master); 44 | log.warn( 45 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 46 | + " this mode is for local based execution and development. Kafka broker in this case will also be 'localhost:9092'." 47 | + " Remember to clean the checkpoint dir for any changes or if you want to start 'clean'"); 48 | spark = 49 | SparkSession.builder() 50 | .master(master) 51 | .appName("CDCLogChangeWriter") 52 | .config("spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 53 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 54 | .config("spark.sql.catalog.spark_catalog.type", "hive") 55 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 56 | .config("spark.sql.catalog.local.type", "hadoop") 57 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 58 | .config("spark.sql.catalog.local.warehouse", "warehouse") 59 | .config("spark.sql.defaultCatalog", "local") 60 | .getOrCreate(); 61 | //local environment with deduplication via watermarking 62 | } else if (args.length == 1) { 63 | master = "local[*]"; 64 | log.warn( 65 | "Running with local master: {} and Iceberg hadoop based file catalog", 66 | master 67 | ); 68 | log.warn( 69 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 70 | + " this mode is for local based execution. Kafka broker in this case will also be 'localhost:9092'."); 71 | 72 | spark = 73 | SparkSession.builder() 74 | .master(master) 75 | .appName("CDCLogChangeWriter") 76 | .config("spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 77 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 78 | .config("spark.sql.catalog.spark_catalog.type", "hive") 79 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 80 | .config("spark.sql.catalog.local.type", "hadoop") 81 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 82 | .config("spark.sql.catalog.local.warehouse", "warehouse") 83 | .config("spark.sql.defaultCatalog", "local") 84 | .getOrCreate(); 85 | } else if (args.length == 6) { 86 | icebergWarehouse = args[1]; 87 | checkpointDir = args[3]; 88 | bootstrapServers = args[4]; 89 | log.warn( 90 | "Master will be inferred from the environment Iceberg Glue catalog will be used, with the warehouse being: {} \n " 91 | + ", the checkpoint is at: {}\n " 92 | + "and Kafka bootstrap is: {}", 93 | icebergWarehouse, 94 | checkpointDir, 95 | bootstrapServers 96 | ); 97 | spark = 98 | SparkSession.builder() 99 | .appName("CDCLogChangeWriter") 100 | .config("spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 101 | .config("spark.sql.catalog.glue_catalog.warehouse", "org.apache.iceberg.spark.SparkCatalog") 102 | .config("spark.sql.catalog.glue_catalog.warehouse", icebergWarehouse) 103 | .config("spark.sql.catalog.glue_catalog.catalog-impl","org.apache.iceberg.aws.glue.GlueCatalog") 104 | .config("spark.sql.catalog.glue_catalog.io-impl", "org.apache.iceberg.aws.s3.S3FileIO") 105 | .config("spark.hadoop.fs.s3.impl","org.apache.hadoop.fs.s3a.S3AFileSystem") 106 | .config("spark.sql.iceberg.data-prefetch.enabled","true") 107 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 108 | .config("spark.sql.defaultCatalog", "glue_catalog") 109 | .getOrCreate(); 110 | } else { 111 | log.error( 112 | "Invalid number of arguments provided, please check the readme for the correct usage"); 113 | System.exit(1); 114 | } 115 | spark.sql( 116 | """ 117 | CREATE DATABASE IF NOT EXISTS bigdata; 118 | """); 119 | 120 | spark.sql( 121 | """ 122 | USE bigdata; 123 | """); 124 | spark.sql( 125 | """ 126 | CREATE TABLE IF NOT EXISTS accounts_changelog 127 | ( 128 | operation string, 129 | account_id bigint, 130 | balance bigint, 131 | last_updated timestamp 132 | ) 133 | PARTITIONED BY (days(last_updated),bucket(8, account_id)) 134 | TBLPROPERTIES ( 135 | 'table_type'='ICEBERG', 136 | 'write.parquet.compression-level'='7', 137 | 'format'='parquet', 138 | 'commit.retry.num-retries'='10', --Number of times to retry a commit before failing 139 | 'commit.retry.min-wait-ms'='250', --Minimum time in milliseconds to wait before retrying a commit 140 | 'commit.retry.max-wait-ms'='60000', -- (1 min) Maximum time in milliseconds to wait before retrying a commit 141 | 'write.parquet.compression-codec'='zstd', 142 | -- if you have a huge number of columns remember to tune dict-size and page-size 143 | 'compatibility.snapshot-id-inheritance.enabled'='true' ); 144 | """); 145 | 146 | Dataset df = 147 | spark 148 | .readStream() 149 | .format("kafka") 150 | .option("kafka.bootstrap.servers", bootstrapServers) 151 | .option("subscribe", "streaming-cdc-log-ingest") 152 | .load(); 153 | 154 | 155 | 156 | var output =df.selectExpr("CAST(value AS STRING)"); 157 | 158 | List schemaList = Arrays.asList("operation","account_id","balance","last_updated"); 159 | Column column = functions.col("value"); 160 | Column linesSplit = functions.split(column,","); 161 | for(int i=0;i df = 172 | spark 173 | .readStream() 174 | .format("kafka") 175 | .option("kafka.bootstrap.servers", bootstrapServers) 176 | .option("subscribe", "avro-demo-topic-pure") 177 | .option("mode","PERMISSIVE") 178 | .load(); 179 | Map avroOptions = new HashMap(); 180 | avroOptions.put("mode", "PERMISSIVE"); 181 | 182 | Dataset output = 183 | df.select(from_avro(col("value"), jsonFormatSchema,avroOptions).as("Employee")) 184 | .select(col("Employee.*")) 185 | .select( 186 | col("employee_id"), 187 | col("age"), 188 | col("start_date").cast("timestamp"), 189 | col("team"), 190 | col("role"), 191 | col("address"), 192 | col("name")); 193 | 194 | if (removeDuplicates) { 195 | output = 196 | output 197 | .withWatermark("start_date", "120 seconds") 198 | .dropDuplicatesWithinWatermark("start_date", "employee_id"); 199 | } 200 | // output.printSchema(); 201 | StreamingQuery query = 202 | output 203 | .writeStream() 204 | .queryName("streaming-protobuf-ingest") 205 | .format("iceberg") 206 | .trigger(Trigger.ProcessingTime(1, TimeUnit.MINUTES)) 207 | .outputMode("append") 208 | .option( 209 | "checkpointLocation", "tmp/") // iceberg native writing requires this to be enabled 210 | .option("fanout-enabled", "true") // disable ordering for low latency writes 211 | .toTable("employee"); 212 | query.awaitTermination(); 213 | } 214 | } 215 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 4 | 4.0.0 5 | com.aws.emr 6 | streaming-iceberg-ingest 7 | 1.0-SNAPSHOT 8 | jar 9 | 10 | 11 | UTF-8 12 | 17 13 | 17 14 | 17 15 | 3.25.5 16 | 3.5.1 17 | 18 | 19 | 20 | emr 21 | 22 | true 23 | 24 | 25 | provided 26 | 27 | 28 | 29 | dev 30 | 31 | compile 32 | 33 | 34 | 35 | 36 | 37 | 40 | 41 | software.amazon.glue 42 | schema-registry-serde 43 | 1.1.20 44 | 45 | 46 | org.slf4j 47 | slf4j-api 48 | 49 | 50 | 51 | 52 | 53 | 54 | org.apache.spark 55 | spark-core_2.12 56 | ${spark.version} 57 | ${profile.scope} 58 | 59 | 60 | org.apache.spark 61 | spark-streaming_2.12 62 | ${profile.scope} 63 | ${spark.version} 64 | 65 | 66 | org.apache.spark 67 | spark-sql_2.12 68 | ${spark.version} 69 | ${profile.scope} 70 | 71 | 72 | 73 | 74 | org.apache.spark 75 | spark-sql-kafka-0-10_2.12 76 | ${spark.version} 77 | ${profile.scope} 78 | 79 | 80 | 81 | 82 | org.apache.spark 83 | spark-avro_2.12 84 | ${spark.version} 85 | 86 | 87 | org.apache.spark 88 | spark-protobuf_2.12 89 | ${spark.version} 90 | 91 | 92 | 93 | 94 | org.apache.iceberg 95 | iceberg-spark-runtime-3.5_2.12 96 | 1.5.2 97 | ${profile.scope} 98 | 99 | 100 | org.apache.iceberg 101 | iceberg-aws-bundle 102 | 1.5.2 103 | ${profile.scope} 104 | 105 | 106 | 107 | 108 | com.sun.xml.ws 109 | jaxws-rt 110 | 2.3.7 111 | ${profile.scope} 112 | 113 | 114 | jakarta.xml.bind 115 | jakarta.xml.bind-api 116 | 2.3.3 117 | ${profile.scope} 118 | 119 | 120 | org.glassfish.jaxb 121 | jaxb-runtime 122 | 2.3.8 123 | ${profile.scope} 124 | 125 | 126 | 127 | 128 | com.google.protobuf 129 | protobuf-java 130 | ${protobuf.version} 131 | 132 | 133 | com.google.protobuf 134 | protobuf-java-util 135 | 4.27.1 136 | 137 | 138 | 139 | org.apache.avro 140 | avro 141 | 1.11.4 142 | 143 | 144 | com.fasterxml.jackson.core 145 | jackson-core 146 | 2.17.1 147 | 148 | 149 | org.apache.logging.log4j 150 | log4j-slf4j-impl 151 | 2.23.1 152 | 153 | 154 | org.slf4j 155 | slf4j-api 156 | 2.0.13 157 | 158 | 159 | 160 | org.scala-lang.modules 161 | scala-collection-compat_2.12 162 | 2.12.0 163 | 164 | 165 | 166 | 167 | 168 | 169 | org.apache.maven.plugins 170 | maven-help-plugin 171 | 3.4.1 172 | 173 | 174 | show-profiles 175 | compile 176 | 177 | active-profiles 178 | 179 | 180 | 181 | 182 | 183 | com.github.os72 184 | protoc-jar-maven-plugin 185 | 3.11.4 186 | 187 | 188 | generate-sources 189 | 190 | run 191 | 192 | 193 | com.google.protobuf:protoc:${protobuf.version} 194 | ${protobuf.version} 195 | direct 196 | 197 | 198 | 199 | 200 | 212 | 213 | org.apache.maven.plugins 214 | maven-enforcer-plugin 215 | 3.5.0 216 | 217 | 218 | enforce-PROFILE_ONE-XOR-PROFILE_TWO-is-active 219 | 220 | enforce 221 | 222 | 223 | 224 | 225 | dev,emr 226 | false 227 | 228 | 229 | \\w+), source: \\w+\\}\\]?", "${profile}").split(",")) { 234 | if("dev".equalsIgnoreCase(s)){ profile1 = true;} 235 | if("emr".equalsIgnoreCase(s)){ profile2 = true;} 236 | } 237 | print("dev XOR emr: "+(profile1 != profile2)); 238 | return profile1 != profile2; 239 | ]]> 240 | 241 | 242 | true 243 | 244 | 245 | 246 | 247 | 248 | org.apache.avro 249 | avro-maven-plugin 250 | 1.11.1 251 | 252 | 253 | generate-sources 254 | 255 | schema 256 | 257 | 258 | ${project.basedir}/src/main/avro/ 259 | ${project.basedir}/src/main/java/ 260 | 261 | 262 | 263 | 264 | 265 | org.apache.maven.plugins 266 | maven-compiler-plugin 267 | 268 | 17 269 | 17 270 | 271 | 3.13.0 272 | 273 | 274 | 275 | org.apache.maven.plugins 276 | maven-shade-plugin 277 | 3.6.0 278 | 279 | 280 | package 281 | 282 | shade 283 | 284 | 285 | 286 | 287 | 288 | com.google.protobuf 289 | org.sparkproject.spark_protobuf.protobuf 290 | 291 | com.google.protobuf.** 292 | 293 | 294 | 295 | 296 | 297 | com.google.protobuf:* 298 | 299 | 300 | 301 | 302 | *:* 303 | 304 | META-INF/*.SF 305 | META-INF/*.DSA 306 | META-INF/*.RSA 307 | 308 | 309 | 310 | 311 | 312 | 313 | 314 | 315 | 316 | 317 | 318 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Streaming Apache Iceberg examples using Apache Spark 2 | AWS Managed Kafka and Apache Kafka, a distributed event streaming platform, has become the de facto standard for building real-time data pipelines. However, ingesting and storing large amounts of streaming data in a scalable and performant manner can be complex and resource-intensive task, often leading to performance issues and increased costs. 3 | 4 | This project covers how open table formats, such as Apache Iceberg, can help address these challenges. It provides a solution that combines the power of [Apache Kafka](https://kafka.apache.org/) , [Apache Spark](https://spark.apache.org/), and [Apache Iceberg](https://iceberg.apache.org/) to achieve high-throughput streaming ingestion 5 | 6 | The focus in this repository is to go further than the typical poc consuming few messages or small csv files. The aim here is to provide support for around **400,000 msg/seg** on all scenarios. 7 | 8 | The concepts seen here are applicable to PySpark or Scala programs with little effort. Remember that we just program 9 | the transformations and those are converted to a logical plan and then to native code via the Java Virtual Machine (JVM) or to native code using projects such as [Apache Data Fusion Comet](https://github.com/apache/datafusion-comet), [Velox](https://github.com/apache/datafusion-comet) or [Photon](https://www.databricks.com/product/photon). 10 | 11 | Why Java? Because why not, remember that this nowadays gets executed by the JVM ( until previous projects arise). Remember that with this approach we can use libraries in an easy way ( without the Scala/Python/Java 'mess'), we can program performant UDFs and there is a friendly local development environment (where you can debug everything with breakpoints) with different options. 12 | 13 | The example uses maven profiles to automatically filter required libraries when deployed to [Amazon EMR](https://aws.amazon.com/emr/) ( the Spark and Iceberg libraries will be marked as provided) and therefore you will be using the optimized Spark runtime from EMR. The logging is implemented using [Log4j2](https://logging.apache.org/log4j/2.12.x/) ( where its config can be further tuned using EMR Serverless configs) as Spark uses it behind the scenes. 14 | 15 | **Environment types:** 16 | 17 | - Local development using a [dockerized Kafka](https://github.com/bitnami/containers/blob/main/bitnami/kafka/). 18 | - Local development against Amazon S3, and AWS Glue Catalog, here we will also use the dockerized Kafka. 19 | - Production mode where we can deploy the code to an Amazon EMR Serverless cluster. 20 | 21 | You can run these examples on any Spark compatible runtime too, but that's for a pull request ( if you like to contribute). 22 | 23 | In the case of Amazon Web Services on AWS Glue, Amazon EMR or Amazon EMR Serverless. 24 | æ 25 | Remember also that these jobs and code can be adapted for **batch mode** easily (and remember that you can use Kafka as batch source!). A batch job is just a special streaming job with a start and an end anyway. 26 | 27 | ### A note on performance 28 | 29 | Although the code here aims for performance more tuning can be done for achieving specific goals such as improving latency. 30 | 31 | Remember that Apache Iceberg have merge-on-read capabilities. In this repo, the default settings for tables are used 32 | but mixing copy-on-write with merge-on-read can lead to some gains as we will write faster. 33 | 34 | Remember that this is not a free lunch, you will need to compact if you want good performance. 35 | 36 | Another cool thing to test is to use Avro for the ingestion tables and then compact to parquet. 37 | 38 | A good doc to read about these settings and more can be seen on the [Best Practices for Optimizing Apache Iceberg workloads](https://docs.aws.amazon.com/prescriptive-guidance/latest/apache-iceberg-on-aws/best-practices.html) from AWS Documentation. 39 | 40 | Another good read can be seen on this blog from Cloudera: [Optimization Strategies for Iceberg Tables](https://blog.cloudera.com/optimization-strategies-for-iceberg-tables/) 41 | 42 | ## IoT Scenarios 43 | 44 | Here we have different approaches and comœmon formats. About the different scenarios the main idea is high throughput streaming 45 | ingestion: 46 | - Native Iceberg writing with deduplication via even-time watermarking. 47 | - Custom process writing with compaction via n-batches and deduplication via merge into. 48 | - Custom process writing with async compaction and Merge-on-read mode. 49 | 50 | For the different formats we will have the native use case implemented and the ProtoBuf one will have all the scenarios. 51 | 52 | The most advanced example using Protocol Buffers is in ```com.aws.emr.spark.iot``` package. 53 | 54 | Later on a job rewriting older partitions to check for duplicates are found and rewrite affected partitions may run. 55 | An example of such approach can be seen also on the Utils class of ```com.aws.emr.spark.iot``` package. 56 | 57 | Remember that exactly once systems are difficult to implement and that for Spark you will need and idempotent sink. 58 | 59 | If you want to use the GlueSchemaRegistry you should create in the console a stream registry named ```employee-schema-registry```. 60 | 61 | ### Protocol Buffers 62 | 63 | [Protocol Buffers](https://protobuf.dev/) are language-neutral, platform-neutral extensible mechanisms for serializing structured data. 64 | 65 | **Examples**: 66 | - Native Java Producer/Consumer. 67 | - AWS Glue Registry based Java Producer/Consumer. 68 | - Native Spark Structured streaming consumer. 69 | - UDF based Spark Structured streaming consumer. 70 | 71 | Create a schema for the Glue registry ```Employee.proto``` if you like to use the Registry based producer/consumer: 72 | 73 | ``` 74 | syntax = "proto3"; 75 | package gsr.proto.post; 76 | 77 | import "google/protobuf/wrappers.proto"; 78 | import "google/protobuf/timestamp.proto"; 79 | 80 | message Employee { 81 | int32 id = 1; 82 | string name = 2; 83 | string address = 3; 84 | google.protobuf.Int32Value employee_age = 4; 85 | google.protobuf.Timestamp start_date = 5; 86 | Team team = 6; 87 | Role role = 7; 88 | 89 | } 90 | message Team { 91 | string name = 1; 92 | string location = 2; 93 | } 94 | enum Role { 95 | MANAGER = 0; 96 | DEVELOPER = 1; 97 | ARCHITECT = 2; 98 | } 99 | ``` 100 | 101 | ### Apache Avro 102 | 103 | [Apache Avro](https://avro.apache.org/) - a data serialization system. 104 | 105 | **Examples**: 106 | - Native Java Producer/Consumer. 107 | - AWS Glue Registry based Java Producer/Consumer. 108 | - Native Spark Structured streaming consumer. 109 | 110 | Create a schema for the Glue registry ```Employee.avsc``` if you like to use the Registry based producer/consumer: 111 | ``` 112 | {"namespace": "gsr.avro.post", 113 | "type": "record", 114 | "name": "Employee", 115 | "fields": [ 116 | {"name": "employee_id", "type": "long"}, 117 | {"name": "age", "type": "int"}, 118 | {"name": "start_date", "type": "long"}, 119 | {"name": "team", "type": "string"}, 120 | {"name": "role", "type": "string"}, 121 | {"name": "address", "type": "string"}, 122 | {"name": "name", "type": "string"}] 123 | } 124 | ``` 125 | 126 | ### Json 127 | 128 | There is plenty of literature over the internet on how integrate Spark with Json data, therefore we just implemented one usecase. 129 | 130 | **Examples**: 131 | - AWS Glue Registry based Java Producer/Consumer. 132 | 133 | 134 | Create a schema for the Glue registry ```Employee.json``` if you like to use the Registry based producer/consumer: 135 | ``` 136 | { 137 | "$id": "https://example.com/Employee.schema.json", 138 | "$schema": "http://json-schema.org/draft-07/schema#", 139 | "title": "Employee", 140 | "description": "", 141 | "type": "object", 142 | "properties": { 143 | "employeeId": { 144 | "description": "The unique identifier for a employee", 145 | "type": "integer" 146 | }, 147 | "name": { 148 | "description": "Name of the employee", 149 | "type": "string" 150 | } 151 | } 152 | } 153 | 154 | ``` 155 | ## CDC Scenarios 156 | 157 | Here the reference is Tabular [Apache Iceberg Cookbook](https://tabular.io/apache-iceberg-cookbook/) and these blogposts: 158 | - https://tabular.io/blog/hello-world-of-cdc/ 159 | - https://tabular.io/blog/cdc-data-gremlins/#eventual-consistency-causes-data-gremlins 160 | - https://tabular.io/blog/cdc-merge-pattern/ 161 | - https://tabular.io/blog/cdc-zen-art-of-cdc-performance/ 162 | 163 | Here we will focus on the Mirror MERGE patter, as stated in the Iceberg Cookbook the first part could be managed by 164 | the Kafka Connect Tabular connector, but we will implement both processing pipelines using Spark. 165 | 166 | The relevant classes are withing the ```com.aws.emr.spark.cdc``` package. 167 | 168 | * ```KafkaCDCSimulator``` class is a Java producer simulating CDC data in [AWS Database Migration Service(DMS)](https://aws.amazon.com/es/dms/) format. 169 | * ```SparkLogChange``` class is a Structured Streaming consumer that outputs a CDC changelog to an Iceberg table. 170 | * ```SparkCDCMirror``` class is a Spark batch pipeline that process the MERGE using the Mirror approach. 171 | * ```SparkIncrementalPipeline``` class uses Incremental pipeline for consuming the CDC changelog into a target table. 172 | 173 | ## Requirements 174 | 175 | * Java 17 + ( you could adapt this code easily to run on Java 8 or Java 11) 176 | * Maven 3.9+ 177 | * 16GB of RAM and more than 2 cores. 178 | * Whatever IDE you like ([Intellij](https://www.jetbrains.com/intellij/), [Visual Studio Code](https://code.visualstudio.com/), [NetBeans](https://apache.netbeans.org/), etc) 179 | 180 | For local development and testing you can use the provided ```docker-compose.yml``` to spin up a Kafka cluster. 181 | 182 | You can generate the description file using the protobuf compiler like this. You need to install the protobuf compiler for your system, for example on MacOs is available on ```brew```. 183 | 184 | ```protoc --include_imports --descriptor_set_out=Employee.desc Employee.proto'``` 185 | 186 | Remember that for simple scenarios you will be better suited using [Kafka Connect Tabular Iceberg Connector](https://github.com/tabular-io/iceberg-kafka-connect/tree/main) or using [Amazon Kinesis Firehose](https://aws.amazon.com/firehose/). 187 | 188 | ### Running on EMR Serverless: 189 | 190 | Create a S3 bucket with the following structure. 191 | 192 | ``` 193 | s3bucket/ 194 | /jars 195 | /employee.desc -- or your custom protocol buffers descriptors 196 | /warehouse 197 | /checkpoint 198 | ``` 199 | 200 | Package your application using the ```emr``` Maven profile, then upload the jar of the project to the ```jars``` folder. The ```warehouse``` will be the place where the Iceberg Data and Metadata will live and ```checkpoint``` will be used for Structured Streaming checkpointing mechanismn. 201 | 202 | Create a Database in the AWS Glue Data Catalog with the name ```bigdata```. 203 | 204 | You need to create an EMR Serverless application with ```default settings for batch jobs only```, application type ```Spark``` release version ```7.2.0``` and ```x86_64``` as architecture, enable ```Java 17``` as runtime, enable ```AWS Glue Data Catalog as metastore``` 205 | integration and enable ```Cloudwatch logs``` if desired. 206 | 207 | Then you can issue a job run using this aws cli command. Remember to change the desired parameters. 208 | 209 | ``` 210 | aws emr-serverless start-job-run --application-id application-identifier --name job-run-name --execution-role-arn arn-of-emrserverless-role --mode 'STREAMING' --job-driver 211 | '{ 212 | "sparkSubmit": { 213 | "entryPoint": "s3://s3bucket/jars/streaming-iceberg-ingest-1.0-SNAPSHOT.jar", 214 | "entryPointArguments": ["true","s3a://s3bucket/warehouse","/home/hadoop/Employee.desc","s3a://s3bucket/checkpoint","kafkaBootstrapString","true"], 215 | "sparkSubmitParameters": "--class com.aws.emr.spark.iot.SparkCustomIcebergIngest --conf spark.executor.cores=4 --conf spark.hadoop.hive.metastore.client.factory.class=com.amazonaws.glue.catalog.metastore.AWSGlueDataCatalogHiveClientFactory --conf spark.executor.memory=16g --conf spark.driver.cores=2 --conf spark.driver.memory=8g --files s3a://s3bucket/Employee.desc --conf spark.dynamicAllocation.minExecutors=4 --conf spark.jars=/usr/share/aws/iceberg/lib/iceberg-spark3-runtime.jar --conf spark.emr-serverless.executor.disk.type=shuffle_optimized --packages org.apache.spark:spark-sql-kafka-0-10_2.12:3.5.1" 216 | } 217 | }' 218 | { 219 | ``` 220 | 221 | Expected performance should be around 450.000 msgs per sec if you use the ```SparkCustomIcebergIngest```. 222 | 223 | 224 | 225 | You can also see the cluster autoscaling into action: 226 | 227 | 228 | 229 | ### Running on a local environment. 230 | 231 | 1. Install a Java SDK 17 like [Amazon Coretto](https://aws.amazon.com/corretto/). 232 | 2. Install [Docker](https://www.docker.com/) for your environment. 233 | 3. Open the desired IDE. 234 | 4. Use the IDE to issue the ```package ``` command of maven selecting the local profile. 235 | 5. If you wish to use the AWS Glue Data Catalog and S3 remember to have the corresponding permissions (have your AWS credentials avaliable), there are plugins for both [Intellij](https://aws.amazon.com/intellij/?pg=developertools) and [Visual Studio Code](https://aws.amazon.com/visualstudiocode/) that can be helpful here. 236 | 6. Start the local Kafka broker via ```docker-compose up``` command. 237 | 7. Run the examples with the desired arguments, remember that you will need to add the required VM options for letting Spark to work on Java 17: 238 | ``` 239 | --add-opens=java.base/java.lang=ALL-UNNAMED 240 | --add-opens=java.base/java.lang.invoke=ALL-UNNAMED 241 | --add-opens=java.base/java.lang.reflect=ALL-UNNAMED 242 | --add-opens=java.base/java.io=ALL-UNNAMED 243 | --add-opens=java.base/java.net=ALL-UNNAMED 244 | --add-opens=java.base/java.nio=ALL-UNNAMED 245 | --add-opens=java.base/java.util=ALL-UNNAMED 246 | --add-opens=java.base/java.util.concurrent=ALL-UNNAMED 247 | --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED 248 | --add-opens=java.base/sun.nio.ch=ALL-UNNAMED 249 | --add-opens=java.base/sun.nio.cs=ALL-UNNAMED 250 | --add-opens=java.base/sun.security.action=ALL-UNNAMED 251 | --add-opens=java.base/sun.util.calendar=ALL-UNNAMED 252 | --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED 253 | ``` 254 | 255 | ### Running the Kafka producer on AWS 256 | 257 | Create a Amazon MSK cluster with at leas two brokers using ```3.5.1```, [Apache Zookeeper](https://zookeeper.apache.org/) mode version and use as instance type ```kafka.m7g.xlarge```. Do not use public access and choose two private subnets to deploy it. For the security group remember that the EMR cluster and the EC2 based producer will need to reach the cluster and act accordingly. For security, use ```PLAINTEXT``` (in production you should secure access to the cluster). Choose ```200GB``` as storage size for each broker and do not enable ```Tiered storage```. For the cluster configuration use this one: 258 | 259 | ``` 260 | auto.create.topics.enable=true 261 | default.replication.factor=3 262 | min.insync.replicas=2 263 | num.io.threads=8 264 | num.network.threads=5 265 | num.partitions=32 266 | num.replica.fetchers=2 267 | replica.lag.time.max.ms=30000 268 | socket.receive.buffer.bytes=102400 269 | socket.request.max.bytes=104857600 270 | socket.send.buffer.bytes=102400 271 | unclean.leader.election.enable=true 272 | zookeeper.session.timeout.ms=18000 273 | compression.type=zstd 274 | log.retention.hours=2 275 | log.retention.bytes=10073741824 276 | ``` 277 | 278 | Running the Kafka producer on an Amazon EC2 instance, remember to change the bootstrap connection string. 279 | 280 | You will need to install Java if you are using and Amazon Linux instance. 281 | ``` 282 | sudo yum install java-17-amazon-corretto-devel 283 | ``` 284 | Then, download the jar to the instance and execute the producer. With the following command you can start the Protocol Buffers Producer. 285 | ``` 286 | aws s3 cp s3://s3bucket/jars/streaming-iceberg-ingest-1.0-SNAPSHOT.jar . 287 | java -cp streaming-iceberg-ingest-1.0-SNAPSHOT.jar com.aws.emr.proto.kafka.producer.ProtoProducer kafkaBoostrapString 288 | ``` 289 | 290 | Remember that your EC2 instance need to have network access to the MSK cluster, you will need to configure the VPC, Security Groups and Subnet/s. 291 | 292 | ## Costs 293 | 294 | Remember that this example is for high throughput scenarios and therefore the config may lead to quite big bill if deployed on top of AWS, remember to stop the EMR Serverless application, the used instance for the Kafka producer and delete the Amazon MSK cluster when not in use. 295 | 296 | ## Security 297 | 298 | The code here is not secured in any way, you should secure your Apache Kafka cluster and be aware that some dependencies may have known vulnerabilities. If you deploy any service on top of AWS you should configure the roles using the least permission model 299 | using [IAM roles](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_roles.html) and [Amazon Lake Formation](https://aws.amazon.com/lake-formation/) if needed. 300 | 301 | ## Contributing 302 | 303 | See [CONTRIBUTING](CONTRIBUTING.md) for more information. 304 | 305 | ## License 306 | 307 | This library is licensed under the MIT-0 License. See the LICENSE file. 308 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/proto/SparkCustomIcebergIngestProtoHex.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.proto; 2 | 3 | import static org.apache.spark.sql.functions.*; 4 | import static org.apache.spark.sql.protobuf.functions.*; 5 | 6 | import java.io.IOException; 7 | import java.util.HexFormat; 8 | import java.util.concurrent.TimeUnit; 9 | import java.util.concurrent.TimeoutException; 10 | import org.apache.logging.log4j.LogManager; 11 | import org.apache.logging.log4j.Logger; 12 | import org.apache.spark.api.java.function.VoidFunction2; 13 | import org.apache.spark.sql.*; 14 | import org.apache.spark.sql.streaming.StreamingQuery; 15 | import org.apache.spark.sql.streaming.StreamingQueryException; 16 | import org.apache.spark.sql.streaming.Trigger; 17 | 18 | 19 | /** 20 | * 21 | * A Spark Structured Streaming consumer implemented in Java that decodes Protocol Buffers using the native spark connectors 22 | * with the option to inject the hex for the descriptor file. 23 | * 24 | * @author acmanjon@amazon.com 25 | */ 26 | 27 | public class SparkCustomIcebergIngestProtoHex { 28 | 29 | private static final Logger log = LogManager.getLogger(SparkCustomIcebergIngestProtoHex.class); 30 | private static String master = ""; 31 | private static boolean removeDuplicates = false; 32 | private static String protoDescFile = "Employee.desc"; 33 | private static String icebergWarehouse = "warehouse/"; 34 | private static String checkpointDir = "tmp/"; 35 | private static String bootstrapServers = "localhost:9092"; 36 | private static boolean compactionEnabled = false; 37 | 38 | private static String hexData="0A86040A1E676F6F676C652F70726F746F6275662F77726170706572732E70726F746F120F676F6F676C652E70726F746F62756622230A0B446F75626C6556616C756512140A0576616C7565180120012801520576616C756522220A0A466C6F617456616C756512140A0576616C7565180120012802520576616C756522220A0A496E74363456616C756512140A0576616C7565180120012803520576616C756522230A0B55496E74363456616C756512140A0576616C7565180120012804520576616C756522220A0A496E74333256616C756512140A0576616C7565180120012805520576616C756522230A0B55496E74333256616C756512140A0576616C756518012001280D520576616C756522210A09426F6F6C56616C756512140A0576616C7565180120012808520576616C756522230A0B537472696E6756616C756512140A0576616C7565180120012809520576616C756522220A0A427974657356616C756512140A0576616C756518012001280C520576616C75654283010A13636F6D2E676F6F676C652E70726F746F627566420D577261707065727350726F746F50015A31676F6F676C652E676F6C616E672E6F72672F70726F746F6275662F74797065732F6B6E6F776E2F77726170706572737062F80101A20203475042AA021E476F6F676C652E50726F746F6275662E57656C6C4B6E6F776E5479706573620670726F746F330AFF010A1F676F6F676C652F70726F746F6275662F74696D657374616D702E70726F746F120F676F6F676C652E70726F746F627566223B0A0954696D657374616D7012180A077365636F6E647318012001280352077365636F6E647312140A056E616E6F7318022001280552056E616E6F734285010A13636F6D2E676F6F676C652E70726F746F627566420E54696D657374616D7050726F746F50015A32676F6F676C652E676F6C616E672E6F72672F70726F746F6275662F74797065732F6B6E6F776E2F74696D657374616D707062F80101A20203475042AA021E476F6F676C652E50726F746F6275662E57656C6C4B6E6F776E5479706573620670726F746F330AEE030A0E456D706C6F7965652E70726F746F120E6773722E70726F746F2E706F73741A1E676F6F676C652F70726F746F6275662F77726170706572732E70726F746F1A1F676F6F676C652F70726F746F6275662F74696D657374616D702E70726F746F2297020A08456D706C6F796565120E0A0269641801200128055202696412120A046E616D6518022001280952046E616D6512180A0761646472657373180320012809520761646472657373123E0A0C656D706C6F7965655F61676518042001280B321B2E676F6F676C652E70726F746F6275662E496E74333256616C7565520B656D706C6F79656541676512390A0A73746172745F6461746518052001280B321A2E676F6F676C652E70726F746F6275662E54696D657374616D70520973746172744461746512280A047465616D18062001280B32142E6773722E70726F746F2E706F73742E5465616D52047465616D12280A04726F6C6518072001280E32142E6773722E70726F746F2E706F73742E526F6C655204726F6C6522360A045465616D12120A046E616D6518012001280952046E616D65121A0A086C6F636174696F6E18022001280952086C6F636174696F6E2A310A04526F6C65120B0A074D414E414745521000120D0A09444556454C4F5045521001120D0A094152434849544543541002620670726F746F33"; 39 | 40 | public static void main(String[] args) 41 | throws IOException, TimeoutException, StreamingQueryException { 42 | 43 | SparkSession spark; 44 | //default local env. 45 | if (args.length < 1) { 46 | master = "local[*]"; 47 | log.warn( 48 | "No arguments provided, running using local default settings: master={} and Iceberg hadoop based file catalog ", 49 | master); 50 | log.warn( 51 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 52 | + " this mode is for local based execution and development. Kafka broker in this case will also be 'localhost:9092'." 53 | + " Remember to clean the checkpoint dir for any changes or if you want to start 'clean'"); 54 | removeDuplicates = false; 55 | spark = 56 | SparkSession.builder() 57 | .master(master) 58 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 59 | .config( 60 | "spark.sql.extensions", 61 | "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 62 | .config( 63 | "spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 64 | .config("spark.sql.catalog.spark_catalog.type", "hive") 65 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 66 | .config("spark.sql.catalog.local.type", "hadoop") 67 | .config( 68 | "spark.sql.shuffle.partitions", 69 | "50") // as we are not using AQE then we need to tune this 70 | .config("spark.sql.catalog.local.warehouse", "warehouse") 71 | .config("spark.sql.defaultCatalog", "local") 72 | .getOrCreate(); 73 | //local env with optional compaction and duplicate removal 74 | } else if (args.length == 2) { 75 | removeDuplicates = Boolean.parseBoolean(args[0]); 76 | compactionEnabled = Boolean.parseBoolean(args[1]); 77 | master = "local[*]"; 78 | log.warn( 79 | "Running with local master: {} and Iceberg hadoop based file catalog " 80 | + "removing duplicates within the watermark is {}, compactions each 'n' batch are {}", 81 | master, 82 | removeDuplicates, 83 | compactionEnabled); 84 | log.warn( 85 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 86 | + " this mode is for local based execution. Kafka broker in this case will also be 'localhost:9092'."); 87 | spark = 88 | SparkSession.builder() 89 | .master(master) 90 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 91 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 92 | .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 93 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 94 | .config("spark.sql.catalog.spark_catalog.type", "hive") 95 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 96 | .config("spark.sql.catalog.local.type", "hadoop") 97 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this for the size of our cluster/tasks 98 | // remember that we should be ideally at 200MB per task minimum 99 | .config("spark.sql.catalog.local.warehouse", "warehouse") 100 | .config("spark.sql.defaultCatalog", "local") 101 | .getOrCreate(); 102 | //local env connected to Glue catalog 103 | } else if ( args.length == 4){ 104 | removeDuplicates = Boolean.parseBoolean(args[0]); 105 | compactionEnabled = Boolean.parseBoolean(args[1]); 106 | icebergWarehouse = args[2]; 107 | checkpointDir = args[3]; 108 | master = "local[*]"; 109 | log.warn( 110 | "Running with local master and Iceberg hadoop based file catalog " 111 | + "removing duplicates within the watermark is {}, compactions each 'n' batch are {}", 112 | 113 | removeDuplicates, 114 | compactionEnabled); 115 | log.warn( 116 | "Iceberg warehouse dir will be {} from the run dir and the checkpoint directory will be {}'\n" 117 | + " this mode is for local based execution. Kafka broker in this case will also be 'localhost:9092'.", 118 | icebergWarehouse,checkpointDir); 119 | spark = 120 | SparkSession.builder() 121 | .master(master) 122 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 123 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 124 | .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 125 | .config("spark.sql.catalog.glue_catalog.catalog-impl", "org.apache.iceberg.aws.glue.GlueCatalog") 126 | .config("spark.sql.catalog.glue_catalog", "org.apache.iceberg.spark.SparkCatalog") 127 | .config("spark.sql.catalog.glue_catalog.warehouse", icebergWarehouse) 128 | .config("spark.sql.catalog.glue_catalog.io-impl", "org.apache.iceberg.aws.s3.S3FileIO") 129 | .config("spark.sql.shuffle.partitions", "100") // as we are not using AQE then we need to tune this 130 | .config("spark.sql.defaultCatalog", "glue_catalog") 131 | .getOrCreate(); 132 | 133 | }else if (args.length == 6) { 134 | removeDuplicates = Boolean.parseBoolean(args[0]); 135 | icebergWarehouse = args[1]; 136 | protoDescFile = args[2]; 137 | checkpointDir = args[3]; 138 | bootstrapServers = args[4]; 139 | compactionEnabled = Boolean.parseBoolean(args[5]); 140 | log.warn( 141 | "Master will be inferred from the environment Iceberg Glue catalog will be used, with the warehouse being: {} \n " 142 | + "removing duplicates within the watermark is {}, the descriptor file is at: {} and the checkpoint is at: {}\n " 143 | + "Kafka bootstrap is: {}, compactions on each 'n' batch are {}", 144 | removeDuplicates,icebergWarehouse, protoDescFile, checkpointDir, bootstrapServers, compactionEnabled); 145 | spark = 146 | SparkSession.builder() 147 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 148 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 149 | .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 150 | .config("spark.sql.catalog.glue_catalog.catalog-impl", "org.apache.iceberg.aws.glue.GlueCatalog") 151 | .config("spark.sql.catalog.glue_catalog", "org.apache.iceberg.spark.SparkCatalog") 152 | .config("spark.sql.catalog.glue_catalog.warehouse", icebergWarehouse) 153 | .config("spark.hadoop.fs.s3.impl","org.apache.hadoop.fs.s3a.S3AFileSystem") 154 | .config("spark.sql.iceberg.data-prefetch.enabled","true") 155 | .config("spark.sql.catalog.glue_catalog.io-impl", "org.apache.iceberg.aws.s3.S3FileIO") 156 | .config("spark.sql.shuffle.partitions", "100") // as we are not using AQE then we need to tune this 157 | .config("spark.sql.defaultCatalog", "glue_catalog") 158 | 159 | .getOrCreate(); 160 | } else { 161 | spark = null; 162 | log.error( 163 | "Invalid number of arguments provided, please check the readme for the correct usage"); 164 | System.exit(1); 165 | } 166 | 167 | spark.sql( 168 | """ 169 | CREATE DATABASE IF NOT EXISTS bigdata; 170 | """); 171 | 172 | spark.sql( 173 | """ 174 | USE bigdata; 175 | """); 176 | 177 | spark.sql( 178 | """ 179 | CREATE TABLE IF NOT EXISTS employee 180 | (employee_id bigint, 181 | age int, 182 | start_date timestamp, 183 | team string, 184 | role string, 185 | address string, 186 | name string 187 | ) 188 | PARTITIONED BY (bucket(8, employee_id), hours(start_date), team) 189 | TBLPROPERTIES ( 190 | 'table_type'='ICEBERG', 191 | 'write.parquet.compression-level'='7', 192 | 'format'='parquet', 193 | 'commit.retry.num-retries'='10', --Number of times to retry a commit before failing 194 | 'commit.retry.min-wait-ms'='250', --Minimum time in milliseconds to wait before retrying a commit 195 | 'commit.retry.max-wait-ms'='60000', -- (1 min) Maximum time in milliseconds to wait before retrying a commit 196 | 'write.parquet.compression-codec'='zstd', 197 | -- if you have a huge number of columns remember to tune dict-size and page-size 198 | 'compatibility.snapshot-id-inheritance.enabled'='true' ); 199 | """); 200 | 201 | 202 | Dataset df = 203 | spark 204 | .readStream() 205 | .format("kafka") 206 | .option("kafka.bootstrap.servers", bootstrapServers) 207 | .option("subscribe", "protobuf-demo-topic-pure") 208 | .load(); 209 | 210 | Dataset output = 211 | df.select(from_protobuf(col("value"),"Employee", HexFormat.of().parseHex(hexData)).as("Employee")) 212 | .select(col("Employee.*")) 213 | .select( 214 | col("id").as("employee_id"), 215 | col("employee_age.value").as("age"), 216 | col("start_date"), 217 | col("team.name").as("team"), 218 | col("role"), 219 | col("address"), 220 | col("name")); 221 | 222 | StreamingQuery query = 223 | output 224 | .writeStream() 225 | .queryName("streaming-protobuf-ingest") 226 | .format("iceberg") 227 | .outputMode("append") 228 | .foreachBatch( 229 | (VoidFunction2, Long>) 230 | // here we want to make normal "commits" and then for each 10 trigger run 231 | // compactions! 232 | (dataframe, batchId) -> { 233 | log.warn("Writing batch {}", batchId); 234 | if (removeDuplicates) { 235 | // first we want to filter affected partitions for filtering 236 | var partitions = 237 | dataframe 238 | .select(col("start_date"), col("team"), col("employee_id")) 239 | .withColumn("day", date_trunc("day", col("start_date"))) 240 | .withColumn("hour", date_trunc("hour", col("start_date"))) 241 | .select(col("employee_id"), col("day"), col("hour"), col("team")) 242 | .dropDuplicates(); 243 | var listPartitions = partitions.collectAsList(); 244 | log.warn("Affected partitions: {}", partitions.count()); 245 | // partitions= 246 | // .dropDuplicates() 247 | // .collectAsList(); 248 | 249 | log.warn("Partitions to merge: {}", partitions); 250 | } else { 251 | dataframe.write().insertInto("bigdata.employee"); 252 | } 253 | if (compactionEnabled) { 254 | // the main idea behind this is in cases where you may have receiving "late data randomly and 255 | // doing the compaction jobs with optimistic concurrency will lead into a 256 | // lot of conflicts where you could increase the number of retries ( as we are using partial 257 | // progress we need to increase the commit retries though), or you can just use this 258 | // strategy for compaction, older partitions on each N batches. 259 | if (batchId % 10 == 0) { 260 | log.warn("\nCompaction in progress:\n"); 261 | spark 262 | .sql( 263 | """ 264 | CALL system.rewrite_data_files( 265 | table => 'employee', 266 | strategy => 'sort', 267 | sort_order => 'start_date', 268 | where => 'start_date >= current_timestamp() - INTERVAL 1 HOURS', -- this sql needs to be adapted to only compact older partitions 269 | options => map( 270 | 'rewrite-job-order','bytes-asc', 271 | 'target-file-size-bytes','273741824', 272 | 'max-file-group-size-bytes','10737418240', 273 | 'partial-progress.enabled', 'true', 274 | 'max-concurrent-file-group-rewrites', '10000', 275 | 'partial-progress.max-commits', '10' 276 | )) 277 | """) 278 | .show(); 279 | } 280 | // rewrite manifests from time to time 281 | log.warn("\nManifest compaction in progress:\n"); 282 | if (batchId % 30 == 0) { 283 | spark 284 | .sql( 285 | """ 286 | CALL system.rewrite_manifests( 287 | table => 'employee' 288 | ) 289 | """) 290 | .show(); 291 | } 292 | 293 | // old snapshots expiration can be done in another job for older partitions. 294 | } 295 | }) 296 | .trigger(Trigger.ProcessingTime(1, TimeUnit.MINUTES)) 297 | .option("fanout-enabled", "true") // disable ordering 298 | .option("checkpointLocation", checkpointDir) // you should enable this on production 299 | .start(); 300 | query.awaitTermination(); 301 | } 302 | } 303 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/spark/iot/SparkCustomIcebergIngest.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.spark.iot; 2 | 3 | import static org.apache.spark.sql.functions.*; 4 | import static org.apache.spark.sql.protobuf.functions.*; 5 | 6 | import java.io.IOException; 7 | import java.util.ArrayList; 8 | import java.util.HexFormat; 9 | import java.util.concurrent.TimeUnit; 10 | import java.util.concurrent.TimeoutException; 11 | 12 | import com.aws.emr.avro.kafka.SparkNativeIcebergIngestAvro; 13 | import org.apache.logging.log4j.LogManager; 14 | import org.apache.logging.log4j.Logger; 15 | import org.apache.spark.api.java.function.VoidFunction2; 16 | import org.apache.spark.sql.*; 17 | import org.apache.spark.sql.streaming.StreamingQuery; 18 | import org.apache.spark.sql.streaming.StreamingQueryException; 19 | import org.apache.spark.sql.streaming.Trigger; 20 | 21 | /** 22 | * 23 | * An example of consuming messages from Kafka using Protocol Buffers and writing them to Iceberg using the native 24 | * data source and writing via custom Spark/Iceberg writing mechanism 25 | * 26 | * This implements all the features and mechanisms that we want to be demostrated. 27 | * 28 | * Watermark deduplication 29 | * Compaction 30 | * MERGE INTO Deduplication 31 | * 32 | * @author acmanjon@amazon.com 33 | * 34 | */ 35 | 36 | public class SparkCustomIcebergIngest { 37 | 38 | private static final Logger log = LogManager.getLogger(SparkCustomIcebergIngest.class); 39 | private static String master = ""; 40 | private static boolean removeDuplicates = true; 41 | private static String protoDescFile = "Employee.desc"; 42 | private static String icebergWarehouse = "warehouse/"; 43 | private static String checkpointDir = "tmp/"; 44 | private static String bootstrapServers = "localhost:9092"; 45 | private static boolean compactionEnabled = false; 46 | 47 | 48 | public static void main(String[] args) 49 | throws IOException, TimeoutException, StreamingQueryException { 50 | 51 | SparkSession spark; 52 | //default local env. 53 | if (args.length < 1) { 54 | master = "local[*]"; 55 | log.warn( 56 | "No arguments provided, running using local default settings: master={} and Iceberg hadoop based file catalog ", 57 | master); 58 | log.warn( 59 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 60 | + " this mode is for local based execution and development. Kafka broker in this case will also be 'localhost:9092'." 61 | + " Remember to clean the checkpoint dir for any changes or if you want to start 'clean'"); 62 | spark = 63 | SparkSession.builder() 64 | .master(master) 65 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 66 | .config( "spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 67 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 68 | .config("spark.sql.catalog.spark_catalog.type", "hive") 69 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 70 | .config("spark.sql.catalog.local.type", "hadoop") 71 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 72 | .config("spark.sql.catalog.local.warehouse", "warehouse") 73 | .config("spark.sql.defaultCatalog", "local") 74 | /** 75 | //enable SPJ 76 | .config("spark.sql.sources.v2.bucketing.enabled","true") 77 | .config("spark.sql.sources.v2.bucketing.pushPartValues.enabled","true") 78 | .config("spark.sql.requireAllClusterKeysForCoPartition","false") 79 | .config("spark.sql.sources.v2.bucketing.partiallyClusteredDistribution.enabled","true") 80 | .config("spark.sql.sources.v2.bucketing.pushPartKeys.enabled","true") 81 | .config("spark.sql.iceberg.planning.preserve-data-grouping","true") 82 | .config("spark.sql.sources.v2.bucketing.allowJoinKeysSubsetOfPartitionKeys.enabled","false") 83 | .config("spark.sql.optimizer.runtime.rowLevelOperationGroupFilter.enabled","false") 84 | // enable shuffle hash join 85 | .config("spark.sql.join.preferSortMergeJoin","false") 86 | .config("spark.sql.shuffledHashJoinFactor","1") 87 | //set none to distribution mode 88 | .config("spark.sql.iceberg.distribution-mode","none") 89 | //disable adaptative 90 | .config("spark.sql.adaptive.coalescePartitions.enabled","false") 91 | .config("spark.sql.adaptive.skewJoin.enabled","false") 92 | .config("spark.sql.adaptive.enabled","false")**/ 93 | 94 | .getOrCreate(); 95 | //local env with optional compaction and duplicate removal 96 | } else if (args.length == 2) { 97 | removeDuplicates = Boolean.parseBoolean(args[0]); 98 | compactionEnabled = Boolean.parseBoolean(args[1]); 99 | master = "local[*]"; 100 | log.warn( 101 | "Running with local master: {} and Iceberg hadoop based file catalog " 102 | + "removing duplicates within the watermark is {}, compactions each 'n' batch are {}", 103 | master, 104 | removeDuplicates, 105 | compactionEnabled); 106 | log.warn( 107 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 108 | + " this mode is for local based execution. Kafka broker in this case will also be 'localhost:9092'."); 109 | spark = 110 | SparkSession.builder() 111 | .master(master) 112 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 113 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 114 | .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 115 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 116 | .config("spark.sql.catalog.spark_catalog.type", "hive") 117 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 118 | .config("spark.sql.catalog.local.type", "hadoop") 119 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this for the size of our cluster/tasks 120 | // remember that we should be ideally at 200MB per task minimum 121 | .config("spark.sql.catalog.local.warehouse", "warehouse") 122 | .config("spark.sql.defaultCatalog", "local") 123 | .getOrCreate(); 124 | //local env connected to Glue catalog 125 | } else if ( args.length == 4){ 126 | removeDuplicates = Boolean.parseBoolean(args[0]); 127 | compactionEnabled = Boolean.parseBoolean(args[1]); 128 | icebergWarehouse = args[2]; 129 | checkpointDir = args[3]; 130 | master = "local[*]"; 131 | log.warn( 132 | "Running with local master and Iceberg hadoop based file catalog " 133 | + "removing duplicates within the watermark is {}, compactions each 'n' batch are {}", 134 | 135 | removeDuplicates, 136 | compactionEnabled); 137 | log.warn( 138 | "Iceberg warehouse dir will be {} from the run dir and the checkpoint directory will be {}'\n" 139 | + " this mode is for local based execution. Kafka broker in this case will also be 'localhost:9092'.", 140 | icebergWarehouse,checkpointDir); 141 | spark = 142 | SparkSession.builder() 143 | .master(master) 144 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 145 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 146 | .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 147 | .config("spark.sql.catalog.glue_catalog.catalog-impl", "org.apache.iceberg.aws.glue.GlueCatalog") 148 | .config("spark.sql.catalog.glue_catalog", "org.apache.iceberg.spark.SparkCatalog") 149 | .config("spark.sql.catalog.glue_catalog.warehouse", icebergWarehouse) 150 | .config("spark.sql.catalog.glue_catalog.io-impl", "org.apache.iceberg.aws.s3.S3FileIO") 151 | .config("spark.sql.shuffle.partitions", "100") // as we are not using AQE then we need to tune this 152 | .config("spark.sql.defaultCatalog", "glue_catalog") 153 | .getOrCreate(); 154 | 155 | }else if (args.length == 6) { 156 | removeDuplicates = Boolean.parseBoolean(args[0]); 157 | icebergWarehouse = args[1]; 158 | protoDescFile = args[2]; 159 | checkpointDir = args[3]; 160 | bootstrapServers = args[4]; 161 | compactionEnabled = Boolean.parseBoolean(args[5]); 162 | log.warn( 163 | "Master will be inferred from the environment Iceberg Glue catalog will be used, with the warehouse being: {} \n " 164 | + "removing duplicates within the watermark is {}, the descriptor file is at: {} and the checkpoint is at: {}\n " 165 | + "Kafka bootstrap is: {}, compactions on each 'n' batch are {}", 166 | removeDuplicates,icebergWarehouse, protoDescFile, checkpointDir, bootstrapServers, compactionEnabled); 167 | spark = 168 | SparkSession.builder() 169 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 170 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 171 | .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 172 | .config("spark.sql.catalog.glue_catalog.catalog-impl", "org.apache.iceberg.aws.glue.GlueCatalog") 173 | .config("spark.sql.catalog.glue_catalog", "org.apache.iceberg.spark.SparkCatalog") 174 | .config("spark.sql.shuffle.partitions", "100") // as we are not using AQE then we need to tune this 175 | .config("spark.sql.iceberg.data-prefetch.enabled","true") 176 | .config("spark.hadoop.fs.s3.impl","org.apache.hadoop.fs.s3a.S3AFileSystem") 177 | .config("spark.sql.catalog.glue_catalog.io-impl", "org.apache.iceberg.aws.s3.S3FileIO") 178 | .config("spark.sql.shuffle.partitions", "100") // as we are not using AQE then we need to tune this 179 | .config("spark.sql.defaultCatalog", "glue_catalog") 180 | 181 | .getOrCreate(); 182 | } else { 183 | spark = null; 184 | log.error( 185 | "Invalid number of arguments provided, please check the readme for the correct usage"); 186 | System.exit(1); 187 | } 188 | spark.sql(""" 189 | CREATE DATABASE IF NOT EXISTS bigdata; 190 | """); 191 | 192 | 193 | spark.sql(""" 194 | USE bigdata; 195 | """); 196 | spark.sql( 197 | """ 198 | CREATE TABLE IF NOT EXISTS employee 199 | (employee_id bigint, 200 | age int, 201 | start_date timestamp, 202 | team string, 203 | role string, 204 | address string, 205 | name string 206 | ) 207 | PARTITIONED BY (bucket(8, employee_id), hours(start_date), team) 208 | TBLPROPERTIES ( 209 | 'table_type'='ICEBERG', 210 | 'write.parquet.compression-level'='7', 211 | 'format'='parquet', 212 | 'commit.retry.num-retries'='10', --Number of times to retry a commit before failing 213 | 'commit.retry.min-wait-ms'='250', --Minimum time in milliseconds to wait before retrying a commit 214 | 'commit.retry.max-wait-ms'='60000', -- (1 min) Maximum time in milliseconds to wait before retrying a commit 215 | 'write.parquet.compression-codec'='zstd', 216 | -- if you have a huge number of columns remember to tune dict-size and page-size 217 | 'compatibility.snapshot-id-inheritance.enabled'='true' ); 218 | """); 219 | 220 | 221 | Dataset df = 222 | spark 223 | .readStream() 224 | .format("kafka") 225 | .option("kafka.bootstrap.servers", bootstrapServers) 226 | .option("subscribe", "protobuf-demo-topic-pure") 227 | .load(); 228 | 229 | Dataset output = 230 | df.select(from_protobuf(col("value"),"Employee", protoDescFile).as("Employee")) 231 | .select(col("Employee.*")) 232 | .select( 233 | col("id").as("employee_id"), 234 | col("employee_age.value").as("age"), 235 | col("start_date"), 236 | col("team.name").as("team"), 237 | col("role"), 238 | col("address"), 239 | col("name")); 240 | 241 | StreamingQuery query = 242 | output 243 | .writeStream() 244 | .queryName("streaming-protobuf-ingest") 245 | .format("iceberg") 246 | .outputMode("append") 247 | .foreachBatch( 248 | (VoidFunction2, Long>) 249 | // here we want to make normal "commits" and then for each 10 trigger run 250 | // compactions! 251 | (dataframe, batchId) -> { 252 | var session=dataframe.sparkSession(); 253 | log.warn("Writing batch {}", batchId); 254 | if (removeDuplicates) { 255 | dataframe.createOrReplaceTempView("insert_data"); 256 | // here we are pushing some filters like the team and the date (we know that 257 | // we will have late events from hour ago.... 258 | // we could improve this filtering by bucket and just merge data from that 259 | // bucket ( using 8 merge queries), one per bucket. Iceberg bucketing can be calculated via 260 | // 'system.bucket(8,employee_id)' 261 | // t.employee_id in (1,2,3,...) or t.employee_id in (7,8,9,....) 262 | // in each 'in' you can put 1000 values. 263 | // another way is to generate a column for the bucket and then make the join/ON there 264 | // this one maybe be easier instead of generate that long in(1,3,4,5,6....) list, 265 | // the problem is that you wouldn't able to use INSERT * 266 | // another thing to test storage-partitioned joins but from streaming sources the performance gains... 267 | // should be tested on cluster, on local laptop mode they hurt, already tested 268 | String merge = 269 | """ 270 | MERGE INTO bigdata.employee as t 271 | USING insert_data as s 272 | ON `s`.`employee_id`=`t`.`employee_id` AND `t`.`start_date` > current_timestamp() - INTERVAL 1 HOURS 273 | AND `t`.`team`='Solutions Architects' AND `t`.`start_date`=`s`.`start_date` 274 | WHEN NOT MATCHED THEN INSERT * 275 | """; 276 | session.sql((merge)); 277 | } else { 278 | dataframe.write().insertInto("bigdata.employee"); 279 | } 280 | if (compactionEnabled) { 281 | // the main idea behind this is in cases where you may have receiving "late 282 | // data randomly and 283 | // doing the compaction jobs with optimistic concurrency will lead into a 284 | // lot of conflicts where you could increase the number of retries ( as we 285 | // are using partial 286 | // progress we need to increase the commit retries though), or you can just 287 | // use this 288 | // strategy for compaction, older partitions on each N batches. 289 | if (batchId % 10 == 0) { 290 | log.warn("\nCompaction in progress:\n"); 291 | spark 292 | .sql( 293 | """ 294 | CALL system.rewrite_data_files( 295 | table => 'employee', 296 | strategy => 'sort', 297 | sort_order => 'start_date', 298 | where => 'start_date >= current_timestamp() - INTERVAL 1 HOURS', -- this sql needs to be adapted to only compact older partitions 299 | options => map( 300 | 'rewrite-job-order','bytes-asc', 301 | 'target-file-size-bytes','273741824', 302 | 'max-file-group-size-bytes','10737418240', 303 | 'partial-progress.enabled', 'true', 304 | 'max-concurrent-file-group-rewrites', '10000', 305 | 'partial-progress.max-commits', '10' 306 | )) 307 | """) 308 | .show(); 309 | } 310 | // rewrite manifests from time to time 311 | log.warn("\nManifest compaction in progress:\n"); 312 | if (batchId % 30 == 0) { 313 | spark 314 | .sql( 315 | """ 316 | CALL system.rewrite_manifests( 317 | table => 'employee' 318 | ) 319 | """) 320 | .show(); 321 | } 322 | 323 | // old snapshots expiration can be done in another job for older partitions. 324 | } 325 | }) 326 | .trigger(Trigger.ProcessingTime(5, TimeUnit.MINUTES)) 327 | .option("fanout-enabled", "true") // disable ordering 328 | .option("checkpointLocation", checkpointDir) // on local mode connected to glue disable it or add hadoop-aws library to add S3 file api . 329 | .start(); 330 | query.awaitTermination(); 331 | } 332 | } 333 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/spark/iot/SparkCustomIcebergIngestMoR.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.spark.iot; 2 | 3 | import static org.apache.spark.sql.functions.*; 4 | import static org.apache.spark.sql.protobuf.functions.*; 5 | 6 | import java.io.IOException; 7 | import java.time.LocalDateTime; 8 | import java.time.temporal.ChronoUnit; 9 | import java.util.concurrent.Executors; 10 | import java.util.concurrent.ScheduledExecutorService; 11 | import java.util.concurrent.TimeUnit; 12 | import java.util.concurrent.TimeoutException; 13 | 14 | import org.apache.logging.log4j.LogManager; 15 | import org.apache.logging.log4j.Logger; 16 | import org.apache.spark.api.java.function.VoidFunction2; 17 | import org.apache.spark.sql.*; 18 | import org.apache.spark.sql.streaming.StreamingQuery; 19 | import org.apache.spark.sql.streaming.StreamingQueryException; 20 | import org.apache.spark.sql.streaming.Trigger; 21 | 22 | /** 23 | * 24 | * An example of consuming messages from Kafka using Protocol Buffers and writing them to Iceberg using the native 25 | * data source and writing via custom Spark/Iceberg writing mechanism 26 | * 27 | * This implements all the features and mechanisms that we want to be demostrated. 28 | * 29 | * Watermark deduplication 30 | * Compaction 31 | * MERGE INTO Deduplication 32 | * 33 | * @author acmanjon@amazon.com 34 | * 35 | */ 36 | 37 | public class SparkCustomIcebergIngestMoR { 38 | 39 | private static final Logger log = LogManager.getLogger(SparkCustomIcebergIngestMoR.class); 40 | private static String master = ""; 41 | private static boolean removeDuplicates = true; 42 | private static String protoDescFile = "Employee.desc"; 43 | private static String icebergWarehouse = "warehouse/"; 44 | private static String checkpointDir = "tmp/"; 45 | private static String bootstrapServers = "localhost:9092"; 46 | private static boolean compactionEnabled = false; 47 | 48 | 49 | public static void main(String[] args) 50 | throws IOException, TimeoutException, StreamingQueryException { 51 | 52 | SparkSession spark; 53 | //default local env. 54 | if (args.length < 1) { 55 | master = "local[*]"; 56 | log.warn( 57 | "No arguments provided, running using local default settings: master={} and Iceberg hadoop based file catalog ", 58 | master); 59 | log.warn( 60 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 61 | + " this mode is for local based execution and development. Kafka broker in this case will also be 'localhost:9092'." 62 | + " Remember to clean the checkpoint dir for any changes or if you want to start 'clean'"); 63 | spark = 64 | SparkSession.builder() 65 | .master(master) 66 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 67 | .config( "spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 68 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 69 | .config("spark.sql.catalog.spark_catalog.type", "hive") 70 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 71 | .config("spark.sql.catalog.local.type", "hadoop") 72 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 73 | .config("spark.sql.catalog.local.warehouse", "warehouse") 74 | .config("spark.sql.defaultCatalog", "local") 75 | /** 76 | //enable SPJ 77 | .config("spark.sql.sources.v2.bucketing.enabled","true") 78 | .config("spark.sql.sources.v2.bucketing.pushPartValues.enabled","true") 79 | .config("spark.sql.requireAllClusterKeysForCoPartition","false") 80 | .config("spark.sql.sources.v2.bucketing.partiallyClusteredDistribution.enabled","true") 81 | .config("spark.sql.sources.v2.bucketing.pushPartKeys.enabled","true") 82 | .config("spark.sql.iceberg.planning.preserve-data-grouping","true") 83 | .config("spark.sql.sources.v2.bucketing.allowJoinKeysSubsetOfPartitionKeys.enabled","false") 84 | .config("spark.sql.optimizer.runtime.rowLevelOperationGroupFilter.enabled","false") 85 | // enable shuffle hash join 86 | .config("spark.sql.join.preferSortMergeJoin","false") 87 | .config("spark.sql.shuffledHashJoinFactor","1") 88 | //set none to distribution mode 89 | .config("spark.sql.iceberg.distribution-mode","none") 90 | //disable adaptative 91 | .config("spark.sql.adaptive.coalescePartitions.enabled","false") 92 | .config("spark.sql.adaptive.skewJoin.enabled","false") 93 | .config("spark.sql.adaptive.enabled","false")**/ 94 | 95 | .getOrCreate(); 96 | //local env with optional compaction and duplicate removal 97 | } else if (args.length == 2) { 98 | removeDuplicates = Boolean.parseBoolean(args[0]); 99 | compactionEnabled = Boolean.parseBoolean(args[1]); 100 | master = "local[*]"; 101 | log.warn( 102 | "Running with local master: {} and Iceberg hadoop based file catalog " 103 | + "removing duplicates within the watermark is {}, compactions each 'n' batch are {}", 104 | master, 105 | removeDuplicates, 106 | compactionEnabled); 107 | log.warn( 108 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 109 | + " this mode is for local based execution. Kafka broker in this case will also be 'localhost:9092'."); 110 | spark = 111 | SparkSession.builder() 112 | .master(master) 113 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 114 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 115 | .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 116 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 117 | .config("spark.sql.catalog.spark_catalog.type", "hive") 118 | .config("spark.hadoop.fs.s3.impl","org.apache.hadoop.fs.s3a.S3AFileSystem") 119 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 120 | .config("spark.sql.catalog.local.type", "hadoop") 121 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this for the size of our cluster/tasks 122 | // remember that we should be ideally at 200MB per task minimum 123 | .config("spark.sql.catalog.local.warehouse", "warehouse") 124 | .config("spark.sql.defaultCatalog", "local") 125 | .getOrCreate(); 126 | //local env connected to Glue catalog 127 | } else if ( args.length == 4){ 128 | removeDuplicates = Boolean.parseBoolean(args[0]); 129 | compactionEnabled = Boolean.parseBoolean(args[1]); 130 | icebergWarehouse = args[2]; 131 | checkpointDir = args[3]; 132 | master = "local[*]"; 133 | log.warn( 134 | "Running with local master and Iceberg hadoop based file catalog " 135 | + "removing duplicates within the watermark is {}, compactions each 'n' batch are {}", 136 | 137 | removeDuplicates, 138 | compactionEnabled); 139 | log.warn( 140 | "Iceberg warehouse dir will be {} from the run dir and the checkpoint directory will be {}'\n" 141 | + " this mode is for local based execution. Kafka broker in this case will also be 'localhost:9092'.", 142 | icebergWarehouse,checkpointDir); 143 | spark = 144 | SparkSession.builder() 145 | .master(master) 146 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 147 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 148 | .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 149 | .config("spark.sql.catalog.glue_catalog.catalog-impl", "org.apache.iceberg.aws.glue.GlueCatalog") 150 | .config("spark.sql.catalog.glue_catalog", "org.apache.iceberg.spark.SparkCatalog") 151 | .config("spark.sql.catalog.glue_catalog.warehouse", icebergWarehouse) 152 | .config("spark.hadoop.fs.s3.impl","org.apache.hadoop.fs.s3a.S3AFileSystem") 153 | .config("spark.sql.catalog.glue_catalog.io-impl", "org.apache.iceberg.aws.s3.S3FileIO") 154 | .config("spark.sql.shuffle.partitions", "100") // as we are not using AQE then we need to tune this 155 | .config("spark.sql.defaultCatalog", "glue_catalog") 156 | .getOrCreate(); 157 | 158 | }else if (args.length == 6) { 159 | removeDuplicates = Boolean.parseBoolean(args[0]); 160 | icebergWarehouse = args[1]; 161 | protoDescFile = args[2]; 162 | checkpointDir = args[3]; 163 | bootstrapServers = args[4]; 164 | compactionEnabled = Boolean.parseBoolean(args[5]); 165 | log.warn( 166 | "Master will be inferred from the environment Iceberg Glue catalog will be used, with the warehouse being: {} \n " 167 | + "removing duplicates within the watermark is {}, the descriptor file is at: {} and the checkpoint is at: {}\n " 168 | + "Kafka bootstrap is: {}, compactions on each 'n' batch are {}", 169 | removeDuplicates,icebergWarehouse, protoDescFile, checkpointDir, bootstrapServers, compactionEnabled); 170 | spark = 171 | SparkSession.builder() 172 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 173 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 174 | .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 175 | .config("spark.sql.catalog.glue_catalog.catalog-impl", "org.apache.iceberg.aws.glue.GlueCatalog") 176 | .config("spark.sql.catalog.glue_catalog", "org.apache.iceberg.spark.SparkCatalog") 177 | .config("spark.sql.catalog.glue_catalog.warehouse", icebergWarehouse) 178 | .config("spark.sql.catalog.glue_catalog.io-impl", "org.apache.iceberg.aws.s3.S3FileIO") 179 | .config("spark.sql.iceberg.data-prefetch.enabled","true") 180 | .config("spark.hadoop.fs.s3.impl","org.apache.hadoop.fs.s3a.S3AFileSystem") 181 | .config("spark.sql.shuffle.partitions", "100") // as we are not using AQE then we need to tune this 182 | .config("spark.sql.defaultCatalog", "glue_catalog") 183 | 184 | .getOrCreate(); 185 | } else { 186 | spark = null; 187 | log.error( 188 | "Invalid number of arguments provided, please check the readme for the correct usage"); 189 | System.exit(1); 190 | } 191 | spark.sql(""" 192 | CREATE DATABASE IF NOT EXISTS bigdata; 193 | """); 194 | 195 | 196 | spark.sql(""" 197 | USE bigdata; 198 | """); 199 | spark.sql( 200 | """ 201 | CREATE TABLE IF NOT EXISTS employee 202 | (employee_id bigint, 203 | age int, 204 | start_date timestamp, 205 | team string, 206 | role string, 207 | address string, 208 | name string 209 | ) 210 | PARTITIONED BY (hours(start_date), team, bucket(42, employee_id)) 211 | TBLPROPERTIES ( 212 | 'table_type'='ICEBERG', 213 | 'write.parquet.compression-level'='7', 214 | 'format'='parquet', 215 | 'write.delete.mode'='copy-on-write', 216 | 'write.update.mode'='merge-on-read', 217 | 'write.merge.mode'='merge-on-read', 218 | 'write.parquet.row-group-size-bytes' = '134217728', -- 128MB 219 | 'write.parquet.page-size-bytes' = '1048576', -- 2MB 220 | 'write.target-file-size-bytes' = '536870912', -- 256MB 221 | 'write.distribution-mode' = 'hash', 222 | 'write.delete.distribution-mode' = 'none', 223 | 'write.update.distribution-mode' = 'none', 224 | 'write.merge.distribution-mode' = 'none', 225 | 'write.spark.fanout.enabled' = 'true', 226 | 'write.metadata.delete-after-commit.enabled' = 'false', 227 | 'write.metadata.previous-versions-max' = '50', 228 | 'history.expire.max-snapshot-age-ms' = '259200000', -- 3 days 229 | 'commit.retry.num-retries'='20', --Number of times to retry a commit before failing 230 | 'commit.retry.min-wait-ms'='250', --Minimum time in milliseconds to wait before retrying a commit 231 | 'commit.retry.max-wait-ms'='60000', -- (1 min) Maximum time in milliseconds to wait before retrying a commit 232 | 'write.parquet.compression-codec'='zstd', 233 | -- if you have a huge number of columns remember to tune dict-size and page-size 234 | 'compatibility.snapshot-id-inheritance.enabled'='true' ); 235 | 236 | """); 237 | 238 | Dataset df = 239 | spark 240 | .readStream() 241 | .format("kafka") 242 | .option("kafka.bootstrap.servers", bootstrapServers) 243 | .option("subscribe", "protobuf-demo-topic-pure") 244 | .load(); 245 | 246 | Dataset output = 247 | df.select(from_protobuf(col("value"),"Employee", protoDescFile).as("Employee")) 248 | .select(col("Employee.*")) 249 | .select( 250 | col("id").as("employee_id"), 251 | col("employee_age.value").as("age"), 252 | col("start_date"), 253 | col("team.name").as("team"), 254 | col("role"), 255 | col("address"), 256 | col("name")); 257 | 258 | StreamingQuery query = 259 | output 260 | .writeStream() 261 | .queryName("streaming-protobuf-ingest") 262 | .format("iceberg") 263 | .outputMode("append") 264 | .foreachBatch( 265 | (VoidFunction2, Long>) 266 | // here we want to make normal "commits" and then for each 10 trigger run 267 | // compactions! 268 | (dataframe, batchId) -> { 269 | var session=dataframe.sparkSession(); 270 | log.warn("Writing batch {}", batchId); 271 | if (removeDuplicates) { 272 | dataframe.createOrReplaceTempView("insert_data"); 273 | // here we are pushing some filters like the team and the date (we know that 274 | // we will have late events from hour ago.... 275 | // we could improve this filtering by bucket and just merge data from that 276 | // bucket ( using 8 merge queries), one per bucket. Iceberg bucketing can be calculated via 277 | // 'system.bucket(8,employee_id)' 278 | // t.employee_id in (1,2,3,...) or t.employee_id in (7,8,9,....) 279 | // in each 'in' you can put 1000 values. 280 | // another way is to generate a column for the bucket and then make the join/ON there 281 | // this one maybe be easier instead of generate that long in(1,3,4,5,6....) list, 282 | // the problem is that you wouldn't able to use INSERT * 283 | // another thing to test storage-partitioned joins but from streaming sources the performance gains... 284 | // should be tested on cluster, on local laptop mode they hurt, already tested 285 | String merge = 286 | """ 287 | MERGE INTO bigdata.employee as t 288 | USING insert_data as s 289 | ON `s`.`employee_id`=`t`.`employee_id` AND `t`.`start_date` > current_timestamp() - INTERVAL 1 HOURS 290 | AND `t`.`team`='Solutions Architects' AND `t`.`start_date`=`s`.`start_date` 291 | WHEN NOT MATCHED THEN INSERT * 292 | """; 293 | session.sql((merge)); 294 | } else { 295 | dataframe.write().insertInto("bigdata.employee"); 296 | } 297 | }) 298 | .trigger(Trigger.ProcessingTime(1, TimeUnit.MINUTES)) 299 | .option("fanout-enabled", "true") // disable ordering 300 | .option("checkpointLocation", checkpointDir) // on local mode connected to glue disable it or add hadoop-aws library to add S3 file api . 301 | .start(); 302 | 303 | if (compactionEnabled) { 304 | ScheduledExecutorService scheduledExecutor = Executors.newSingleThreadScheduledExecutor(); 305 | scheduledExecutor.scheduleAtFixedRate(new Compact(spark), millisToNextHour(), 60*60*1000, TimeUnit.MILLISECONDS); 306 | } 307 | 308 | query.awaitTermination(); 309 | } 310 | 311 | private static long millisToNextHour() { 312 | // we wait 5 minutes to start the compaction process for previous partition 313 | LocalDateTime nextHour = LocalDateTime.now().plusHours(1).truncatedTo(ChronoUnit.HOURS).plusMinutes(5); 314 | return LocalDateTime.now().until(nextHour, ChronoUnit.MILLIS); 315 | } 316 | 317 | 318 | private static class Compact implements Runnable { 319 | private final SparkSession spark; 320 | public Compact(SparkSession spark) { 321 | this.spark = spark; 322 | } 323 | 324 | @Override 325 | public void run() { 326 | // the main idea behind this is in cases where you may have receiving "late 327 | // data randomly and 328 | // doing the compaction jobs with optimistic concurrency will lead into a 329 | // lot of conflicts where you could increase the number of retries ( as we 330 | // are using partial 331 | // progress we need to increase the commit retries though), or you can just 332 | // use this 333 | // strategy for compaction, older partitions on each N batches. 334 | log.warn("\nCompaction in progress:\n"); 335 | spark 336 | .sql( 337 | """ 338 | CALL system.rewrite_data_files( 339 | table => 'employee', 340 | strategy => 'sort', 341 | sort_order => 'start_date', 342 | where => 'start_date >= current_timestamp() - INTERVAL 1 HOURS', -- this sql needs to be adapted to only compact older partitions 343 | options => map( 344 | 'rewrite-job-order','bytes-asc', 345 | 'target-file-size-bytes','273741824', 346 | 'max-file-group-size-bytes','10737418240', 347 | 'partial-progress.enabled', 'true', 348 | 'max-concurrent-file-group-rewrites', '1000', 349 | 'partial-progress.max-commits', '10' 350 | )) 351 | """) 352 | .show(); 353 | // rewrite manifests from time to time 354 | log.warn("\nManifest compaction in progress:\n"); 355 | spark 356 | .sql( 357 | """ 358 | CALL system.rewrite_manifests( 359 | table => 'employee' 360 | ) 361 | """) 362 | .show(); 363 | } 364 | 365 | // old snapshots expiration can be done in another job for older partitions. 366 | } 367 | } 368 | 369 | -------------------------------------------------------------------------------- /src/main/java/com/aws/emr/spark/iot/SparkCustomIcebergIngestMoRS3BucketsAutoAvro.java: -------------------------------------------------------------------------------- 1 | package com.aws.emr.spark.iot; 2 | 3 | import static org.apache.spark.sql.functions.*; 4 | import static org.apache.spark.sql.protobuf.functions.*; 5 | 6 | import java.io.IOException; 7 | import java.time.LocalDateTime; 8 | import java.time.temporal.ChronoUnit; 9 | import java.util.concurrent.Executors; 10 | import java.util.concurrent.ScheduledExecutorService; 11 | import java.util.concurrent.TimeUnit; 12 | import java.util.concurrent.TimeoutException; 13 | import org.apache.logging.log4j.LogManager; 14 | import org.apache.logging.log4j.Logger; 15 | import org.apache.spark.api.java.function.VoidFunction2; 16 | import org.apache.spark.sql.*; 17 | import org.apache.spark.sql.streaming.StreamingQuery; 18 | import org.apache.spark.sql.streaming.StreamingQueryException; 19 | import org.apache.spark.sql.streaming.Trigger; 20 | 21 | /** 22 | * 23 | * An example of consuming messages from Kafka using Protocol Buffers and writing them to Iceberg using the native 24 | * data source and writing via custom Spark/Iceberg writing mechanism 25 | * 26 | * This implements all the features and mechanisms that we want to be demostrated. 27 | * 28 | * Watermark deduplication 29 | * Compaction 30 | * MERGE INTO Deduplication 31 | * 32 | * @author acmanjon@amazon.com 33 | * 34 | */ 35 | 36 | public class SparkCustomIcebergIngestMoRS3BucketsAutoAvro { 37 | 38 | private static final Logger log = LogManager.getLogger(SparkCustomIcebergIngestMoRS3BucketsAutoAvro.class); 39 | private static String master = ""; 40 | private static boolean removeDuplicates = true; 41 | private static String protoDescFile = "Employee.desc"; 42 | private static String icebergWarehouse = "warehouse/"; 43 | private static String checkpointDir = "tmp/"; 44 | private static String bootstrapServers = "localhost:9092"; 45 | private static boolean compactionEnabled = false; 46 | 47 | 48 | public static void main(String[] args) 49 | throws IOException, TimeoutException, StreamingQueryException { 50 | 51 | SparkSession spark; 52 | //default local env. 53 | if (args.length < 1) { 54 | master = "local[*]"; 55 | log.warn( 56 | "No arguments provided, running using local default settings: master={} and Iceberg hadoop based file catalog ", 57 | master); 58 | log.warn( 59 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 60 | + " this mode is for local based execution and development. Kafka broker in this case will also be 'localhost:9092'." 61 | + " Remember to clean the checkpoint dir for any changes or if you want to start 'clean'"); 62 | spark = 63 | SparkSession.builder() 64 | .master(master) 65 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 66 | .config( "spark.sql.extensions","org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 67 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 68 | .config("spark.sql.catalog.spark_catalog.type", "hive") 69 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 70 | .config("spark.sql.catalog.local.type", "hadoop") 71 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this 72 | .config("spark.sql.catalog.local.warehouse", "warehouse") 73 | .config("spark.sql.defaultCatalog", "local") 74 | /** 75 | //enable SPJ 76 | .config("spark.sql.sources.v2.bucketing.enabled","true") 77 | .config("spark.sql.sources.v2.bucketing.pushPartValues.enabled","true") 78 | .config("spark.sql.requireAllClusterKeysForCoPartition","false") 79 | .config("spark.sql.sources.v2.bucketing.partiallyClusteredDistribution.enabled","true") 80 | .config("spark.sql.sources.v2.bucketing.pushPartKeys.enabled","true") 81 | .config("spark.sql.iceberg.planning.preserve-data-grouping","true") 82 | .config("spark.sql.sources.v2.bucketing.allowJoinKeysSubsetOfPartitionKeys.enabled","false") 83 | .config("spark.sql.optimizer.runtime.rowLevelOperationGroupFilter.enabled","false") 84 | // enable shuffle hash join 85 | .config("spark.sql.join.preferSortMergeJoin","false") 86 | .config("spark.sql.shuffledHashJoinFactor","1") 87 | //set none to distribution mode 88 | .config("spark.sql.iceberg.distribution-mode","none") 89 | //disable adaptative 90 | .config("spark.sql.adaptive.coalescePartitions.enabled","false") 91 | .config("spark.sql.adaptive.skewJoin.enabled","false") 92 | .config("spark.sql.adaptive.enabled","false")**/ 93 | 94 | .getOrCreate(); 95 | //local env with optional compaction and duplicate removal 96 | } else if (args.length == 2) { 97 | removeDuplicates = Boolean.parseBoolean(args[0]); 98 | compactionEnabled = Boolean.parseBoolean(args[1]); 99 | master = "local[*]"; 100 | log.warn( 101 | "Running with local master: {} and Iceberg hadoop based file catalog " 102 | + "removing duplicates within the watermark is {}, compactions each 'n' batch are {}", 103 | master, 104 | removeDuplicates, 105 | compactionEnabled); 106 | log.warn( 107 | "Iceberg warehouse dir will be 'warehouse/' from the run dir and the checkpoint directory will be 'tmp/'\n" 108 | + " this mode is for local based execution. Kafka broker in this case will also be 'localhost:9092'."); 109 | spark = 110 | SparkSession.builder() 111 | .master(master) 112 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 113 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 114 | .config("spark.sqleiifcbnchlkglgcrllkldnukrkbuhjkcutcievelnble" + 115 | ".extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 116 | .config("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") 117 | .config("spark.sql.catalog.spark_catalog.type", "hive") 118 | .config("spark.hadoop.fs.s3.impl","org.apache.hadoop.fs.s3a.S3AFileSystem") 119 | .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") 120 | .config("spark.sql.catalog.local.type", "hadoop") 121 | .config("spark.sql.shuffle.partitions","50") // as we are not using AQE then we need to tune this for the size of our cluster/tasks 122 | // remember that we should be ideally at 200MB per task minimum 123 | .config("spark.sql.catalog.local.warehouse", "warehouse") 124 | .config("spark.sql.defaultCatalog", "local") 125 | .getOrCreate(); 126 | //local env connected to Glue catalog 127 | } else if ( args.length == 4){ 128 | removeDuplicates = Boolean.parseBoolean(args[0]); 129 | compactionEnabled = Boolean.parseBoolean(args[1]); 130 | icebergWarehouse = args[2]; 131 | checkpointDir = args[3]; 132 | master = "local[*]"; 133 | log.warn( 134 | "Running with local master and Iceberg hadoop based file catalog " 135 | + "removing duplicates within the watermark is {}, compactions each 'n' batch are {}", 136 | 137 | removeDuplicates, 138 | compactionEnabled); 139 | log.warn( 140 | "Iceberg warehouse dir will be {} from the run dir and the checkpoint directory will be {}'\n" 141 | + " this mode is for local based execution. Kafka broker in this case will also be 'localhost:9092'.", 142 | icebergWarehouse,checkpointDir); 143 | spark = 144 | SparkSession.builder() 145 | .master(master) 146 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 147 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 148 | .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 149 | .config("spark.sql.catalog.glue_catalog.catalog-impl", "org.apache.iceberg.aws.glue.GlueCatalog") 150 | .config("spark.sql.catalog.glue_catalog", "org.apache.iceberg.spark.SparkCatalog") 151 | .config("spark.sql.catalog.glue_catalog.warehouse", icebergWarehouse) 152 | .config("spark.hadoop.fs.s3.impl","org.apache.hadoop.fs.s3a.S3AFileSystem") 153 | .config("spark.sql.catalog.glue_catalog.io-impl", "org.apache.iceberg.aws.s3.S3FileIO") 154 | .config("spark.sql.shuffle.partitions", "100") // as we are not using AQE then we need to tune this 155 | .config("spark.sql.defaultCatalog", "glue_catalog") 156 | .getOrCreate(); 157 | 158 | }else if (args.length == 6) { 159 | removeDuplicates = Boolean.parseBoolean(args[0]); 160 | icebergWarehouse = args[1]; 161 | protoDescFile = args[2]; 162 | checkpointDir = args[3]; 163 | bootstrapServers = args[4]; 164 | compactionEnabled = Boolean.parseBoolean(args[5]); 165 | log.warn( 166 | "Master will be inferred from the environment Iceberg Glue catalog will be used, with the warehouse being: {} \n " 167 | + "removing duplicates within the watermark is {}, the descriptor file is at: {} and the checkpoint is at: {}\n " 168 | + "Kafka bootstrap is: {}, compactions on each 'n' batch are {}", 169 | removeDuplicates,icebergWarehouse, protoDescFile, checkpointDir, bootstrapServers, compactionEnabled); 170 | spark = 171 | SparkSession.builder() 172 | .appName("JavaIoTProtoBufDescriptor2Iceberg") 173 | .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 174 | .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") 175 | .config("spark.sql.catalog.s3tablesbucket", "org.apache.iceberg.spark.SparkCatalog") 176 | .config("spark.sql.catalog.s3tablesbucket.warehouse", icebergWarehouse) 177 | .config("spark.sql.catalog.s3tablesbucket.catalog-impl", "software.amazon.s3tables.iceberg.S3TablesCatalog") 178 | .config("spark.sql.iceberg.data-prefetch.enabled","true") 179 | // .config("spark.hadoop.fs.s3.impl","org.apache.hadoop.fs.s3a.S3AFileSystem") 180 | .config("spark.sql.shuffle.partitions", "500") // as we are not using AQE then we need to tune this 181 | .config("spark.sql.defaultCatalog", "s3tablesbucket") 182 | 183 | .getOrCreate(); 184 | } else { 185 | spark = null; 186 | log.error( 187 | "Invalid number of arguments provided, please check the readme for the correct usage"); 188 | System.exit(1); 189 | } 190 | spark.sql(""" 191 | CREATE DATABASE IF NOT EXISTS bigdata; 192 | """); 193 | 194 | 195 | spark.sql(""" 196 | USE bigdata; 197 | """); 198 | spark.sql( 199 | """ 200 | CREATE TABLE IF NOT EXISTS employee_avro 201 | (employee_id bigint, 202 | age int, 203 | start_date timestamp, 204 | team string, 205 | role string, 206 | address string, 207 | name string 208 | ) 209 | PARTITIONED BY (hours(start_date), team, bucket(42, employee_id)) 210 | TBLPROPERTIES ( 211 | 'table_type'='ICEBERG', 212 | 'write.parquet.compression-level'='7', 213 | 'write.format.default'='avro', 214 | 'write.delete.format.default'='avro', 215 | 'write.delete.mode'='merge-on-read', 216 | 'write.update.mode'='merge-on-read', 217 | 'write.merge.mode'='merge-on-read', 218 | 'write.parquet.row-group-size-bytes' = '134217728', -- 128MB 219 | 'write.parquet.page-size-bytes' = '1048576', -- 2MB 220 | 'write.target-file-size-bytes' = '536870912', -- 256MB 221 | 'write.distribution-mode' = 'hash', 222 | 'write.delete.distribution-mode' = 'none', 223 | 'write.update.distribution-mode' = 'none', 224 | 'write.merge.distribution-mode' = 'none', 225 | 'write.object-storage.enabled' = 'true', 226 | 'write.spark.fanout.enabled' = 'true', 227 | 'write.metadata.delete-after-commit.enabled' = 'false', 228 | 'write.metadata.previous-versions-max' = '50', 229 | 'history.expire.max-snapshot-age-ms' = '259200000', -- 3 days 230 | 'commit.retry.num-retries'='20', --Number of times to retry a commit before failing 231 | 'commit.retry.min-wait-ms'='250', --Minimum time in milliseconds to wait before retrying a commit 232 | 'commit.retry.max-wait-ms'='60000', -- (1 min) Maximum time in milliseconds to wait before retrying a commit 233 | 'write.parquet.compression-codec'='zstd', 234 | -- if you have a huge number of columns remember to tune dict-size and page-size 235 | 'compatibility.snapshot-id-inheritance.enabled'='true' ); 236 | 237 | """); 238 | 239 | Dataset df = 240 | spark 241 | .readStream() 242 | .format("kafka") 243 | .option("kafka.bootstrap.servers", bootstrapServers) 244 | .option("subscribe", "protobuf-demo-topic-pure") 245 | .option("groupIdPrefix","s3table") 246 | .option("max.poll.records", "40000") 247 | .option("poll.timeout.ms", "2000") 248 | .option("fetch.max.bytes", "25728640") 249 | .option("max.partition.fetch.bytes", "35728640") 250 | .load(); 251 | 252 | Dataset output = 253 | df.select(from_protobuf(col("value"),"Employee", protoDescFile).as("Employee")) 254 | .select(col("Employee.*")) 255 | .select( 256 | col("id").as("employee_id"), 257 | col("employee_age.value").as("age"), 258 | col("start_date"), 259 | col("team.name").as("team"), 260 | col("role"), 261 | col("address"), 262 | col("name")); 263 | 264 | StreamingQuery query = 265 | output 266 | .writeStream() 267 | .queryName("streaming-protobuf-ingest") 268 | .format("iceberg") 269 | .outputMode("append") 270 | .foreachBatch( 271 | (VoidFunction2, Long>) 272 | // here we want to make normal "commits" and then for each 10 trigger run 273 | // compactions! 274 | (dataframe, batchId) -> { 275 | var session=dataframe.sparkSession(); 276 | log.warn("Writing batch {}", batchId); 277 | if (removeDuplicates) { 278 | dataframe.createOrReplaceTempView("insert_data"); 279 | // here we are pushing some filters like the team and the date (we know that 280 | // we will have late events from hour ago.... 281 | // we could improve this filtering by bucket and just merge data from that 282 | // bucket ( using 8 merge queries), one per bucket. Iceberg bucketing can be calculated via 283 | // 'system.bucket(8,employee_id)' 284 | // t.employee_id in (1,2,3,...) or t.employee_id in (7,8,9,....) 285 | // in each 'in' you can put 1000 values. 286 | // another way is to generate a column for the bucket and then make the join/ON there 287 | // this one maybe be easier instead of generate that long in(1,3,4,5,6....) list, 288 | // the problem is that you wouldn't able to use INSERT * 289 | // another thing to test storage-partitioned joins but from streaming sources the performance gains... 290 | // should be tested on cluster, on local laptop mode they hurt, already tested 291 | String merge = 292 | """ 293 | MERGE INTO bigdata.employee_avro as t 294 | USING insert_data as s 295 | ON `s`.`employee_id`=`t`.`employee_id` AND `t`.`start_date` > current_timestamp() - INTERVAL 1 HOURS 296 | AND `t`.`team`='Solutions Architects' AND `t`.`start_date`=`s`.`start_date` 297 | WHEN NOT MATCHED THEN INSERT * 298 | """; 299 | session.sql((merge)); 300 | } else { 301 | dataframe.write().insertInto("bigdata.employee_avro"); 302 | } 303 | }) 304 | .trigger(Trigger.ProcessingTime(1, TimeUnit.MINUTES)) 305 | .option("fanout-enabled", "true") // disable ordering 306 | .option("checkpointLocation", checkpointDir) // on local mode connected to glue disable it or add hadoop-aws library to add S3 file api . 307 | .start(); 308 | 309 | if (compactionEnabled) { 310 | ScheduledExecutorService scheduledExecutor = Executors.newSingleThreadScheduledExecutor(); 311 | scheduledExecutor.scheduleAtFixedRate(new Compact(spark), millisToNextHour(), 60*60*1000, TimeUnit.MILLISECONDS); 312 | } 313 | 314 | query.awaitTermination(); 315 | } 316 | 317 | private static long millisToNextHour() { 318 | // we wait 5 minutes to start the compaction process for previous partition 319 | LocalDateTime nextHour = LocalDateTime.now().plusHours(1).truncatedTo(ChronoUnit.HOURS).plusMinutes(5); 320 | return LocalDateTime.now().until(nextHour, ChronoUnit.MILLIS); 321 | } 322 | 323 | 324 | private static class Compact implements Runnable { 325 | private final SparkSession spark; 326 | public Compact(SparkSession spark) { 327 | this.spark = spark; 328 | } 329 | 330 | @Override 331 | public void run() { 332 | // the main idea behind this is in cases where you may have receiving "late 333 | // data randomly and 334 | // doing the compaction jobs with optimistic concurrency will lead into a 335 | // lot of conflicts where you could increase the number of retries ( as we 336 | // are using partial 337 | // progress we need to increase the commit retries though), or you can just 338 | // use this 339 | // strategy for compaction, older partitions on each N batches. 340 | log.warn("\nCompaction in progress:\n"); 341 | spark 342 | .sql( 343 | """ 344 | CALL system.rewrite_data_files( 345 | table => 'employee_avro', 346 | strategy => 'sort', 347 | sort_order => 'start_date', 348 | where => 'start_date >= current_timestamp() - INTERVAL 1 HOURS', -- this sql needs to be adapted to only compact older partitions 349 | options => map( 350 | 'rewrite-job-order','bytes-asc', 351 | 'target-file-size-bytes','273741824', 352 | 'max-file-group-size-bytes','10737418240', 353 | 'partial-progress.enabled', 'true', 354 | 'max-concurrent-file-group-rewrites', '1000', 355 | 'partial-progress.max-commits', '10' 356 | )) 357 | """) 358 | .show(); 359 | // rewrite manifests from time to time 360 | log.warn("\nManifest compaction in progress:\n"); 361 | spark 362 | .sql( 363 | """ 364 | CALL system.rewrite_manifests( 365 | table => 'employee_avro' 366 | ) 367 | """) 368 | .show(); 369 | } 370 | 371 | // old snapshots expiration can be done in another job for older partitions. 372 | } 373 | } 374 | 375 | --------------------------------------------------------------------------------