From 16d1166426473137ad02ab61918151ec3a8c5521 Mon Sep 17 00:00:00 2001 From: Kartikey Khandelwal Date: Wed, 19 Nov 2025 00:41:11 +0530 Subject: [PATCH] feat(hive-lineage): Add table and column-level lineage extraction for hive --- .../java/acryl-hive-lineage/README.md | 457 +++++++++++ .../hive-lineage-consumer/README.md | 331 ++++++++ .../hive-lineage-consumer/build.gradle | 59 ++ .../LineageConsumerMCPEmitterApplication.java | 16 + .../hive/consumer/config/Constants.java | 59 ++ .../hive/consumer/config/KafkaConfig.java | 97 +++ .../hive/consumer/model/HiveLineage.java | 64 ++ .../LineageConsumerMCPEmitterService.java | 141 ++++ .../service/MCPTransformerService.java | 23 + .../DatasetMCPTransformerServiceImpl.java | 240 ++++++ .../impl/QueryMCPTransformerServiceImpl.java | 117 +++ .../hive/consumer/util/MCPEmitterUtil.java | 94 +++ .../datahub/hive/consumer/util/TimeUtils.java | 21 + .../src/main/resources/application.properties | 32 + .../src/main/resources/logback.xml | 60 ++ .../config/KafkaConfigIntegrationTest.java | 172 ++++ .../consumer/config/KafkaConfigSSLTest.java | 164 ++++ .../hive/consumer/config/KafkaConfigTest.java | 230 ++++++ .../hive/consumer/model/HiveLineageTest.java | 413 ++++++++++ .../LineageConsumerMCPEmitterServiceTest.java | 155 ++++ .../DatasetMCPTransformerServiceImplTest.java | 327 ++++++++ .../QueryMCPTransformerServiceImplTest.java | 310 +++++++ .../consumer/util/MCPEmitterUtilTest.java | 359 ++++++++ .../hive/consumer/util/TimeUtilsTest.java | 146 ++++ .../hive-lineage-producer/README.md | 491 +++++++++++ .../hive-lineage-producer/build.gradle | 64 ++ .../hive/producer/HiveLineageLogger.java | 768 ++++++++++++++++++ .../hive/producer/KafkaProducerService.java | 73 ++ .../java/datahub/hive/producer/TimeUtils.java | 14 + .../hive/producer/HiveLineageLoggerTest.java | 318 ++++++++ .../producer/KafkaProducerServiceTest.java | 220 +++++ .../datahub/hive/producer/TimeUtilsTest.java | 79 ++ 32 files changed, 6114 insertions(+) create mode 100644 metadata-integration/java/acryl-hive-lineage/README.md create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/README.md create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/build.gradle create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/LineageConsumerMCPEmitterApplication.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/config/Constants.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/config/KafkaConfig.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/model/HiveLineage.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/LineageConsumerMCPEmitterService.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/MCPTransformerService.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/impl/DatasetMCPTransformerServiceImpl.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/impl/QueryMCPTransformerServiceImpl.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/util/MCPEmitterUtil.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/util/TimeUtils.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/resources/application.properties create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/resources/logback.xml create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/config/KafkaConfigIntegrationTest.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/config/KafkaConfigSSLTest.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/config/KafkaConfigTest.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/model/HiveLineageTest.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/service/LineageConsumerMCPEmitterServiceTest.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/service/impl/DatasetMCPTransformerServiceImplTest.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/service/impl/QueryMCPTransformerServiceImplTest.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/util/MCPEmitterUtilTest.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/util/TimeUtilsTest.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/README.md create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/build.gradle create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/main/java/datahub/hive/producer/HiveLineageLogger.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/main/java/datahub/hive/producer/KafkaProducerService.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/main/java/datahub/hive/producer/TimeUtils.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/test/java/datahub/hive/producer/HiveLineageLoggerTest.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/test/java/datahub/hive/producer/KafkaProducerServiceTest.java create mode 100644 metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/test/java/datahub/hive/producer/TimeUtilsTest.java diff --git a/metadata-integration/java/acryl-hive-lineage/README.md b/metadata-integration/java/acryl-hive-lineage/README.md new file mode 100644 index 00000000000000..1c06804b76614a --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/README.md @@ -0,0 +1,457 @@ +# Hive Lineage Integration for DataHub + +This directory contains the Hive Lineage integration components for DataHub, organized into two main modules: + +1. **hive-lineage-producer** - Hive hook that captures lineage information from Hive queries +2. **hive-lineage-consumer** - Spring Boot application that consumes lineage messages and emits MCPs to DataHub + +## Directory Structure + +``` +acryl-hive-lineage/ +├── hive-lineage-producer/ # Producer module (Hive hook) +│ ├── src/ +│ │ ├── main/ +│ │ │ └── java/ +│ │ │ └── datahub/hive/producer/ +│ │ │ ├── HiveLineageLogger.java +│ │ │ ├── KafkaProducerService.java +│ │ │ └── TimeUtils.java +│ │ └── test/ +│ │ └── java/ +│ │ └── datahub/hive/producer/ +│ │ ├── HiveLineageLoggerTest.java +│ │ ├── KafkaProducerServiceTest.java +│ │ └── TimeUtilsTest.java +│ ├── build.gradle +│ └── README.md +│ +├── hive-lineage-consumer/ # Consumer module (Spring Boot app) +│ ├── src/ +│ │ ├── main/ +│ │ │ └── java/ +│ │ │ └── datahub/hive/consumer/ +│ │ │ ├── LineageConsumerMCPEmitterApplication.java +│ │ │ ├── config/ +│ │ │ │ ├── Constants.java +│ │ │ │ └── KafkaConfig.java +│ │ │ ├── model/ +│ │ │ │ └── HiveLineage.java +│ │ │ ├── service/ +│ │ │ │ ├── LineageConsumerMCPEmitterService.java +│ │ │ │ ├── MCPTransformerService.java +│ │ │ │ └── impl/ +│ │ │ │ ├── DatasetMCPTransformerServiceImpl.java +│ │ │ │ └── QueryMCPTransformerServiceImpl.java +│ │ │ └── util/ +│ │ │ ├── MCPEmitterUtil.java +│ │ │ └── TimeUtils.java +│ │ └── test/ +│ │ └── java/ +│ │ └── datahub/hive/consumer/ +│ │ ├── config/ +│ │ │ ├── KafkaConfigTest.java +│ │ │ ├── KafkaConfigIntegrationTest.java +│ │ │ └── KafkaConfigSSLTest.java +│ │ ├── model/ +│ │ │ └── HiveLineageTest.java +│ │ ├── service/ +│ │ │ ├── LineageConsumerMCPEmitterServiceTest.java +│ │ │ └── impl/ +│ │ │ ├── DatasetMCPTransformerServiceImplTest.java +│ │ │ └── QueryMCPTransformerServiceImplTest.java +│ │ └── util/ +│ │ ├── MCPEmitterUtilTest.java +│ │ └── TimeUtilsTest.java +│ ├── build.gradle +│ └── README.md +│ +└── README.md (this file) +``` + +## Overview + +### Producer (Hive Hook) + +The producer is a Hive post-execution hook that: +- Captures lineage information from Hive CTAS (CREATE TABLE AS SELECT) queries +- Generates JSON lineage messages with source/target relationships +- Publishes messages to a Kafka topic for consumption + +### Consumer (Spring Boot Application) + +The consumer is a Spring Boot application that: +- Listens to the Hive lineage Kafka topic +- Transforms lineage messages into DataHub Metadata Change Proposals (MCPs) +- Emits MCPs to DataHub via Kafka for ingestion + +## Building the JARs + +### Prerequisites + +- Java 17 or higher +- Gradle 8.7 or higher + +### Building the Producer JAR + +The producer builds a shadow JAR by default that includes only Kafka client dependencies (other Hive/Hadoop dependencies are provided by the runtime environment), can change the build.gradle to include other dependencies in shadow JAR if required. + +```bash +cd hive-lineage-producer +./gradlew clean build + +# The JAR will be created at: +# build/libs/hive-lineage-producer.jar +``` + +**Output:** `hive-lineage-producer.jar` (~25-30 KB) + +### Building the Consumer JAR + +The consumer builds a Spring Boot executable JAR with all dependencies included. + +```bash +cd hive-lineage-consumer +./gradlew clean build + +# The JAR will be created at: +# build/libs/hive-lineage-consumer.jar +``` + +**Output:** `hive-lineage-consumer.jar` (~100-120 MB) + +### Building Both JARs + +From the `acryl-hive-lineage` directory: + +```bash +# Build producer +(cd hive-lineage-producer && ./gradlew clean build) + +# Build consumer +(cd hive-lineage-consumer && ./gradlew clean build) +``` + +## Configuration + +### Producer Configuration + +The producer reads configuration from an XML file (default: `/etc/hive/conf/hive-lineage-config.xml`). + +**Configuration Properties:** + +```xml + + + + + hive.lineage.environment + DEV + Environment name (e.g., DEV, QA, PROD) + + + + hive.lineage.platform.instance + your-hive-cluster-name + Platform instance identifier + + + + + hive.lineage.kafka.bootstrap.servers + kafka-broker1:9093,kafka-broker2:9093 + Kafka bootstrap servers + + + + hive.lineage.kafka.topic + HiveLineage_v1 + Kafka topic for lineage messages + + + + hive.lineage.kafka.retries + 0 + Number of retries for failed Kafka sends + + + + hive.lineage.kafka.retry.backoff.ms + 100 + Backoff time in milliseconds between retry attempts + + + + hive.lineage.kafka.enable.idempotence + false + Enable idempotent producer to avoid duplicate messages + + + + hive.lineage.kafka.max.block.ms + 3000 + Maximum time in milliseconds to block waiting for Kafka metadata + + + + hive.lineage.kafka.request.timeout.ms + 3000 + Maximum time in milliseconds to wait for a Kafka request response + + + + hive.lineage.kafka.delivery.timeout.ms + 5000 + Maximum time in milliseconds for message delivery including retries + + + + hive.lineage.vdc.kafka.close.timeout.ms + 120000 + Timeout in milliseconds for closing Kafka producer gracefully + + + + + hive.lineage.kafka.ssl.truststore.location + /path/to/truststore.jks + + + + hive.lineage.kafka.ssl.truststore.password + truststore-password + + + + hive.lineage.kafka.ssl.keystore.location + /path/to/keystore.jks + + + + hive.lineage.kafka.ssl.keystore.password + keystore-password + + + + + hive.lineage.thread.max.pool.size + 100 + Maximum thread pool size multiplier + + + + hive.lineage.thread.queue.capacity + 500 + Thread pool queue capacity + + + + hive.lineage.thread.name + HiveLineageComputationThread- + Thread name prefix for lineage computation threads + + + + hive.lineage.thread.keep.alive.time + 60 + Keep-alive time in seconds for idle threads in the pool + + + + hive.lineage.executor.timeout.seconds + 30 + Timeout in seconds for executor service shutdown + + +``` + +**Hive Configuration:** + +Add to `hive-site.xml`: + +```xml + + + hive.exec.post.hooks + datahub.hive.producer.HiveLineageLogger + + + + + hive.lineage.hook.info.enabled + true + + + + + hive.lineage.custom.config.path + /custom/path/to/hive-lineage-config.xml + +``` + +### Consumer Configuration + +The consumer uses Spring Boot application properties (typically `application.yml` or `application.properties`). + +**application.yml Example:** + +```yaml +spring: + kafka: + consumer: + bootstrap-servers: kafka-broker1:9093,kafka-broker2:9093 + group-id: hive-lineage-consumer-group + auto-offset-reset: earliest + key-deserializer: org.apache.kafka.common.serialization.StringDeserializer + value-deserializer: org.apache.kafka.common.serialization.StringDeserializer + concurrency: 7 + retry: + max-attempts: 3 + initial-delay: 1000 + hive: + lineage: + topic: HiveLineage + + producer: + bootstrap-servers: kafka-broker1:9093,kafka-broker2:9093 + key-serializer: org.apache.kafka.common.serialization.StringSerializer + value-serializer: io.confluent.kafka.serializers.KafkaAvroSerializer + properties: + schema.registry.url: http://schema-registry:8081 + mcp.topic: MetadataChangeProposal_v1 + + security: + protocol: SSL + + ssl: + trust-store-location: file:/path/to/truststore.jks + trust-store-password: truststore-password + key-store-location: file:/path/to/keystore.jks + key-store-password: keystore-password +``` + +## Deployment + +### Producer Deployment + +1. **Copy the JAR to Hive auxiliary path:** + ```bash + cp hive-lineage-producer.jar /path/to/hive/auxlib/ + ``` + +2. **Configure HIVE_AUX_JARS_PATH in hive-env:** + + Add the following to your `hive-env.sh` template under Advanced hive-env configuration: + + ```bash + # Adding hive lineage logger hook jar to Hive aux jars path if present + if [ -f "/path/to/hive/auxlib/hive-lineage-producer.jar" ]; then + export HIVE_AUX_JARS_PATH=${HIVE_AUX_JARS_PATH}:/path/to/hive/auxlib/hive-lineage-producer.jar + fi + ``` + + This ensures the lineage JAR is available in Hive's classpath. + +3. **Configure Hive** (see Configuration section above) + +4. **Restart Hive services** (HiveServer2, Beeline, etc.) + +### Consumer Deployment + +1. **Run as a standalone application:** + ```bash + java -jar hive-lineage-consumer.jar \ + --spring.config.location=/path/to/application.yml + ``` + +## Testing + +### Producer Testing + +Run unit tests: + +```bash +cd hive-lineage-producer +./gradlew test +``` + +Check test coverage: + +```bash +./gradlew jacocoTestReport +# Report available at: build/reports/tests/test/index.html +``` + +The producer automatically logs lineage for CTAS queries. Test with: + +```sql +-- Enable lineage +SET hive.lineage.hook.info.enabled=true; + +-- Run a CTAS query +CREATE TABLE test_output AS +SELECT * FROM test_input WHERE id > 100; + +``` + +Check Hive logs for lineage messages and verify messages appear in Kafka topic. + +### Consumer Testing + +Run unit tests: + +```bash +cd hive-lineage-consumer +./gradlew test +``` + +Check test coverage: + +```bash +./gradlew jacocoTestReport +# Report available at: build/reports/jacoco/test/html/index.html +``` + +## Monitoring + +### Producer Monitoring + +- Check Hive logs for lineage computation times +- Monitor Kafka topic for incoming messages +- Verify thread pool metrics in logs + +### Consumer Monitoring + +- Monitor Spring Boot actuator endpoints (if enabled) +- Check consumer lag using Kafka tools +- Review application logs for processing errors + +## Troubleshooting + +### Producer Issues + +1. **Hook not executing:** + - Verify JAR is in Hive classpath + - Check `hive.exec.post.hooks` configuration + - Ensure `hive.lineage.hook.info.enabled=true` + +2. **Kafka connection errors:** + - Verify bootstrap servers configuration + - Check SSL certificates and passwords + - Ensure network connectivity to Kafka + +3. **Thread pool exhaustion:** + - Increase `hive.lineage.thread.max.pool.size` + - Increase `hive.lineage.thread.queue.capacity` + +### Consumer Issues + +1. **Consumer not receiving messages:** + - Verify Kafka topic name matches producer + - Check consumer group offset + - Verify SSL configuration + +2. **MCP emission failures:** + - Check schema registry connectivity + - Verify MCP topic exists + - Review DataHub logs for ingestion errors diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/README.md b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/README.md new file mode 100644 index 00000000000000..bd8f7fc6eaa5c4 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/README.md @@ -0,0 +1,331 @@ +# Hive Lineage Consumer + +A Spring Boot application that consumes Hive lineage messages from Kafka and transforms them into DataHub Metadata Change Proposals (MCPs) for ingestion into DataHub. + +## Overview + +The Hive Lineage Consumer is a microservice that: +- Listens to Kafka topic for Hive lineage messages +- Transforms lineage JSON into DataHub MCPs +- Emits dataset and query entities with lineage aspects to DataHub +- Supports both table-level and column-level (fine-grained) lineage +- Provides retry mechanisms and error handling + +## Architecture + +``` +Kafka Topic (HiveLineage_v1) + ↓ +Kafka Consumer (Spring Kafka) + ↓ +LineageConsumerMCPEmitterService + ↓ + ┌────────────────┴────────────────┐ + ↓ ↓ +DatasetMCPTransformer QueryMCPTransformer + ↓ ↓ +Dataset MCPs Query MCPs + └────────────────┬────────────────┘ + ↓ + MCPEmitterUtil + ↓ + Kafka Producer (Avro) + ↓ + Kafka Topic (MetadataChangeProposal_v1) + ↓ + DataHub GMS +``` + +## Directory Structure + +``` +hive-lineage-consumer/ +├── src/ +│ ├── main/ +│ │ ├── java/ +│ │ │ └── datahub/hive/consumer/ +│ │ │ ├── LineageConsumerMCPEmitterApplication.java # Main Spring Boot app +│ │ │ ├── config/ +│ │ │ │ ├── Constants.java # Application constants +│ │ │ │ └── KafkaConfig.java # Kafka configuration +│ │ │ ├── model/ +│ │ │ │ └── HiveLineage.java # Lineage data model +│ │ │ ├── service/ +│ │ │ │ ├── LineageConsumerMCPEmitterService.java # Main consumer service +│ │ │ │ ├── MCPTransformerService.java # Transformer interface +│ │ │ │ └── impl/ +│ │ │ │ ├── DatasetMCPTransformerServiceImpl.java # Dataset transformer +│ │ │ │ └── QueryMCPTransformerServiceImpl.java # Query transformer +│ │ │ └── util/ +│ │ │ ├── MCPEmitterUtil.java # MCP emission utility +│ │ │ └── TimeUtils.java # Time utilities +│ │ └── resources/ +│ │ └── application.properties # Application configuration +│ └── test/ +│ └── java/ +│ └── datahub/hive/consumer/ +│ ├── config/ +│ │ ├── KafkaConfigTest.java +│ │ ├── KafkaConfigIntegrationTest.java +│ │ └── KafkaConfigSSLTest.java +│ ├── model/ +│ │ └── HiveLineageTest.java +│ ├── service/ +│ │ ├── LineageConsumerMCPEmitterServiceTest.java +│ │ └── impl/ +│ │ ├── DatasetMCPTransformerServiceImplTest.java +│ │ └── QueryMCPTransformerServiceImplTest.java +│ └── util/ +│ ├── MCPEmitterUtilTest.java +│ └── TimeUtilsTest.java +├── build.gradle +└── README.md (this file) +``` + +## Components + +### 1. LineageConsumerMCPEmitterService + +The main Kafka consumer service that: +- Listens to Hive lineage Kafka topic using `@KafkaListener` +- Parses incoming JSON lineage messages +- Delegates transformation to appropriate transformer services +- Handles errors and retries +- Logs processing metrics + +### 2. DatasetMCPTransformerServiceImpl + +Transforms lineage into dataset MCPs: +- Creates `UpstreamLineage` aspect with table-level lineage +- Creates `FineGrainedLineage` for column-level lineage +- Generates `DataPlatformInstance` aspect +- Adds `Status` and `SubTypes` aspects +- Links to query entities via query URNs + +### 3. QueryMCPTransformerServiceImpl + +Transforms lineage into query MCPs: +- Creates `QueryProperties` aspect with SQL statement +- Creates `QuerySubjects` aspect linking to output datasets +- Captures query metadata (user, timestamps, etc.) +- Sets query source as MANUAL + +### 4. MCPEmitterUtil + +Utility for emitting MCPs to DataHub: +- Creates `MetadataChangeProposalWrapper` objects +- Sends MCPs to Kafka using Avro serialization +- Provides callback-based async emission +- Handles errors and logs results + +### 5. KafkaConfig + +Spring configuration for Kafka: +- Configures consumer factory with retry logic +- Sets up SSL/TLS for secure connections +- Configures producer for MCP emission +- Manages Kafka emitter bean lifecycle + +## Building + +### Prerequisites + +- Java 17 or higher +- Gradle 8.7 or higher + +### Build Commands + +```bash +# Clean and build +./gradlew clean build + +# Build without tests +./gradlew clean build -x test + +# Build Spring Boot JAR +./gradlew bootJar + +# The JAR will be created at: +# build/libs/hive-lineage-consumer.jar +``` + +**Output:** `hive-lineage-consumer.jar` (~100-120 MB with all dependencies) + +## Deployment + +### Run as Standalone Application + +```bash +# Run with default configuration +java -jar hive-lineage-consumer.jar + +# Run with custom configuration +java -jar hive-lineage-consumer.jar \ + --spring.config.location=/path/to/application.yml + +# Run with environment-specific profile +java -jar hive-lineage-consumer.jar \ + --spring.profiles.active=production +``` + +## Testing + +### Running Unit Tests + +```bash +# Run all tests +./gradlew test + +# Run specific test class +./gradlew test --tests LineageConsumerMCPEmitterServiceTest +./gradlew test --tests DatasetMCPTransformerServiceImplTest +./gradlew test --tests QueryMCPTransformerServiceImplTest + +# Run with detailed output +./gradlew test --info + +# Generate test coverage report +./gradlew jacocoTestReport +# Report available at: build/reports/jacoco/test/html/index.html +``` + +### Verification + +1. **Check Consumer Logs:** + + The application logs to multiple locations based on logback.xml configuration: + + ```bash + # Main application log (default location) + tail -f /datahub/hive-lineage-consumer/logs/hive-lineage-consumer.log + + # Error log (ERROR level only) + tail -f /datahub/hive-lineage-consumer/logs/hive-lineage-consumer.error.log + + # Custom log directory (set via LOG_DIR environment variable) + export LOG_DIR=/custom/log/path + tail -f $LOG_DIR/hive-lineage-consumer.log + ``` + + **Log Configuration:** + - Default log directory: `/datahub/hive-lineage-consumer/logs/` + - Log rotation: Daily, max 100MB per file + - Retention: 30 days, max 10GB total + - Log pattern: `%date{ISO8601} [%thread] %-5level %logger{36}:%L - %msg%n` + + +2. **Verify MCP Emission:** + ```bash + kafka-console-consumer --bootstrap-server localhost:9092 \ + --topic MetadataChangeProposal_v1 --from-beginning + ``` + +3. **Check DataHub UI:** + - Navigate to dataset: `urn:li:dataset:(urn:li:dataPlatform:hive,.default.output_table,DEV)` + - Verify lineage tab shows upstream relationships + - Check column-level lineage if available + +## Monitoring + +### Key Metrics + +Monitor the following metrics: + +- **Consumer Lag:** Kafka consumer group lag +- **Processing Time:** Time to process each lineage message +- **MCP Emission Success Rate:** Percentage of successful MCP emissions +- **Error Rate:** Number of processing errors +- **Throughput:** Messages processed per second + +## Troubleshooting + +### Consumer Not Receiving Messages + +**Symptoms:** No messages being consumed from Kafka + +**Solutions:** +1. Verify Kafka topic exists and has messages: + ```bash + kafka-topics --bootstrap-server localhost:9092 --describe --topic HiveLineage_v1 + ``` +2. Check consumer group offset: + ```bash + kafka-consumer-groups --bootstrap-server localhost:9092 \ + --describe --group hive-lineage-consumer-group + ``` +3. Verify SSL configuration and certificates +4. Check network connectivity to Kafka brokers + +### MCP Emission Failures + +**Symptoms:** "Failed to emit MCP" errors in logs + +**Solutions:** +1. Verify schema registry is accessible: + ```bash + curl http://schema-registry:8081/subjects + ``` +2. Check MCP topic exists: + ```bash + kafka-topics --bootstrap-server localhost:9092 --describe --topic MetadataChangeProposal_v1 + ``` +3. Verify Avro schema compatibility +4. Check DataHub GMS logs for ingestion errors + +### High Consumer Lag + +**Symptoms:** Consumer lag increasing continuously + +**Solutions:** +1. Increase consumer concurrency in configuration +2. Scale horizontally by adding more consumer instances +3. Check for slow network or Kafka issues + +### Memory Issues + +**Symptoms:** OutOfMemoryError or high heap usage + +**Solutions:** +1. Increase JVM heap size: + ```bash + java -Xmx4g -Xms2g -jar hive-lineage-consumer.jar + ``` +2. Reduce consumer concurrency +3. Monitor for memory leaks +4. Enable GC logging for analysis + +## Dependencies + +### Runtime Dependencies + +- Spring Boot 3.3.4 +- Spring Kafka +- DataHub Client 1.1.0 +- Kafka Avro Serializer +- Gson for JSON processing + +### Test Dependencies + +- JUnit 5 (Jupiter) +- Mockito +- Spring Boot Test +- Spring Kafka Test + +## Data Model + +### Input: Hive Lineage Message + +See producer README for complete message format. + +### Output: DataHub MCPs + +**Dataset Entity MCPs:** +- UpstreamLineage aspect +- FineGrainedLineage aspect (if column lineage available) +- DataPlatformInstance aspect +- Status aspect +- SubTypes aspect + +**Query Entity MCPs:** +- QueryProperties aspect +- QuerySubjects aspect diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/build.gradle b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/build.gradle new file mode 100644 index 00000000000000..6d6b3e18385fbf --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/build.gradle @@ -0,0 +1,59 @@ +plugins { + id 'java' + id 'jacoco' + id 'org.springframework.boot' version '3.3.4' + id 'io.spring.dependency-management' version '1.1.6' +} + +group = 'datahub.hive' + +java { + toolchain { + languageVersion = JavaLanguageVersion.of(17) + } +} + +jacoco { + toolVersion = "0.8.8" +} + +test { + finalizedBy jacocoTestReport +} + +jacocoTestReport { + reports { + xml.required = true + html.required = true + } +} + +dependencies { + implementation 'org.springframework.boot:spring-boot-starter' + implementation 'org.springframework.boot:spring-boot-starter-web' + implementation 'org.springframework.kafka:spring-kafka' + implementation 'io.acryl:datahub-client:1.1.0' + implementation 'io.confluent:kafka-avro-serializer:7.4.0' + implementation 'io.confluent:kafka-schema-registry-client:7.4.0' + implementation 'org.apache.avro:avro:1.11.1' + compileOnly 'org.projectlombok:lombok' + annotationProcessor 'org.projectlombok:lombok' + implementation 'com.google.code.gson:gson' + implementation 'org.apache.commons:commons-text:1.10.0' + testImplementation 'org.springframework.boot:spring-boot-starter-test' + testImplementation 'org.springframework.kafka:spring-kafka-test' + testRuntimeOnly 'org.junit.platform:junit-platform-launcher' +} + +tasks.named('jar') { + enabled = false +} + +tasks.named('test') { + useJUnitPlatform() +} + +bootJar { + archiveBaseName.set('hive-lineage-consumer') + mainClass = 'datahub.hive.consumer.LineageConsumerMCPEmitterApplication' +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/LineageConsumerMCPEmitterApplication.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/LineageConsumerMCPEmitterApplication.java new file mode 100644 index 00000000000000..6f0ca1509cab72 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/LineageConsumerMCPEmitterApplication.java @@ -0,0 +1,16 @@ +package datahub.hive.consumer; + +import org.springframework.boot.SpringApplication; +import org.springframework.boot.autoconfigure.SpringBootApplication; + +/** + * Main application class for the Lineage Consumer service. + * This service consumes Hive lineage data from Kafka and emits Metadata Change Proposals (MCPs) to DataHub. + */ +@SpringBootApplication +public class LineageConsumerMCPEmitterApplication { + + public static void main(String[] args) { + SpringApplication.run(LineageConsumerMCPEmitterApplication.class, args); + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/config/Constants.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/config/Constants.java new file mode 100644 index 00000000000000..0270e2b95827fe --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/config/Constants.java @@ -0,0 +1,59 @@ +package datahub.hive.consumer.config; + +/** + * Application constants. + */ +public final class Constants { + + public static final String PLATFORM_NAME = "hive"; + + public static final String ENVIRONMENT_KEY = "environment"; + + public static final String PLATFORM_INSTANCE_KEY = "platformInstance"; + + public static final String TABLE_KEY = "Table"; + + public static final String DATASET_KEY = "dataset"; + + public static final String INPUTS_KEY = "inputs"; + + public static final String OUTPUTS_KEY = "outputs"; + + public static final String HASH_KEY = "hash"; + + public static final String QUERY_KEY = "query"; + + public static final String QUERY_URN_PREFIX = "urn:li:query:"; + + public static final String CORP_USER_URN_PREFIX = "urn:li:corpuser:"; + + public static final String EDGES_KEY = "edges"; + + public static final String VERTICES_KEY = "vertices"; + + public static final String SOURCES_KEY = "sources"; + + public static final String TARGETS_KEY = "targets"; + + public static final String EDGE_TYPE_KEY = "edgeType"; + + public static final String VERTEX_TYPE_KEY = "vertexType"; + + public static final String VERTEX_ID_KEY = "vertexId"; + + public static final String EXPRESSION_KEY = "expression"; + + public static final String QUERY_TEXT_KEY = "queryText"; + + public static final String PROJECTION_KEY = "PROJECTION"; + + public static final String COLUMN_KEY = "COLUMN"; + + public static final String SCHEMA_FIELD_URN_PREFIX = "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:"; + + public static final String DATA_PLATFORM_INSTANCE_URN_PREFIX = "urn:li:dataPlatformInstance:(urn:li:dataPlatform:hive,"; + + private Constants() { + // Private constructor to prevent instantiation + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/config/KafkaConfig.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/config/KafkaConfig.java new file mode 100644 index 00000000000000..76fcf325ecd73d --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/config/KafkaConfig.java @@ -0,0 +1,97 @@ +package datahub.hive.consumer.config; + +import datahub.client.kafka.KafkaEmitter; +import datahub.client.kafka.KafkaEmitterConfig; + +import org.springframework.beans.factory.annotation.Value; +import org.springframework.boot.autoconfigure.kafka.KafkaProperties; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; +import org.springframework.kafka.annotation.EnableKafka; +import org.springframework.kafka.config.ConcurrentKafkaListenerContainerFactory; +import org.springframework.kafka.core.ConsumerFactory; +import org.springframework.kafka.listener.ContainerProperties; +import org.springframework.kafka.listener.DefaultErrorHandler; +import org.springframework.util.backoff.FixedBackOff; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +/** + * Configuration class for Kafka consumer and producer. + */ +@Configuration +@EnableKafka +public class KafkaConfig { + + private final KafkaProperties kafkaProperties; + + @Value("${spring.kafka.consumer.concurrency:7}") + private int consumerConcurrency; + + @Value("${spring.kafka.consumer.retry.max-attempts:3}") + private int maxRetryAttempts; + + @Value("${spring.kafka.consumer.retry.initial-delay:1000}") + private int delay; + + public KafkaConfig(KafkaProperties kafkaProperties) { + this.kafkaProperties = kafkaProperties; + } + + /** + * Bean for KafkaEmitter with Avro serialization used to send messages to the MCP topic. + */ + @Bean + public KafkaEmitter kafkaEmitter() throws IOException { + + String bootstrapServers = String.join(",", kafkaProperties.getProducer().getBootstrapServers()); + String schemaRegistryUrl = kafkaProperties.getProducer().getProperties().get("schema.registry.url"); + String mcpTopic = kafkaProperties.getProducer().getProperties().get("mcp.topic"); + + Map producerConfig = new HashMap<>(); + + producerConfig.put("security.protocol", kafkaProperties.getSecurity().getProtocol()); + try { + producerConfig.put("ssl.truststore.location", kafkaProperties.getSsl().getTrustStoreLocation().getFile().getAbsolutePath()); + producerConfig.put("ssl.keystore.location", kafkaProperties.getSsl().getKeyStoreLocation().getFile().getAbsolutePath()); + } catch (IOException e) { + throw new IOException("Error getting SSL store locations", e); + } + producerConfig.put("ssl.truststore.password", kafkaProperties.getSsl().getTrustStorePassword()); + producerConfig.put("ssl.keystore.password", kafkaProperties.getSsl().getKeyStorePassword()); + + producerConfig.putAll(kafkaProperties.getProducer().getProperties()); + + KafkaEmitterConfig config = KafkaEmitterConfig.builder() + .bootstrap(bootstrapServers) + .schemaRegistryUrl(schemaRegistryUrl) + .producerConfig(producerConfig) + .build(); + + return new KafkaEmitter(config, mcpTopic); + } + + /** + * Bean for Kafka listener container factory with record acknowledgment. + */ + @Bean + public ConcurrentKafkaListenerContainerFactory kafkaListenerContainerFactory( + ConsumerFactory consumerFactory) { + ConcurrentKafkaListenerContainerFactory factory = new ConcurrentKafkaListenerContainerFactory<>(); + factory.setConsumerFactory(consumerFactory); + factory.getContainerProperties().setAckMode(ContainerProperties.AckMode.RECORD); + factory.setConcurrency(consumerConcurrency); + + // The first parameter is the interval between retries in milliseconds + // The second parameter is the maximum number of attempts (UNLIMITED = -1) + FixedBackOff fixedBackOff = new FixedBackOff(delay, maxRetryAttempts - 1); + + DefaultErrorHandler errorHandler = new DefaultErrorHandler(fixedBackOff); + + factory.setCommonErrorHandler(errorHandler); + + return factory; + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/model/HiveLineage.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/model/HiveLineage.java new file mode 100644 index 00000000000000..16debfe57a4750 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/model/HiveLineage.java @@ -0,0 +1,64 @@ +package datahub.hive.consumer.model; + +import com.google.gson.annotations.SerializedName; +import lombok.Data; + +import java.util.List; + +/** + * Model class representing the Hive lineage data received from Kafka. + */ +@Data +public class HiveLineage { + private String version; + private String user; + private long timestamp; + private long duration; + + @SerializedName("jobIds") + private List jobIds; + + private String engine; + private String database; + private String hash; + + @SerializedName("queryText") + private String queryText; + + private String environment; + + @SerializedName("platformInstance") + private String platformInstance; + + private List inputs; + private List outputs; + private List edges; + private List vertices; + + /** + * Edge in the lineage graph. + */ + @Data + public static class Edge { + private List sources; + private List targets; + private String expression; + + @SerializedName("edgeType") + private String edgeType; + } + + /** + * Vertex in the lineage graph. + */ + @Data + public static class Vertex { + private int id; + + @SerializedName("vertexType") + private String vertexType; + + @SerializedName("vertexId") + private String vertexId; + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/LineageConsumerMCPEmitterService.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/LineageConsumerMCPEmitterService.java new file mode 100644 index 00000000000000..6819c1aeb05250 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/LineageConsumerMCPEmitterService.java @@ -0,0 +1,141 @@ +package datahub.hive.consumer.service; + +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; +import com.linkedin.data.DataMap; +import com.linkedin.data.template.DataTemplate; +import com.linkedin.common.urn.DatasetUrn; +import com.linkedin.common.urn.DataPlatformUrn; +import com.linkedin.common.FabricType; +import datahub.client.kafka.KafkaEmitter; +import datahub.hive.consumer.util.MCPEmitterUtil; +import datahub.hive.consumer.util.TimeUtils; +import datahub.hive.consumer.config.Constants; +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.springframework.beans.factory.annotation.Qualifier; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.kafka.annotation.KafkaListener; +import org.springframework.stereotype.Service; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.List; + +/** + * Service for consuming Hive lineage messages from Kafka, transforming them into MCPs and emitting them to DataHub. + */ +@Service +@Slf4j +public class LineageConsumerMCPEmitterService { + + private final MCPTransformerService datasetMCPTransformerService; + + private final MCPTransformerService queryMCPTransformerService; + + private final KafkaEmitter kafkaEmitter; + + @Value("${spring.kafka.consumer.retry.max-attempts:3}") + private int maxRetryAttempts; + + public LineageConsumerMCPEmitterService(@Qualifier("datasetMCPTransformerServiceImpl") MCPTransformerService datasetMCPTransformerService, @Qualifier("queryMCPTransformerServiceImpl") MCPTransformerService queryMCPTransformerService, KafkaEmitter kafkaEmitter) { + this.datasetMCPTransformerService = datasetMCPTransformerService; + this.queryMCPTransformerService = queryMCPTransformerService; + this.kafkaEmitter = kafkaEmitter; + } + + /** + * Consumes messages from the Hive lineage topic and processes them. + * + * @param record The Kafka consumer record + * @throws Exception If there's an error during message processing + */ + @KafkaListener( + topics = "${spring.kafka.consumer.hive.lineage.topic}", + containerFactory = "kafkaListenerContainerFactory" + ) + public void consumeLineageMessage(ConsumerRecord record) throws Exception { + long startTime = System.currentTimeMillis(); + String key = record.key(); + String value = record.value(); + + log.info("Received lineage message with key: {}", key); + + try { + JsonObject lineageJson = JsonParser.parseString(value).getAsJsonObject(); + processLineageMessage(lineageJson); + long duration = TimeUtils.calculateDuration(startTime); + log.info("Processed lineage message with key: {}. Time taken: {} ms", key, duration); + } catch (Exception e) { + log.error("Error processing lineage message with key: {} after {} retries", key, maxRetryAttempts, e); + throw e; + } + } + + /** + * Processes lineage message by transforming it into MCPs. + * + * @param lineageJson The JSON object representing the lineage message + * @throws IOException If there's an error during MCP emission + * @throws URISyntaxException If there's an error with URIs in the lineage data + */ + private void processLineageMessage(JsonObject lineageJson) throws IOException, URISyntaxException { + if (lineageJson.has(Constants.OUTPUTS_KEY) && !lineageJson.getAsJsonArray(Constants.OUTPUTS_KEY).isEmpty()) { + // Transform and emit MCPs for the dataset entity + List> datasetAspects = datasetMCPTransformerService.transformToMCP(lineageJson); + emit(lineageJson, datasetAspects, kafkaEmitter); + + // Transform and emit MCPs for the query entity + List> queryAspects = queryMCPTransformerService.transformToMCP(lineageJson); + emit(lineageJson, queryAspects, kafkaEmitter); + } else { + log.info("Skipping lineage message as it does not have any outputs"); + } + } + + /** + * Emits MCPs for the given aspects. + * + * @param lineageJson The JSON object representing the lineage message + * @param aspects The list of aspects to emit + * @param kafkaEmitter The Kafka emitter to use for emitting MCPs + * @throws IOException If there's an error during MCP emission + * @throws URISyntaxException If there's an error with URIs in the lineage data + */ + private void emit(JsonObject lineageJson, List> aspects, KafkaEmitter kafkaEmitter) throws IOException, URISyntaxException { + String entityUrn; + String entityType; + + for (DataTemplate aspect : aspects) { + // Determine entity type and URN based on the aspect type + if (aspect.getClass().getName().contains(Constants.QUERY_KEY)) { + entityType = Constants.QUERY_KEY; + String queryId = lineageJson.get(Constants.HASH_KEY).getAsString(); + entityUrn = Constants.QUERY_URN_PREFIX + queryId; + } else { + entityType = Constants.DATASET_KEY; + entityUrn = getDatasetUrn(lineageJson).toString(); + } + MCPEmitterUtil.emitMCP(aspect, entityUrn, entityType, kafkaEmitter); + log.info("Emitted MCP for entity: {}", entityUrn); + } + } + + /** + * Gets the dataset URN for the given dataset JSON object. + * + * @param datasetJsonObject The JSON object representing the dataset + * @return The dataset URN + */ + private DatasetUrn getDatasetUrn(JsonObject datasetJsonObject) { + String platformInstance = datasetJsonObject.get(Constants.PLATFORM_INSTANCE_KEY).getAsString(); + String datasetName = platformInstance + "." + datasetJsonObject.get(Constants.OUTPUTS_KEY).getAsJsonArray().get(0).getAsString(); + String environment = datasetJsonObject.get(Constants.ENVIRONMENT_KEY).getAsString(); + + return new DatasetUrn( + new DataPlatformUrn(Constants.PLATFORM_NAME), + datasetName, + FabricType.valueOf(environment) + ); + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/MCPTransformerService.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/MCPTransformerService.java new file mode 100644 index 00000000000000..8bdb462add750e --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/MCPTransformerService.java @@ -0,0 +1,23 @@ +package datahub.hive.consumer.service; + +import com.google.gson.JsonObject; +import com.linkedin.data.DataMap; +import com.linkedin.data.template.DataTemplate; + +import java.net.URISyntaxException; +import java.util.List; + +/** + * Interface for services that transform Hive lineage data into Metadata Change Proposals (MCPs). + */ +public interface MCPTransformerService { + + /** + * Transforms lineage message to MCPs for the given entity. + * + * @param entityJson The JSON object representing the entity + * @return A list of DataTemplate representing Aspects of the entity + * @throws URISyntaxException If a URI syntax error occurs + */ + List> transformToMCP(JsonObject entityJson) throws URISyntaxException; +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/impl/DatasetMCPTransformerServiceImpl.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/impl/DatasetMCPTransformerServiceImpl.java new file mode 100644 index 00000000000000..9f33304404340f --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/impl/DatasetMCPTransformerServiceImpl.java @@ -0,0 +1,240 @@ +package datahub.hive.consumer.service.impl; + +import com.google.gson.JsonArray; +import com.google.gson.JsonObject; +import com.linkedin.common.AuditStamp; +import com.linkedin.common.DataPlatformInstance; +import com.linkedin.common.FabricType; +import com.linkedin.common.Status; +import com.linkedin.common.SubTypes; +import com.linkedin.common.UrnArray; +import com.linkedin.common.urn.DataPlatformUrn; +import com.linkedin.common.urn.DatasetUrn; +import com.linkedin.common.urn.Urn; +import com.linkedin.data.DataMap; +import com.linkedin.data.template.DataTemplate; +import com.linkedin.data.template.StringArray; +import com.linkedin.dataset.UpstreamLineage; +import com.linkedin.dataset.DatasetLineageType; +import com.linkedin.dataset.FineGrainedLineage; +import com.linkedin.dataset.FineGrainedLineageArray; +import com.linkedin.dataset.FineGrainedLineageDownstreamType; +import com.linkedin.dataset.FineGrainedLineageUpstreamType; +import com.linkedin.dataset.Upstream; +import com.linkedin.dataset.UpstreamArray; + +import datahub.hive.consumer.config.Constants; +import datahub.hive.consumer.service.MCPTransformerService; +import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.stereotype.Service; + +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.List; + +/** + * Implementation of MCPTransformerService for Dataset entities. + */ +@Service +@Slf4j +public class DatasetMCPTransformerServiceImpl implements MCPTransformerService { + + @Value("${application.environment}") + private String environment; + + @Value("${application.service.user}") + private String serviceUser; + + @Override + public List> transformToMCP(JsonObject datasetJsonObject) throws URISyntaxException { + List> aspects = new ArrayList<>(); + + // Build upstreamLineage aspect + aspects.add(buildUpstreamLineageAspect(datasetJsonObject)); + + // Build dataPlatformInstance aspect + aspects.add(buildDataPlatformInstanceAspect(datasetJsonObject)); + + // Build status aspect + aspects.add(buildStatusAspect()); + + // Build subTypes aspect + aspects.add(buildSubTypesAspect()); + + return aspects; + } + + /** + * Build the upstream lineage aspect for the dataset. + */ + private DataTemplate buildUpstreamLineageAspect(JsonObject datasetJsonObject) throws URISyntaxException { + UpstreamLineage upstreamLineage = new UpstreamLineage(); + List upstreams = new ArrayList<>(); + + // Process inputs as upstream datasets + if (datasetJsonObject.has(Constants.INPUTS_KEY) && !datasetJsonObject.getAsJsonArray(Constants.INPUTS_KEY).isEmpty()) { + for (var inputElement : datasetJsonObject.getAsJsonArray(Constants.INPUTS_KEY)) { + String inputDataset = datasetJsonObject.get(Constants.PLATFORM_INSTANCE_KEY).getAsString() + "." + inputElement.getAsString(); + + Upstream upstream = new Upstream(); + + DatasetUrn upstreamUrn = new DatasetUrn( + new DataPlatformUrn(Constants.PLATFORM_NAME), + inputDataset, + FabricType.valueOf(environment) + ); + upstream.setDataset(upstreamUrn); + + upstream.setType(DatasetLineageType.TRANSFORMED); + + if (datasetJsonObject.has(Constants.HASH_KEY)) { + String queryId = datasetJsonObject.get(Constants.HASH_KEY).getAsString(); + Urn queryUrn = Urn.createFromString(Constants.QUERY_URN_PREFIX + queryId); + upstream.setQuery(queryUrn); + } + + long timestamp = System.currentTimeMillis(); + + AuditStamp created = new AuditStamp(); + created.setTime(timestamp); + created.setActor(Urn.createFromString(Constants.CORP_USER_URN_PREFIX + serviceUser)); + upstream.setCreated(created); + + AuditStamp lastModified = new AuditStamp(); + lastModified.setTime(timestamp); + lastModified.setActor(Urn.createFromString(Constants.CORP_USER_URN_PREFIX + serviceUser)); + upstream.setAuditStamp(lastModified); + + upstreams.add(upstream); + } + } + + List fineGrainedLineages = new ArrayList<>(); + if (datasetJsonObject.has(Constants.EDGES_KEY) && datasetJsonObject.has(Constants.VERTICES_KEY)) { + // Process edges to find column-level lineage + for (var edgeElement : datasetJsonObject.getAsJsonArray(Constants.EDGES_KEY)) { + JsonObject edge = edgeElement.getAsJsonObject(); + String edgeType = edge.get(Constants.EDGE_TYPE_KEY).getAsString(); + + if (Constants.PROJECTION_KEY.equals(edgeType)) { + JsonArray sources = edge.getAsJsonArray(Constants.SOURCES_KEY); + JsonArray targets = edge.getAsJsonArray(Constants.TARGETS_KEY); + + if (sources != null && targets != null && !sources.isEmpty() && !targets.isEmpty()) { + FineGrainedLineage fgl = new FineGrainedLineage(); + + fgl.setUpstreamType(FineGrainedLineageUpstreamType.FIELD_SET); + fgl.setDownstreamType(FineGrainedLineageDownstreamType.FIELD_SET); + + fgl.setConfidenceScore(1.0f); + + // Map vertex IDs to field URNs + JsonArray vertices = datasetJsonObject.getAsJsonArray(Constants.VERTICES_KEY); + + // Process source vertices for upstream fields + List upstreamFields = new ArrayList<>(processVertices(datasetJsonObject, vertices, sources, Constants.INPUTS_KEY)); + + // Process target vertices for downstream fields + List downstreamFields = new ArrayList<>(processVertices(datasetJsonObject, vertices, targets, Constants.OUTPUTS_KEY)); + + // Set upstream and downstream fields + if (!upstreamFields.isEmpty() && !downstreamFields.isEmpty()) { + UrnArray upstreamUrnArray = new UrnArray(); + for (String fieldUrn : upstreamFields) { + upstreamUrnArray.add(Urn.createFromString(fieldUrn)); + } + fgl.setUpstreams(upstreamUrnArray); + + UrnArray downstreamUrnArray = new UrnArray(); + for (String fieldUrn : downstreamFields) { + downstreamUrnArray.add(Urn.createFromString(fieldUrn)); + } + fgl.setDownstreams(downstreamUrnArray); + + if (edge.has(Constants.EXPRESSION_KEY)) { + fgl.setTransformOperation(edge.get(Constants.EXPRESSION_KEY).getAsString()); + } + + fineGrainedLineages.add(fgl); + } + } + } + } + } + + // Set upstreams and fine-grained lineages + upstreamLineage.setUpstreams(new UpstreamArray(upstreams)); + if (!fineGrainedLineages.isEmpty()) { + upstreamLineage.setFineGrainedLineages(new FineGrainedLineageArray(fineGrainedLineages)); + } + + return upstreamLineage; + } + + /** + * Build the data platform instance aspect for the dataset. + */ + private DataTemplate buildDataPlatformInstanceAspect(JsonObject datasetJsonObject) throws URISyntaxException { + String platformInstanceUrn = Constants.DATA_PLATFORM_INSTANCE_URN_PREFIX + datasetJsonObject.get(Constants.PLATFORM_INSTANCE_KEY).getAsString() + ")"; + + return new DataPlatformInstance() + .setPlatform(new DataPlatformUrn(Constants.PLATFORM_NAME)) + .setInstance(Urn.createFromString(platformInstanceUrn)); + } + + /** + * Build the subTypes aspect for the dataset. + */ + private DataTemplate buildSubTypesAspect() { + return new SubTypes().setTypeNames(new StringArray(Constants.TABLE_KEY)); + } + + /** + * Build the status aspect for the dataset. + */ + private DataTemplate buildStatusAspect() { + return new Status().setRemoved(false); + } + + /** + * Process vertices to extract field URNs. + * + * @param datasetJsonObject The JSON object representing the dataset + * @param vertices The array of all vertices + * @param vertexIds The array of vertex IDs to process + * @param datasetArrayKey The key for the dataset array ("inputs" or "outputs") + * @return A list of field URNs + */ + private List processVertices(JsonObject datasetJsonObject, JsonArray vertices, + JsonArray vertexIds, String datasetArrayKey) { + List fieldUrns = new ArrayList<>(); + + for (var vertexIdElement : vertexIds) { + int vertexId = vertexIdElement.getAsInt(); + if (vertexId < vertices.size()) { + JsonObject vertex = vertices.get(vertexId).getAsJsonObject(); + if (Constants.COLUMN_KEY.equals(vertex.get(Constants.VERTEX_TYPE_KEY).getAsString())) { + String vertexLabel = vertex.get(Constants.VERTEX_ID_KEY).getAsString(); + // Create field URN + String[] parts = vertexLabel.split("\\."); + if (parts.length >= 2) { + String tableName = parts[0]; + String columnName = parts[parts.length - 1]; + for (var datasetElement : datasetJsonObject.getAsJsonArray(datasetArrayKey)) { + String dataset = datasetElement.getAsString(); + if (dataset.contains(tableName)) { + String platformInstance = datasetJsonObject.get(Constants.PLATFORM_INSTANCE_KEY).getAsString(); + String fieldUrn = Constants.SCHEMA_FIELD_URN_PREFIX + + Constants.PLATFORM_NAME + "," + platformInstance + "." + dataset + "," + environment + ")," + columnName + ")"; + fieldUrns.add(fieldUrn); + } + } + } + } + } + } + + return fieldUrns; + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/impl/QueryMCPTransformerServiceImpl.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/impl/QueryMCPTransformerServiceImpl.java new file mode 100644 index 00000000000000..c7ac3271668427 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/service/impl/QueryMCPTransformerServiceImpl.java @@ -0,0 +1,117 @@ +package datahub.hive.consumer.service.impl; + +import com.google.gson.JsonObject; +import com.linkedin.common.AuditStamp; +import com.linkedin.common.FabricType; +import com.linkedin.common.urn.DataPlatformUrn; +import com.linkedin.common.urn.DatasetUrn; +import com.linkedin.common.urn.Urn; +import com.linkedin.data.DataMap; +import com.linkedin.data.template.DataTemplate; +import com.linkedin.query.QueryLanguage; +import com.linkedin.query.QueryProperties; +import com.linkedin.query.QuerySource; +import com.linkedin.query.QueryStatement; +import com.linkedin.query.QuerySubject; +import com.linkedin.query.QuerySubjectArray; +import com.linkedin.query.QuerySubjects; + +import datahub.hive.consumer.config.Constants; +import datahub.hive.consumer.service.MCPTransformerService; +import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.stereotype.Service; + +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.List; + +/** + * Implementation of MCPTransformerService for Query entities. + */ +@Service +@Slf4j +public class QueryMCPTransformerServiceImpl implements MCPTransformerService { + + @Value("${application.environment}") + private String environment; + + @Value("${application.service.user}") + private String serviceUser; + + @Override + public List> transformToMCP(JsonObject queryJsonObject) throws URISyntaxException { + List> aspects = new ArrayList<>(); + + // Build querySubjects aspect + aspects.add(buildQuerySubjectsAspect(queryJsonObject)); + + // Build queryProperties aspect + aspects.add(buildQueryPropertiesAspect(queryJsonObject)); + + return aspects; + } + + /** + * Build the query subjects aspect for the query. + */ + private DataTemplate buildQuerySubjectsAspect(JsonObject queryJsonObject) { + QuerySubjects querySubjects = new QuerySubjects(); + List subjects = new ArrayList<>(); + + // Add all output dataset as subjects + String datasetName; + if (queryJsonObject.has(Constants.OUTPUTS_KEY) && !queryJsonObject.getAsJsonArray(Constants.OUTPUTS_KEY).isEmpty()) { + String fullName = queryJsonObject.getAsJsonArray(Constants.OUTPUTS_KEY).get(0).getAsString(); + String platformInstance = queryJsonObject.get(Constants.PLATFORM_INSTANCE_KEY).getAsString(); + datasetName = platformInstance + "." + fullName; + + // Create subject for the output dataset + QuerySubject subject = new QuerySubject(); + + // Create dataset URN + DatasetUrn datasetUrn = new DatasetUrn( + new DataPlatformUrn(Constants.PLATFORM_NAME), + datasetName, + FabricType.valueOf(environment) + ); + + subject.setEntity(datasetUrn); + subjects.add(subject); + + querySubjects.setSubjects(new QuerySubjectArray(subjects)); + } + + return querySubjects; + } + + /** + * Build the query properties aspect for the query. + */ + private DataTemplate buildQueryPropertiesAspect(JsonObject queryJsonObject) throws URISyntaxException { + QueryProperties queryProperties = new QueryProperties(); + + // Set the query statement + QueryStatement statement = new QueryStatement(); + statement.setValue(queryJsonObject.get(Constants.QUERY_TEXT_KEY).getAsString()); + statement.setLanguage(QueryLanguage.SQL); + queryProperties.setStatement(statement); + + // Set the created and last modified timestamps + long timestamp = System.currentTimeMillis(); + + AuditStamp created = new AuditStamp(); + created.setTime(timestamp); + created.setActor(Urn.createFromString(Constants.CORP_USER_URN_PREFIX + serviceUser)); + queryProperties.setCreated(created); + + AuditStamp lastModified = new AuditStamp(); + lastModified.setTime(timestamp); + lastModified.setActor(Urn.createFromString(Constants.CORP_USER_URN_PREFIX + serviceUser)); + queryProperties.setLastModified(lastModified); + + queryProperties.setSource(QuerySource.MANUAL); + + return queryProperties; + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/util/MCPEmitterUtil.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/util/MCPEmitterUtil.java new file mode 100644 index 00000000000000..bb738cfb49baab --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/util/MCPEmitterUtil.java @@ -0,0 +1,94 @@ +package datahub.hive.consumer.util; + +import com.linkedin.data.template.DataTemplate; +import com.linkedin.data.DataMap; +import datahub.client.kafka.KafkaEmitter; +import datahub.client.Callback; +import datahub.client.MetadataWriteResponse; +import datahub.event.MetadataChangeProposalWrapper; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.text.StringEscapeUtils; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +/** + * Utility class for emitting Metadata Change Proposals (MCPs) to DataHub. + */ +@Slf4j +public class MCPEmitterUtil { + + private MCPEmitterUtil() { + // Private constructor to prevent instantiation + } + + /** + * Emits a Metadata Change Proposal (MCP) to DataHub via Kafka. + * + * @param aspect The aspect to emit + * @param entityUrn The URN of entity + * @param entityType The type of entity + * @param kafkaEmitter The KafkaEmitter to use for sending messages + * @throws IOException If an I/O error occurs + * @throws URISyntaxException If a URI syntax error occurs + */ + public static void emitMCP(DataTemplate aspect, String entityUrn, String entityType, + KafkaEmitter kafkaEmitter) throws IOException, URISyntaxException { + + // Create the MCP wrapper + MetadataChangeProposalWrapper mcpw = MetadataChangeProposalWrapper.builder() + .entityType(entityType) + .entityUrn(entityUrn) + .upsert() + .aspect(aspect) + .build(); + + try { + long startTime = System.currentTimeMillis(); + Callback callback = new Callback() { + @Override + public void onCompletion(MetadataWriteResponse response) { + if (response.isSuccess()) { + long duration = TimeUtils.calculateDuration(startTime); + log.info("Successfully emitted metadata change event for aspect {} for {}. Time taken: {} ms", + StringEscapeUtils.escapeJava(mcpw.getAspectName()), + StringEscapeUtils.escapeJava(mcpw.getEntityUrn()), + duration); + } else { + log.error("Failed to emit metadata change event for {}, aspect: {} due to {}", + entityUrn, StringEscapeUtils.escapeJava(mcpw.getAspectName()), response.getResponseContent()); + } + } + + @Override + public void onFailure(Throwable throwable) { + log.error("Failed to emit metadata change event for {}, aspect: {} due to {}", + entityUrn, mcpw.getAspectName(), throwable.getMessage(), throwable); + } + }; + + log.info("Emitting MCP for entity: {}, aspect: {}", + StringEscapeUtils.escapeJava(mcpw.getEntityUrn()), + StringEscapeUtils.escapeJava(mcpw.getAspectName())); + + Future future = kafkaEmitter.emit(mcpw, callback); + MetadataWriteResponse response = future.get(); + + if (!response.isSuccess()) { + log.error("Failed to emit MCP: {}", response.getResponseContent()); + throw new IOException("Failed to emit MCP: " + response.getResponseContent()); + } else { + log.info("Successfully emitted MCP for entity: {}, aspect: {}", + StringEscapeUtils.escapeJava(mcpw.getEntityUrn()), + StringEscapeUtils.escapeJava(mcpw.getAspectName())); + } + } catch (InterruptedException | ExecutionException e) { + log.error("Failed to emit metadata change event for {}, aspect: {} due to {}", + entityUrn, mcpw.getAspectName(), e.getMessage()); + Thread.currentThread().interrupt(); + throw new IOException("Error emitting MCP", e); + } + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/util/TimeUtils.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/util/TimeUtils.java new file mode 100644 index 00000000000000..5abe30a315df93 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/java/datahub/hive/consumer/util/TimeUtils.java @@ -0,0 +1,21 @@ +package datahub.hive.consumer.util; + +/** + * Utility class for time-related operations. + */ +public class TimeUtils { + + private TimeUtils() { + // Private constructor to prevent instantiation + } + + /** + * Calculates and returns the duration in milliseconds between a start time and the current time. + * + * @param startTime The start time in milliseconds + * @return The duration in milliseconds + */ + public static long calculateDuration(long startTime) { + return System.currentTimeMillis() - startTime; + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/resources/application.properties b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/resources/application.properties new file mode 100644 index 00000000000000..0748f9516ef160 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/resources/application.properties @@ -0,0 +1,32 @@ +# Application Configuration +application.environment=${APPLICATION_ENVIRONMENT:DEV} +application.service.user=${APPLICATION_SERVICE_USER:application_user} + +# Spring Kafka Consumer Configuration +spring.kafka.consumer.hive.lineage.topic=${KAFKA_HIVE_LINEAGE_TOPIC:HiveLineage_v1} +spring.kafka.consumer.bootstrap-servers=${KAFKA_BOOTSTRAP_SERVERS:localhost:9092} +spring.kafka.consumer.group-id=${KAFKA_CONSUMER_GROUP_ID:lineage-consumer-group} +spring.kafka.consumer.auto-offset-reset=${KAFKA_CONSUMER_AUTO_OFFSET_RESET:earliest} +spring.kafka.consumer.key-deserializer=${KAFKA_CONSUMER_KEY_DESERIALIZER:org.apache.kafka.common.serialization.StringDeserializer} +spring.kafka.consumer.value-deserializer=${KAFKA_CONSUMER_VALUE_DESERIALIZER:org.apache.kafka.common.serialization.StringDeserializer} +spring.kafka.consumer.concurrency=${KAFKA_CONSUMER_CONCURRENCY:7} +spring.kafka.consumer.retry.max-attempts=${KAFKA_CONSUMER_RETRY_MAX_ATTEMPTS:3} +spring.kafka.consumer.retry.initial-delay=${KAFKA_CONSUMER_RETRY_INITIAL_DELAY:1000} + +# Spring Kafka Producer Configuration +spring.kafka.producer.properties.schema.registry.url=${KAFKA_SCHEMA_REGISTRY_URL:http://localhost/schema-registry/api/} +spring.kafka.producer.properties.mcp.topic=${KAFKA_MCP_TOPIC:MetadataChangeProposal_v1} +spring.kafka.producer.bootstrap-servers=${KAFKA_BOOTSTRAP_SERVERS:localhost:9092} +spring.kafka.producer.properties.retries=${KAFKA_PRODUCER_RETRIES:3} +spring.kafka.producer.properties.retry.backoff.ms=${KAFKA_PRODUCER_RETRY_BACKOFF_MS:100} +spring.kafka.producer.properties.enable.idempotence=${KAFKA_PRODUCER_ENABLE_IDEMPOTENCE:true} +spring.kafka.producer.properties.max.block.ms=${KAFKA_PRODUCER_MAX_BLOCK_MS:3000} +spring.kafka.producer.properties.request.timeout.ms=${KAFKA_PRODUCER_REQUEST_TIMEOUT_MS:3000} +spring.kafka.producer.properties.delivery.timeout.ms=${KAFKA_PRODUCER_DELIVERY_TIMEOUT_MS:5000} + +# SSL configuration for Spring Kafka +spring.kafka.security.protocol=${KAFKA_SECURITY_PROTOCOL:SSL} +spring.kafka.ssl.trust-store-location=file:${KAFKA_SSL_TRUSTSTORE_LOCATION:/path/to/truststore.jks} +spring.kafka.ssl.trust-store-password=${KAFKA_SSL_TRUSTSTORE_PASSWORD:pass} +spring.kafka.ssl.key-store-location=file:${KAFKA_SSL_KEYSTORE_LOCATION:/path/to/keystore.jks} +spring.kafka.ssl.key-store-password=${KAFKA_SSL_KEYSTORE_PASSWORD:pass} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/resources/logback.xml b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/resources/logback.xml new file mode 100644 index 00000000000000..0e976beb44eb47 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/main/resources/logback.xml @@ -0,0 +1,60 @@ + + + + + + %date{ISO8601} [%thread] %-5level %logger{36}:%L - %msg%n + + + INFO + + + + + ${LOG_DIR}/hive-lineage-consumer.log + + ${LOG_DIR}/hive-lineage-consumer.%d{yyyy-dd-MM}-%i.log + + 100MB + + 10GB + + 30 + + + ${logging.appender.error_file.level:-INFO} + ACCEPT + DENY + + + %date{ISO8601} [%thread] %-5level %logger{36}:%L - %msg%n + + + + + ${LOG_DIR}/hive-lineage-consumer.error.log + + ${LOG_DIR}/hive-lineage-consumer.error.%d{yyyy-dd-MM}-%i.log + + 100MB + + 10GB + + 30 + + + ${logging.appender.error_file.level:-ERROR} + ACCEPT + DENY + + + %date{ISO8601} [%thread] %-5level %logger{36}:%L - %msg%n + + + + + + + + + diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/config/KafkaConfigIntegrationTest.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/config/KafkaConfigIntegrationTest.java new file mode 100644 index 00000000000000..810f5ceec3d173 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/config/KafkaConfigIntegrationTest.java @@ -0,0 +1,172 @@ +package datahub.hive.consumer.config; + +import org.junit.jupiter.api.Test; +import org.springframework.boot.autoconfigure.kafka.KafkaProperties; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.*; + +/** + * Integration tests for KafkaConfig focusing on kafkaEmitter() method. + * Tests the configuration building and exception handling paths. + */ +public class KafkaConfigIntegrationTest { + + /** + * Tests kafkaEmitter() method with minimal configuration. + */ + @Test + void testKafkaEmitter_MinimalConfig() { + // Create real KafkaProperties with minimal configuration + KafkaProperties kafkaProperties = new KafkaProperties(); + + // Set up producer properties + kafkaProperties.getProducer().setBootstrapServers(Arrays.asList("localhost:9092")); + + Map producerProps = new HashMap<>(); + producerProps.put("schema.registry.url", "http://localhost:8081"); + producerProps.put("mcp.topic", "MetadataChangeProposal_v1"); + kafkaProperties.getProducer().getProperties().putAll(producerProps); + + // Set up security without SSL to avoid file issues + kafkaProperties.getSecurity().setProtocol("PLAINTEXT"); + + // Create KafkaConfig + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + + assertThrows(Exception.class, () -> { + kafkaConfig.kafkaEmitter(); + }); + } + + /** + * Tests kafkaEmitter() method with multiple bootstrap servers. + * This should cover the bootstrap servers joining logic. + */ + @Test + void testKafkaEmitter_MultipleBootstrapServers() { + KafkaProperties kafkaProperties = new KafkaProperties(); + + // Set up multiple bootstrap servers + kafkaProperties.getProducer().setBootstrapServers(Arrays.asList( + "broker1:9092", "broker2:9092", "broker3:9092" + )); + + Map producerProps = new HashMap<>(); + producerProps.put("schema.registry.url", "http://localhost:8081"); + producerProps.put("mcp.topic", "MetadataChangeProposal_v1"); + kafkaProperties.getProducer().getProperties().putAll(producerProps); + + kafkaProperties.getSecurity().setProtocol("PLAINTEXT"); + + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + + // This should cover the String.join() operation for bootstrap servers + assertThrows(Exception.class, () -> { + kafkaConfig.kafkaEmitter(); + }); + } + + /** + * Tests kafkaEmitter() method with additional producer properties. + * This should cover the putAll operation with multiple properties. + */ + @Test + void testKafkaEmitter_AdditionalProducerProperties() { + KafkaProperties kafkaProperties = new KafkaProperties(); + + kafkaProperties.getProducer().setBootstrapServers(Arrays.asList("localhost:9092")); + + // Set up producer properties with additional configs + Map producerProps = new HashMap<>(); + producerProps.put("schema.registry.url", "http://localhost:8081"); + producerProps.put("mcp.topic", "MetadataChangeProposal_v1"); + producerProps.put("acks", "all"); + producerProps.put("retries", "3"); + producerProps.put("batch.size", "16384"); + producerProps.put("linger.ms", "5"); + kafkaProperties.getProducer().getProperties().putAll(producerProps); + + kafkaProperties.getSecurity().setProtocol("PLAINTEXT"); + + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + + assertThrows(Exception.class, () -> { + kafkaConfig.kafkaEmitter(); + }); + } + + /** + * Tests kafkaEmitter() method with empty producer properties. + */ + @Test + void testKafkaEmitter_EmptyProducerProperties() { + KafkaProperties kafkaProperties = new KafkaProperties(); + + kafkaProperties.getProducer().setBootstrapServers(Arrays.asList("localhost:9092")); + + // Empty producer properties + kafkaProperties.getSecurity().setProtocol("PLAINTEXT"); + + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + + assertThrows(Exception.class, () -> { + kafkaConfig.kafkaEmitter(); + }); + } + + /** + * Tests KafkaConfig constructor with different properties. + * Verifies that the config can be instantiated with various properties. + */ + @Test + void testKafkaConfigInstantiation() { + KafkaProperties kafkaProperties = new KafkaProperties(); + + // Test constructor + assertDoesNotThrow(() -> { + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + assertNotNull(kafkaConfig); + }); + } + + /** + * Tests KafkaConfig with null properties. + * Verifies handling of null KafkaProperties. + */ + @Test + void testKafkaConfigWithNullProperties() { + // Test constructor with null properties + assertDoesNotThrow(() -> { + KafkaConfig kafkaConfig = new KafkaConfig(null); + assertNotNull(kafkaConfig); + }); + } + + /** + * Tests kafkaEmitter() method with different security protocols. + */ + @Test + void testKafkaEmitter_DifferentSecurityProtocols() { + KafkaProperties kafkaProperties = new KafkaProperties(); + + kafkaProperties.getProducer().setBootstrapServers(Arrays.asList("localhost:9092")); + + Map producerProps = new HashMap<>(); + producerProps.put("schema.registry.url", "http://localhost:8081"); + producerProps.put("mcp.topic", "MetadataChangeProposal_v1"); + kafkaProperties.getProducer().getProperties().putAll(producerProps); + + // Test with SASL_SSL protocol + kafkaProperties.getSecurity().setProtocol("SASL_SSL"); + + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + + assertThrows(Exception.class, () -> { + kafkaConfig.kafkaEmitter(); + }); + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/config/KafkaConfigSSLTest.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/config/KafkaConfigSSLTest.java new file mode 100644 index 00000000000000..d953602206d4b8 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/config/KafkaConfigSSLTest.java @@ -0,0 +1,164 @@ +package datahub.hive.consumer.config; + +import org.junit.jupiter.api.Test; +import org.springframework.boot.autoconfigure.kafka.KafkaProperties; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.*; + +/** + * Tests for KafkaConfig focusing on kafkaEmitter() method coverage. + */ +public class KafkaConfigSSLTest { + + /** + * Tests kafkaEmitter() method with multiple bootstrap servers. + */ + @Test + void testKafkaEmitter_MultipleBootstrapServers() { + KafkaProperties kafkaProperties = new KafkaProperties(); + + // Multiple bootstrap servers + kafkaProperties.getProducer().setBootstrapServers(Arrays.asList( + "broker1:9092", "broker2:9092", "broker3:9092" + )); + + Map producerProps = new HashMap<>(); + producerProps.put("schema.registry.url", "http://localhost:8081"); + producerProps.put("mcp.topic", "MetadataChangeProposal_v1"); + kafkaProperties.getProducer().getProperties().putAll(producerProps); + + kafkaProperties.getSecurity().setProtocol("PLAINTEXT"); + + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + + assertThrows(Exception.class, () -> { + kafkaConfig.kafkaEmitter(); + }); + } + + /** + * Tests kafkaEmitter() method with single bootstrap server. + */ + @Test + void testKafkaEmitter_SingleBootstrapServer() { + KafkaProperties kafkaProperties = new KafkaProperties(); + + kafkaProperties.getProducer().setBootstrapServers(Arrays.asList("single-broker:9092")); + + Map producerProps = new HashMap<>(); + producerProps.put("schema.registry.url", "http://localhost:8081"); + producerProps.put("mcp.topic", "MetadataChangeProposal_v1"); + kafkaProperties.getProducer().getProperties().putAll(producerProps); + + kafkaProperties.getSecurity().setProtocol("PLAINTEXT"); + + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + + assertThrows(Exception.class, () -> { + kafkaConfig.kafkaEmitter(); + }); + } + + /** + * Tests kafkaEmitter() method with different schema registry URLs. + */ + @Test + void testKafkaEmitter_DifferentSchemaRegistryUrl() { + KafkaProperties kafkaProperties = new KafkaProperties(); + + kafkaProperties.getProducer().setBootstrapServers(Arrays.asList("localhost:9092")); + + // Different schema registry URL + Map producerProps = new HashMap<>(); + producerProps.put("schema.registry.url", "http://schema-registry:8081"); + producerProps.put("mcp.topic", "MetadataChangeProposal_v1"); + kafkaProperties.getProducer().getProperties().putAll(producerProps); + + kafkaProperties.getSecurity().setProtocol("PLAINTEXT"); + + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + + assertThrows(Exception.class, () -> { + kafkaConfig.kafkaEmitter(); + }); + } + + /** + * Tests kafkaEmitter() method with different MCP topic. + */ + @Test + void testKafkaEmitter_DifferentMcpTopic() { + KafkaProperties kafkaProperties = new KafkaProperties(); + + kafkaProperties.getProducer().setBootstrapServers(Arrays.asList("localhost:9092")); + + // Different MCP topic + Map producerProps = new HashMap<>(); + producerProps.put("schema.registry.url", "http://localhost:8081"); + producerProps.put("mcp.topic", "CustomMetadataChangeProposal"); + kafkaProperties.getProducer().getProperties().putAll(producerProps); + + kafkaProperties.getSecurity().setProtocol("PLAINTEXT"); + + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + + assertThrows(Exception.class, () -> { + kafkaConfig.kafkaEmitter(); + }); + } + + /** + * Tests kafkaEmitter() method with additional producer properties. + */ + @Test + void testKafkaEmitter_AdditionalProducerProperties() { + KafkaProperties kafkaProperties = new KafkaProperties(); + + kafkaProperties.getProducer().setBootstrapServers(Arrays.asList("localhost:9092")); + + // Multiple producer properties + Map producerProps = new HashMap<>(); + producerProps.put("schema.registry.url", "http://localhost:8081"); + producerProps.put("mcp.topic", "MetadataChangeProposal_v1"); + producerProps.put("acks", "all"); + producerProps.put("retries", "3"); + producerProps.put("batch.size", "16384"); + kafkaProperties.getProducer().getProperties().putAll(producerProps); + + kafkaProperties.getSecurity().setProtocol("PLAINTEXT"); + + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + + assertThrows(Exception.class, () -> { + kafkaConfig.kafkaEmitter(); + }); + } + + /** + * Tests kafkaEmitter() method with SASL_SSL protocol. + */ + @Test + void testKafkaEmitter_SASLSSLProtocol() { + KafkaProperties kafkaProperties = new KafkaProperties(); + + kafkaProperties.getProducer().setBootstrapServers(Arrays.asList("localhost:9092")); + + Map producerProps = new HashMap<>(); + producerProps.put("schema.registry.url", "http://localhost:8081"); + producerProps.put("mcp.topic", "MetadataChangeProposal_v1"); + kafkaProperties.getProducer().getProperties().putAll(producerProps); + + // SASL_SSL protocol + kafkaProperties.getSecurity().setProtocol("SASL_SSL"); + + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); + + assertThrows(Exception.class, () -> { + kafkaConfig.kafkaEmitter(); + }); + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/config/KafkaConfigTest.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/config/KafkaConfigTest.java new file mode 100644 index 00000000000000..8aad07aca459ef --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/config/KafkaConfigTest.java @@ -0,0 +1,230 @@ +package datahub.hive.consumer.config; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.springframework.boot.autoconfigure.kafka.KafkaProperties; +import org.springframework.kafka.config.ConcurrentKafkaListenerContainerFactory; +import org.springframework.kafka.core.ConsumerFactory; +import org.springframework.kafka.listener.ContainerProperties; +import org.springframework.kafka.listener.DefaultErrorHandler; +import org.springframework.test.util.ReflectionTestUtils; + + +import static org.junit.jupiter.api.Assertions.*; + + +/** + * Unit tests for KafkaConfig. + * Tests the configuration of Kafka consumer and producer beans. + */ +@ExtendWith(MockitoExtension.class) +public class KafkaConfigTest { + + @Mock + private KafkaProperties kafkaProperties; + + @Mock + private ConsumerFactory consumerFactory; + + @InjectMocks + private KafkaConfig kafkaConfig; + + @BeforeEach + void setUp() { + // Set up test properties using ReflectionTestUtils + ReflectionTestUtils.setField(kafkaConfig, "consumerConcurrency", 7); + ReflectionTestUtils.setField(kafkaConfig, "maxRetryAttempts", 3); + ReflectionTestUtils.setField(kafkaConfig, "delay", 1000); + } + + /** + * Tests successful KafkaListenerContainerFactory bean creation. + * Verifies that the container factory is configured correctly. + */ + @Test + void testKafkaListenerContainerFactory_Success() { + // Call the method under test + ConcurrentKafkaListenerContainerFactory factory = + kafkaConfig.kafkaListenerContainerFactory(consumerFactory); + + // Verify that factory is created + assertNotNull(factory); + + // Verify factory configuration + assertEquals(consumerFactory, factory.getConsumerFactory()); + assertEquals(ContainerProperties.AckMode.RECORD, factory.getContainerProperties().getAckMode()); + + // Verify concurrency is set + Object concurrency = ReflectionTestUtils.getField(factory, "concurrency"); + assertEquals(7, concurrency); + + // Verify error handler is set + Object errorHandler = ReflectionTestUtils.getField(factory, "commonErrorHandler"); + assertNotNull(errorHandler); + assertTrue(errorHandler instanceof DefaultErrorHandler); + } + + /** + * Tests KafkaListenerContainerFactory with different concurrency setting. + * Verifies that concurrency configuration is applied correctly. + */ + @Test + void testKafkaListenerContainerFactory_DifferentConcurrency() { + // Set different concurrency + ReflectionTestUtils.setField(kafkaConfig, "consumerConcurrency", 10); + + // Call the method under test + ConcurrentKafkaListenerContainerFactory factory = + kafkaConfig.kafkaListenerContainerFactory(consumerFactory); + + // Verify concurrency setting using reflection + Object concurrency = ReflectionTestUtils.getField(factory, "concurrency"); + assertEquals(10, concurrency); + } + + /** + * Tests KafkaListenerContainerFactory with different retry settings. + * Verifies that retry configuration is applied correctly. + */ + @Test + void testKafkaListenerContainerFactory_DifferentRetrySettings() { + // Set different retry settings + ReflectionTestUtils.setField(kafkaConfig, "maxRetryAttempts", 5); + ReflectionTestUtils.setField(kafkaConfig, "delay", 2000); + + // Call the method under test + ConcurrentKafkaListenerContainerFactory factory = + kafkaConfig.kafkaListenerContainerFactory(consumerFactory); + + // Verify that factory is created with error handler + assertNotNull(factory); + Object errorHandler = ReflectionTestUtils.getField(factory, "commonErrorHandler"); + assertNotNull(errorHandler); + assertTrue(errorHandler instanceof DefaultErrorHandler); + } + + /** + * Tests KafkaListenerContainerFactory with minimum retry attempts. + * Verifies that minimum retry configuration works. + */ + @Test + void testKafkaListenerContainerFactory_MinimumRetryAttempts() { + // Set minimum retry attempts + ReflectionTestUtils.setField(kafkaConfig, "maxRetryAttempts", 1); + ReflectionTestUtils.setField(kafkaConfig, "delay", 500); + + // Call the method under test + ConcurrentKafkaListenerContainerFactory factory = + kafkaConfig.kafkaListenerContainerFactory(consumerFactory); + + // Verify that factory is created successfully + assertNotNull(factory); + Object errorHandler = ReflectionTestUtils.getField(factory, "commonErrorHandler"); + assertNotNull(errorHandler); + } + + /** + * Tests KafkaListenerContainerFactory with null consumer factory. + * Verifies that null consumer factory is handled appropriately. + */ + @Test + void testKafkaListenerContainerFactory_NullConsumerFactory() { + // Call the method under test with null consumer factory + ConcurrentKafkaListenerContainerFactory factory = + kafkaConfig.kafkaListenerContainerFactory(null); + + // Verify that factory is still created + assertNotNull(factory); + assertNull(factory.getConsumerFactory()); + } + + /** + * Tests KafkaListenerContainerFactory with maximum concurrency. + * Verifies that high concurrency values are handled correctly. + */ + @Test + void testKafkaListenerContainerFactory_MaxConcurrency() { + // Set high concurrency + ReflectionTestUtils.setField(kafkaConfig, "consumerConcurrency", 20); + + // Call the method under test + ConcurrentKafkaListenerContainerFactory factory = + kafkaConfig.kafkaListenerContainerFactory(consumerFactory); + + // Verify concurrency setting + Object concurrency = ReflectionTestUtils.getField(factory, "concurrency"); + assertEquals(20, concurrency); + } + + /** + * Tests KafkaListenerContainerFactory with zero delay. + * Verifies that zero delay configuration works. + */ + @Test + void testKafkaListenerContainerFactory_ZeroDelay() { + // Set zero delay + ReflectionTestUtils.setField(kafkaConfig, "delay", 0); + + // Call the method under test + ConcurrentKafkaListenerContainerFactory factory = + kafkaConfig.kafkaListenerContainerFactory(consumerFactory); + + // Verify that factory is created successfully + assertNotNull(factory); + Object errorHandler = ReflectionTestUtils.getField(factory, "commonErrorHandler"); + assertNotNull(errorHandler); + } + + /** + * Tests KafkaListenerContainerFactory with high retry attempts. + * Verifies that high retry configuration works. + */ + @Test + void testKafkaListenerContainerFactory_HighRetryAttempts() { + // Set high retry attempts + ReflectionTestUtils.setField(kafkaConfig, "maxRetryAttempts", 10); + ReflectionTestUtils.setField(kafkaConfig, "delay", 5000); + + // Call the method under test + ConcurrentKafkaListenerContainerFactory factory = + kafkaConfig.kafkaListenerContainerFactory(consumerFactory); + + // Verify that factory is created successfully + assertNotNull(factory); + Object errorHandler = ReflectionTestUtils.getField(factory, "commonErrorHandler"); + assertNotNull(errorHandler); + assertTrue(errorHandler instanceof DefaultErrorHandler); + } + + /** + * Tests KafkaListenerContainerFactory container properties. + * Verifies that container properties are set correctly. + */ + @Test + void testKafkaListenerContainerFactory_ContainerProperties() { + // Call the method under test + ConcurrentKafkaListenerContainerFactory factory = + kafkaConfig.kafkaListenerContainerFactory(consumerFactory); + + // Verify container properties + assertNotNull(factory.getContainerProperties()); + assertEquals(ContainerProperties.AckMode.RECORD, factory.getContainerProperties().getAckMode()); + } + + /** + * Tests KafkaConfig constructor with KafkaProperties. + * Verifies that the constructor properly initializes the config. + */ + @Test + void testKafkaConfigConstructor() { + // Create a new KafkaConfig with mocked properties + KafkaConfig config = new KafkaConfig(kafkaProperties); + + // Verify that the config is created successfully + assertNotNull(config); + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/model/HiveLineageTest.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/model/HiveLineageTest.java new file mode 100644 index 00000000000000..20363fccf5a654 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/model/HiveLineageTest.java @@ -0,0 +1,413 @@ +package datahub.hive.consumer.model; + +import com.google.gson.Gson; +import com.google.gson.JsonSyntaxException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.*; + +/** + * Unit tests for HiveLineage model class. + * Tests serialization, deserialization, and data integrity of the model. + */ +public class HiveLineageTest { + + private Gson gson; + private String validHiveLineageJson; + private HiveLineage validHiveLineage; + + @BeforeEach + void setUp() { + gson = new Gson(); + + // Create a valid HiveLineage object for testing + validHiveLineage = new HiveLineage(); + validHiveLineage.setVersion("1.0"); + validHiveLineage.setUser("testuser"); + validHiveLineage.setTimestamp(1234567890L); + validHiveLineage.setDuration(5000L); + validHiveLineage.setJobIds(Arrays.asList("job1", "job2")); + validHiveLineage.setEngine("hive"); + validHiveLineage.setDatabase("testdb"); + validHiveLineage.setHash("testhash123"); + validHiveLineage.setQueryText("CREATE TABLE testdb.output_table AS SELECT * FROM testdb.input_table"); + validHiveLineage.setEnvironment("DEV"); + validHiveLineage.setPlatformInstance("urn:li:dataPlatformInstance:(urn:li:dataPlatform:hive,OPR.OCE.DEV)"); + validHiveLineage.setInputs(Arrays.asList("testdb.input_table")); + validHiveLineage.setOutputs(Arrays.asList("testdb.output_table")); + + // Create edges + HiveLineage.Edge edge = new HiveLineage.Edge(); + edge.setSources(Arrays.asList(0)); + edge.setTargets(Arrays.asList(1)); + edge.setExpression("col1"); + edge.setEdgeType("PROJECTION"); + validHiveLineage.setEdges(Arrays.asList(edge)); + + // Create vertices + HiveLineage.Vertex vertex1 = new HiveLineage.Vertex(); + vertex1.setId(0); + vertex1.setVertexType("COLUMN"); + vertex1.setVertexId("input_table.col1"); + + HiveLineage.Vertex vertex2 = new HiveLineage.Vertex(); + vertex2.setId(1); + vertex2.setVertexType("COLUMN"); + vertex2.setVertexId("output_table.col1"); + + validHiveLineage.setVertices(Arrays.asList(vertex1, vertex2)); + + // Create corresponding JSON string + validHiveLineageJson = "{" + + "\"version\":\"1.0\"," + + "\"user\":\"testuser\"," + + "\"timestamp\":1234567890," + + "\"duration\":5000," + + "\"jobIds\":[\"job1\",\"job2\"]," + + "\"engine\":\"hive\"," + + "\"database\":\"testdb\"," + + "\"hash\":\"testhash123\"," + + "\"queryText\":\"CREATE TABLE testdb.output_table AS SELECT * FROM testdb.input_table\"," + + "\"environment\":\"DEV\"," + + "\"platformInstance\":\"urn:li:dataPlatformInstance:(urn:li:dataPlatform:hive,OPR.OCE.DEV)\"," + + "\"inputs\":[\"testdb.input_table\"]," + + "\"outputs\":[\"testdb.output_table\"]," + + "\"edges\":[{\"sources\":[0],\"targets\":[1],\"expression\":\"col1\",\"edgeType\":\"PROJECTION\"}]," + + "\"vertices\":[{\"id\":0,\"vertexType\":\"COLUMN\",\"vertexId\":\"input_table.col1\"},{\"id\":1,\"vertexType\":\"COLUMN\",\"vertexId\":\"output_table.col1\"}]" + + "}"; + } + + /** + * Tests successful deserialization of valid HiveLineage JSON. + * Verifies that all fields are correctly mapped from JSON. + */ + @Test + void testDeserialization_ValidJson() { + // Deserialize JSON to HiveLineage object + HiveLineage hiveLineage = gson.fromJson(validHiveLineageJson, HiveLineage.class); + + // Verify all fields are correctly deserialized + assertNotNull(hiveLineage); + assertEquals("1.0", hiveLineage.getVersion()); + assertEquals("testuser", hiveLineage.getUser()); + assertEquals(1234567890L, hiveLineage.getTimestamp()); + assertEquals(5000L, hiveLineage.getDuration()); + assertEquals(Arrays.asList("job1", "job2"), hiveLineage.getJobIds()); + assertEquals("hive", hiveLineage.getEngine()); + assertEquals("testdb", hiveLineage.getDatabase()); + assertEquals("testhash123", hiveLineage.getHash()); + assertEquals("CREATE TABLE testdb.output_table AS SELECT * FROM testdb.input_table", hiveLineage.getQueryText()); + assertEquals("DEV", hiveLineage.getEnvironment()); + assertEquals("urn:li:dataPlatformInstance:(urn:li:dataPlatform:hive,OPR.OCE.DEV)", hiveLineage.getPlatformInstance()); + assertEquals(Arrays.asList("testdb.input_table"), hiveLineage.getInputs()); + assertEquals(Arrays.asList("testdb.output_table"), hiveLineage.getOutputs()); + + // Verify edges + assertNotNull(hiveLineage.getEdges()); + assertEquals(1, hiveLineage.getEdges().size()); + HiveLineage.Edge edge = hiveLineage.getEdges().get(0); + assertEquals(Arrays.asList(0), edge.getSources()); + assertEquals(Arrays.asList(1), edge.getTargets()); + assertEquals("col1", edge.getExpression()); + assertEquals("PROJECTION", edge.getEdgeType()); + + // Verify vertices + assertNotNull(hiveLineage.getVertices()); + assertEquals(2, hiveLineage.getVertices().size()); + + HiveLineage.Vertex vertex1 = hiveLineage.getVertices().get(0); + assertEquals(0, vertex1.getId()); + assertEquals("COLUMN", vertex1.getVertexType()); + assertEquals("input_table.col1", vertex1.getVertexId()); + + HiveLineage.Vertex vertex2 = hiveLineage.getVertices().get(1); + assertEquals(1, vertex2.getId()); + assertEquals("COLUMN", vertex2.getVertexType()); + assertEquals("output_table.col1", vertex2.getVertexId()); + } + + /** + * Tests successful serialization of HiveLineage object to JSON. + * Verifies that all fields are correctly serialized. + */ + @Test + void testSerialization_ValidObject() { + // Serialize HiveLineage object to JSON + String json = gson.toJson(validHiveLineage); + + // Verify JSON is not null and contains expected fields + assertNotNull(json); + assertTrue(json.contains("\"version\":\"1.0\"")); + assertTrue(json.contains("\"user\":\"testuser\"")); + assertTrue(json.contains("\"timestamp\":1234567890")); + assertTrue(json.contains("\"duration\":5000")); + assertTrue(json.contains("\"jobIds\":[\"job1\",\"job2\"]")); + assertTrue(json.contains("\"engine\":\"hive\"")); + assertTrue(json.contains("\"database\":\"testdb\"")); + assertTrue(json.contains("\"hash\":\"testhash123\"")); + assertTrue(json.contains("\"queryText\":\"CREATE TABLE testdb.output_table AS SELECT * FROM testdb.input_table\"")); + assertTrue(json.contains("\"environment\":\"DEV\"")); + assertTrue(json.contains("\"platformInstance\":\"urn:li:dataPlatformInstance:(urn:li:dataPlatform:hive,OPR.OCE.DEV)\"")); + assertTrue(json.contains("\"inputs\":[\"testdb.input_table\"]")); + assertTrue(json.contains("\"outputs\":[\"testdb.output_table\"]")); + assertTrue(json.contains("\"edgeType\":\"PROJECTION\"")); + assertTrue(json.contains("\"vertexType\":\"COLUMN\"")); + } + + /** + * Tests round-trip serialization and deserialization. + * Verifies that data integrity is maintained through the process. + */ + @Test + void testRoundTripSerialization() { + // Serialize to JSON and then deserialize back + String json = gson.toJson(validHiveLineage); + HiveLineage deserializedHiveLineage = gson.fromJson(json, HiveLineage.class); + + // Verify all fields match the original + assertEquals(validHiveLineage.getVersion(), deserializedHiveLineage.getVersion()); + assertEquals(validHiveLineage.getUser(), deserializedHiveLineage.getUser()); + assertEquals(validHiveLineage.getTimestamp(), deserializedHiveLineage.getTimestamp()); + assertEquals(validHiveLineage.getDuration(), deserializedHiveLineage.getDuration()); + assertEquals(validHiveLineage.getJobIds(), deserializedHiveLineage.getJobIds()); + assertEquals(validHiveLineage.getEngine(), deserializedHiveLineage.getEngine()); + assertEquals(validHiveLineage.getDatabase(), deserializedHiveLineage.getDatabase()); + assertEquals(validHiveLineage.getHash(), deserializedHiveLineage.getHash()); + assertEquals(validHiveLineage.getQueryText(), deserializedHiveLineage.getQueryText()); + assertEquals(validHiveLineage.getEnvironment(), deserializedHiveLineage.getEnvironment()); + assertEquals(validHiveLineage.getPlatformInstance(), deserializedHiveLineage.getPlatformInstance()); + assertEquals(validHiveLineage.getInputs(), deserializedHiveLineage.getInputs()); + assertEquals(validHiveLineage.getOutputs(), deserializedHiveLineage.getOutputs()); + + // Verify edges + assertEquals(validHiveLineage.getEdges().size(), deserializedHiveLineage.getEdges().size()); + assertEquals(validHiveLineage.getEdges().get(0).getSources(), deserializedHiveLineage.getEdges().get(0).getSources()); + assertEquals(validHiveLineage.getEdges().get(0).getTargets(), deserializedHiveLineage.getEdges().get(0).getTargets()); + assertEquals(validHiveLineage.getEdges().get(0).getExpression(), deserializedHiveLineage.getEdges().get(0).getExpression()); + assertEquals(validHiveLineage.getEdges().get(0).getEdgeType(), deserializedHiveLineage.getEdges().get(0).getEdgeType()); + + // Verify vertices + assertEquals(validHiveLineage.getVertices().size(), deserializedHiveLineage.getVertices().size()); + assertEquals(validHiveLineage.getVertices().get(0).getId(), deserializedHiveLineage.getVertices().get(0).getId()); + assertEquals(validHiveLineage.getVertices().get(0).getVertexType(), deserializedHiveLineage.getVertices().get(0).getVertexType()); + assertEquals(validHiveLineage.getVertices().get(0).getVertexId(), deserializedHiveLineage.getVertices().get(0).getVertexId()); + } + + /** + * Tests deserialization with missing optional fields. + * Verifies that the model handles missing fields gracefully. + */ + @Test + void testDeserialization_MissingOptionalFields() { + // JSON with only required fields + String minimalJson = "{" + + "\"version\":\"1.0\"," + + "\"user\":\"testuser\"," + + "\"queryText\":\"SELECT * FROM table\"" + + "}"; + + // Deserialize and verify + HiveLineage hiveLineage = gson.fromJson(minimalJson, HiveLineage.class); + + assertNotNull(hiveLineage); + assertEquals("1.0", hiveLineage.getVersion()); + assertEquals("testuser", hiveLineage.getUser()); + assertEquals("SELECT * FROM table", hiveLineage.getQueryText()); + + // Optional fields should be null or default values + assertNull(hiveLineage.getJobIds()); + assertNull(hiveLineage.getEdges()); + assertNull(hiveLineage.getVertices()); + } + + /** + * Tests deserialization with invalid JSON. + * Verifies that appropriate exceptions are thrown for malformed JSON. + */ + @Test + void testDeserialization_InvalidJson() { + String invalidJson = "{ invalid json }"; + + // Should throw JsonSyntaxException for invalid JSON + assertThrows(JsonSyntaxException.class, () -> { + gson.fromJson(invalidJson, HiveLineage.class); + }); + } + + /** + * Tests deserialization with empty JSON object. + * Verifies that empty JSON creates a valid but empty HiveLineage object. + */ + @Test + void testDeserialization_EmptyJson() { + String emptyJson = "{}"; + + // Deserialize empty JSON + HiveLineage hiveLineage = gson.fromJson(emptyJson, HiveLineage.class); + + // Should create a valid object with null/default values + assertNotNull(hiveLineage); + assertNull(hiveLineage.getVersion()); + assertNull(hiveLineage.getUser()); + assertNull(hiveLineage.getQueryText()); + } + + /** + * Tests Edge nested class functionality. + * Verifies that Edge objects work correctly with Lombok annotations. + */ + @Test + void testEdgeClass() { + HiveLineage.Edge edge1 = new HiveLineage.Edge(); + edge1.setSources(Arrays.asList(1, 2)); + edge1.setTargets(Arrays.asList(3, 4)); + edge1.setExpression("test_expression"); + edge1.setEdgeType("DEPENDENCY"); + + HiveLineage.Edge edge2 = new HiveLineage.Edge(); + edge2.setSources(Arrays.asList(1, 2)); + edge2.setTargets(Arrays.asList(3, 4)); + edge2.setExpression("test_expression"); + edge2.setEdgeType("DEPENDENCY"); + + // Test getters + assertEquals(Arrays.asList(1, 2), edge1.getSources()); + assertEquals(Arrays.asList(3, 4), edge1.getTargets()); + assertEquals("test_expression", edge1.getExpression()); + assertEquals("DEPENDENCY", edge1.getEdgeType()); + + // Test Lombok generated equals and hashCode + assertEquals(edge1, edge2); + assertEquals(edge1.hashCode(), edge2.hashCode()); + + // Test toString (Lombok generated) + assertNotNull(edge1.toString()); + assertTrue(edge1.toString().contains("Edge")); + } + + /** + * Tests Vertex nested class functionality. + * Verifies that Vertex objects work correctly with Lombok annotations. + */ + @Test + void testVertexClass() { + HiveLineage.Vertex vertex1 = new HiveLineage.Vertex(); + vertex1.setId(5); + vertex1.setVertexType("TABLE"); + vertex1.setVertexId("test_table"); + + HiveLineage.Vertex vertex2 = new HiveLineage.Vertex(); + vertex2.setId(5); + vertex2.setVertexType("TABLE"); + vertex2.setVertexId("test_table"); + + // Test getters + assertEquals(5, vertex1.getId()); + assertEquals("TABLE", vertex1.getVertexType()); + assertEquals("test_table", vertex1.getVertexId()); + + // Test Lombok generated equals and hashCode + assertEquals(vertex1, vertex2); + assertEquals(vertex1.hashCode(), vertex2.hashCode()); + + // Test toString (Lombok generated) + assertNotNull(vertex1.toString()); + assertTrue(vertex1.toString().contains("Vertex")); + } + + /** + * Tests HiveLineage class Lombok functionality. + * Verifies that Lombok annotations work correctly for the main class. + */ + @Test + void testHiveLineageLombokFunctionality() { + HiveLineage hiveLineage1 = new HiveLineage(); + hiveLineage1.setVersion("2.0"); + hiveLineage1.setUser("user1"); + hiveLineage1.setEngine("spark"); + + HiveLineage hiveLineage2 = new HiveLineage(); + hiveLineage2.setVersion("2.0"); + hiveLineage2.setUser("user1"); + hiveLineage2.setEngine("spark"); + + // Test Lombok generated equals and hashCode + assertEquals(hiveLineage1, hiveLineage2); + assertEquals(hiveLineage1.hashCode(), hiveLineage2.hashCode()); + + // Test toString (Lombok generated) + assertNotNull(hiveLineage1.toString()); + assertTrue(hiveLineage1.toString().contains("HiveLineage")); + } + + /** + * Tests serialization with @SerializedName annotations. + * Verifies that field names are correctly mapped in JSON. + */ + @Test + void testSerializedNameAnnotations() { + HiveLineage hiveLineage = new HiveLineage(); + hiveLineage.setJobIds(Arrays.asList("job123")); + hiveLineage.setQueryText("SELECT * FROM test"); + hiveLineage.setPlatformInstance("test-instance"); + + String json = gson.toJson(hiveLineage); + + // Verify that @SerializedName annotations work correctly + assertTrue(json.contains("\"jobIds\"")); + assertTrue(json.contains("\"queryText\"")); + assertTrue(json.contains("\"platformInstance\"")); + + // Verify field names match the annotations + assertFalse(json.contains("\"job_ids\"")); + assertFalse(json.contains("\"query_text\"")); + assertFalse(json.contains("\"platform_instance\"")); + } + + /** + * Tests deserialization with complex nested structures. + * Verifies handling of multiple edges and vertices. + */ + @Test + void testComplexNestedStructures() { + String complexJson = "{" + + "\"version\":\"1.0\"," + + "\"edges\":[" + + "{\"sources\":[0,1],\"targets\":[2],\"edgeType\":\"PROJECTION\"}," + + "{\"sources\":[2],\"targets\":[3,4],\"edgeType\":\"DEPENDENCY\"}" + + "]," + + "\"vertices\":[" + + "{\"id\":0,\"vertexType\":\"COLUMN\",\"vertexId\":\"table1.col1\"}," + + "{\"id\":1,\"vertexType\":\"COLUMN\",\"vertexId\":\"table1.col2\"}," + + "{\"id\":2,\"vertexType\":\"COLUMN\",\"vertexId\":\"table2.col1\"}," + + "{\"id\":3,\"vertexType\":\"TABLE\",\"vertexId\":\"table3\"}," + + "{\"id\":4,\"vertexType\":\"TABLE\",\"vertexId\":\"table4\"}" + + "]" + + "}"; + + HiveLineage hiveLineage = gson.fromJson(complexJson, HiveLineage.class); + + // Verify complex structures are correctly deserialized + assertNotNull(hiveLineage.getEdges()); + assertEquals(2, hiveLineage.getEdges().size()); + + assertNotNull(hiveLineage.getVertices()); + assertEquals(5, hiveLineage.getVertices().size()); + + // Verify first edge + HiveLineage.Edge edge1 = hiveLineage.getEdges().get(0); + assertEquals(Arrays.asList(0, 1), edge1.getSources()); + assertEquals(Arrays.asList(2), edge1.getTargets()); + assertEquals("PROJECTION", edge1.getEdgeType()); + + // Verify last vertex + HiveLineage.Vertex lastVertex = hiveLineage.getVertices().get(4); + assertEquals(4, lastVertex.getId()); + assertEquals("TABLE", lastVertex.getVertexType()); + assertEquals("table4", lastVertex.getVertexId()); + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/service/LineageConsumerMCPEmitterServiceTest.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/service/LineageConsumerMCPEmitterServiceTest.java new file mode 100644 index 00000000000000..81f8ba04b74bee --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/service/LineageConsumerMCPEmitterServiceTest.java @@ -0,0 +1,155 @@ +package datahub.hive.consumer.service; + +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; +import com.linkedin.data.DataMap; +import com.linkedin.data.template.DataTemplate; +import datahub.client.kafka.KafkaEmitter; +import datahub.hive.consumer.util.MCPEmitterUtil; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.MockitoAnnotations; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.*; + +@ExtendWith(MockitoExtension.class) +public class LineageConsumerMCPEmitterServiceTest { + + @Mock + private MCPTransformerService datasetMCPTransformerService; + + @Mock + private MCPTransformerService queryMCPTransformerService; + + @Mock + private KafkaEmitter kafkaEmitter; + + private LineageConsumerMCPEmitterService lineageConsumerMCPEmitterService; + + private String validLineageMessage; + + @BeforeEach + void setUp() { + MockitoAnnotations.openMocks(this); + lineageConsumerMCPEmitterService = new LineageConsumerMCPEmitterService( + datasetMCPTransformerService, queryMCPTransformerService, kafkaEmitter); + // Create a valid lineage message for testing + JsonObject lineageJson = new JsonObject(); + lineageJson.addProperty("version", "1.0"); + lineageJson.addProperty("user", "testuser"); + lineageJson.addProperty("timestamp", System.currentTimeMillis() / 1000); + lineageJson.addProperty("duration", 1000); + lineageJson.addProperty("engine", "hive"); + lineageJson.addProperty("database", "testdb"); + lineageJson.addProperty("hash", "testhash"); + lineageJson.addProperty("queryText", "CREATE TABLE testdb.output_table AS SELECT * FROM testdb.input_table"); + lineageJson.addProperty("environment", "DEV"); + lineageJson.addProperty("platformInstance", "urn:li:dataPlatformInstance:(urn:li:dataPlatform:hive,OPR.OCE.DEV)"); + + // Add inputs and outputs + JsonObject inputs = new JsonObject(); + inputs.add("inputs", JsonParser.parseString("[\"testdb.input_table\"]").getAsJsonArray()); + lineageJson.add("inputs", JsonParser.parseString("[\"testdb.input_table\"]").getAsJsonArray()); + + JsonObject outputs = new JsonObject(); + outputs.add("outputs", JsonParser.parseString("[\"testdb.output_table\"]").getAsJsonArray()); + lineageJson.add("outputs", JsonParser.parseString("[\"testdb.output_table\"]").getAsJsonArray()); + + validLineageMessage = lineageJson.toString(); + } + + /** + * Tests the successful consumption and processing of a valid lineage message. + * Verifies that both transformer services are called and MCPs are emitted. + */ + @Test + void testConsumeLineageMessage() throws Exception { + // Create a consumer record with the valid lineage message + ConsumerRecord record = new ConsumerRecord<>("HiveLineage_v1", 0, 0, "testkey", validLineageMessage); + + // Create mock aspects + List> datasetAspects = new ArrayList<>(); + DataTemplate datasetAspect = mock(DataTemplate.class); + datasetAspects.add(datasetAspect); + + List> queryAspects = new ArrayList<>(); + DataTemplate queryAspect = mock(DataTemplate.class); + queryAspects.add(queryAspect); + + // Mock the transformer services to return aspects + when(datasetMCPTransformerService.transformToMCP(any())).thenReturn(datasetAspects); + when(queryMCPTransformerService.transformToMCP(any())).thenReturn(queryAspects); + + // Use try-with-resources for MockedStatic to ensure it's properly closed + try (MockedStatic mockedStatic = mockStatic(MCPEmitterUtil.class)) { + // Mock the MCPEmitterUtil to not throw exceptions + mockedStatic.when(() -> MCPEmitterUtil.emitMCP(any(), any(), any(), any())) + .thenAnswer(invocation -> null); // void method, so return null + + // Call the method under test + lineageConsumerMCPEmitterService.consumeLineageMessage(record); + + // Verify that the transformer services were called + verify(datasetMCPTransformerService).transformToMCP(any()); + verify(queryMCPTransformerService).transformToMCP(any()); + + // Verify that MCPEmitterUtil.emitMCP was called for each aspect + mockedStatic.verify(() -> MCPEmitterUtil.emitMCP(any(), any(), any(), eq(kafkaEmitter)), times(2)); + } + } + + /** + * Tests handling of invalid JSON in the lineage message. + * Verifies that an exception is thrown and no transformer services are called. + */ + @Test + void testConsumeLineageMessageWithInvalidJson() throws Exception { + // Create a consumer record with invalid JSON + ConsumerRecord record = new ConsumerRecord<>("HiveLineage_v1", 0, 0, "testkey", "invalid json"); + + try { + // Call the method under test - should throw an exception for invalid JSON + lineageConsumerMCPEmitterService.consumeLineageMessage(record); + fail("Expected an exception for invalid JSON"); + } catch (Exception e) { + // Expected exception + } + + // Verify that the transformer services were not called + verifyNoInteractions(datasetMCPTransformerService, queryMCPTransformerService); + } + + /** + * Tests handling of lineage messages with no outputs. + * Verifies that such messages are skipped and no transformer services are called. + */ + @Test + void testConsumeLineageMessageWithNoOutputs() throws Exception { + // Create a lineage message with no outputs + JsonObject lineageJson = new JsonObject(); + lineageJson.addProperty("version", "1.0"); + lineageJson.addProperty("queryText", "SELECT * FROM testdb.input_table"); + lineageJson.add("inputs", JsonParser.parseString("[\"testdb.input_table\"]").getAsJsonArray()); + // No outputs + + // Create a consumer record with the lineage message + ConsumerRecord record = new ConsumerRecord<>("HiveLineage_v1", 0, 0, "testkey", lineageJson.toString()); + + // Call the method under test + lineageConsumerMCPEmitterService.consumeLineageMessage(record); + + // Verify that the transformer services were not called + verifyNoInteractions(datasetMCPTransformerService, queryMCPTransformerService); + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/service/impl/DatasetMCPTransformerServiceImplTest.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/service/impl/DatasetMCPTransformerServiceImplTest.java new file mode 100644 index 00000000000000..bc823e552249ae --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/service/impl/DatasetMCPTransformerServiceImplTest.java @@ -0,0 +1,327 @@ +package datahub.hive.consumer.service.impl; + +import com.google.gson.JsonArray; +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; +import com.linkedin.common.DataPlatformInstance; +import com.linkedin.common.Status; +import com.linkedin.common.SubTypes; +import com.linkedin.data.DataMap; +import com.linkedin.data.template.DataTemplate; +import com.linkedin.dataset.UpstreamLineage; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.InjectMocks; +import org.mockito.junit.jupiter.MockitoExtension; +import org.springframework.test.util.ReflectionTestUtils; + +import java.net.URISyntaxException; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.*; + +/** + * Unit tests for DatasetMCPTransformerServiceImpl. + * Tests the transformation of dataset lineage JSON to MCP aspects. + */ +@ExtendWith(MockitoExtension.class) +public class DatasetMCPTransformerServiceImplTest { + + @InjectMocks + private DatasetMCPTransformerServiceImpl datasetMCPTransformerService; + + private JsonObject validDatasetJson; + private JsonObject datasetJsonWithEdges; + + @BeforeEach + void setUp() { + // Set required properties using ReflectionTestUtils + ReflectionTestUtils.setField(datasetMCPTransformerService, "environment", "DEV"); + ReflectionTestUtils.setField(datasetMCPTransformerService, "serviceUser", "testuser"); + + // Create a valid dataset JSON for testing + validDatasetJson = new JsonObject(); + validDatasetJson.addProperty("version", "1.0"); + validDatasetJson.addProperty("user", "testuser"); + validDatasetJson.addProperty("timestamp", System.currentTimeMillis() / 1000); + validDatasetJson.addProperty("duration", 1000); + validDatasetJson.addProperty("engine", "hive"); + validDatasetJson.addProperty("database", "testdb"); + validDatasetJson.addProperty("hash", "testhash123"); + validDatasetJson.addProperty("queryText", "CREATE TABLE testdb.output_table AS SELECT * FROM testdb.input_table"); + validDatasetJson.addProperty("environment", "DEV"); + validDatasetJson.addProperty("platformInstance", "urn:li:dataPlatformInstance:(urn:li:dataPlatform:hive,OPR.OCE.DEV)"); + + // Add inputs and outputs + validDatasetJson.add("inputs", JsonParser.parseString("[\"testdb.input_table\"]").getAsJsonArray()); + validDatasetJson.add("outputs", JsonParser.parseString("[\"testdb.output_table\"]").getAsJsonArray()); + + // Create dataset JSON with edges and vertices for fine-grained lineage testing + datasetJsonWithEdges = validDatasetJson.deepCopy(); + + // Add vertices + JsonArray vertices = new JsonArray(); + JsonObject vertex1 = new JsonObject(); + vertex1.addProperty("id", 0); + vertex1.addProperty("vertexType", "COLUMN"); + vertex1.addProperty("vertexId", "input_table.col1"); + vertices.add(vertex1); + + JsonObject vertex2 = new JsonObject(); + vertex2.addProperty("id", 1); + vertex2.addProperty("vertexType", "COLUMN"); + vertex2.addProperty("vertexId", "output_table.col1"); + vertices.add(vertex2); + + datasetJsonWithEdges.add("vertices", vertices); + + // Add edges + JsonArray edges = new JsonArray(); + JsonObject edge = new JsonObject(); + edge.add("sources", JsonParser.parseString("[0]").getAsJsonArray()); + edge.add("targets", JsonParser.parseString("[1]").getAsJsonArray()); + edge.addProperty("edgeType", "PROJECTION"); + edge.addProperty("expression", "col1"); + edges.add(edge); + + datasetJsonWithEdges.add("edges", edges); + } + + /** + * Tests the main transformToMCP method with valid dataset JSON. + * Verifies that all expected aspects are created. + */ + @Test + void testTransformToMCP_ValidDataset() throws URISyntaxException { + // Call the method under test + List> aspects = datasetMCPTransformerService.transformToMCP(validDatasetJson); + + // Verify that 4 aspects are created + assertEquals(4, aspects.size()); + + // Verify aspect types + assertTrue(aspects.get(0) instanceof UpstreamLineage); + assertTrue(aspects.get(1) instanceof DataPlatformInstance); + assertTrue(aspects.get(2) instanceof Status); + assertTrue(aspects.get(3) instanceof SubTypes); + } + + /** + * Tests transformToMCP with dataset JSON containing edges and vertices. + * Verifies fine-grained lineage processing. + */ + @Test + void testTransformToMCP_WithEdgesAndVertices() throws URISyntaxException { + // Call the method under test + List> aspects = datasetMCPTransformerService.transformToMCP(datasetJsonWithEdges); + + // Verify that 4 aspects are created + assertEquals(4, aspects.size()); + + // Verify the upstream lineage aspect contains fine-grained lineage + UpstreamLineage upstreamLineage = (UpstreamLineage) aspects.get(0); + assertNotNull(upstreamLineage.getUpstreams()); + assertFalse(upstreamLineage.getUpstreams().isEmpty()); + + // Verify fine-grained lineage is present + if (upstreamLineage.hasFineGrainedLineages()) { + assertNotNull(upstreamLineage.getFineGrainedLineages()); + assertFalse(upstreamLineage.getFineGrainedLineages().isEmpty()); + } + } + + /** + * Tests transformToMCP with dataset JSON without inputs. + * Verifies handling of missing inputs. + */ + @Test + void testTransformToMCP_NoInputs() throws URISyntaxException { + // Remove inputs from the dataset JSON + validDatasetJson.remove("inputs"); + + // Call the method under test + List> aspects = datasetMCPTransformerService.transformToMCP(validDatasetJson); + + // Verify that aspects are still created + assertEquals(4, aspects.size()); + + // Verify upstream lineage aspect + UpstreamLineage upstreamLineage = (UpstreamLineage) aspects.get(0); + assertNotNull(upstreamLineage.getUpstreams()); + assertTrue(upstreamLineage.getUpstreams().isEmpty()); + } + + /** + * Tests transformToMCP with empty inputs array. + * Verifies handling of empty inputs. + */ + @Test + void testTransformToMCP_EmptyInputs() throws URISyntaxException { + // Set empty inputs array + validDatasetJson.add("inputs", new JsonArray()); + + // Call the method under test + List> aspects = datasetMCPTransformerService.transformToMCP(validDatasetJson); + + // Verify that aspects are still created + assertEquals(4, aspects.size()); + + // Verify upstream lineage aspect has empty upstreams + UpstreamLineage upstreamLineage = (UpstreamLineage) aspects.get(0); + assertNotNull(upstreamLineage.getUpstreams()); + assertTrue(upstreamLineage.getUpstreams().isEmpty()); + } + + /** + * Tests the data platform instance aspect creation. + * Verifies correct platform instance URN generation. + */ + @Test + void testBuildDataPlatformInstanceAspect() throws URISyntaxException { + // Call the method under test + List> aspects = datasetMCPTransformerService.transformToMCP(validDatasetJson); + + // Get the data platform instance aspect + DataPlatformInstance dataPlatformInstance = (DataPlatformInstance) aspects.get(1); + + // Verify platform and instance are set + assertNotNull(dataPlatformInstance.getPlatform()); + assertNotNull(dataPlatformInstance.getInstance()); + assertTrue(dataPlatformInstance.getPlatform().toString().contains("hive")); + } + + /** + * Tests the status aspect creation. + * Verifies that status is set to not removed. + */ + @Test + void testBuildStatusAspect() throws URISyntaxException { + // Call the method under test + List> aspects = datasetMCPTransformerService.transformToMCP(validDatasetJson); + + // Get the status aspect + Status status = (Status) aspects.get(2); + + // Verify status is not removed + assertFalse(status.isRemoved()); + } + + /** + * Tests the subTypes aspect creation. + * Verifies that subTypes contains "Table". + */ + @Test + void testBuildSubTypesAspect() throws URISyntaxException { + // Call the method under test + List> aspects = datasetMCPTransformerService.transformToMCP(validDatasetJson); + + // Get the subTypes aspect + SubTypes subTypes = (SubTypes) aspects.get(3); + + // Verify subTypes contains "Table" + assertNotNull(subTypes.getTypeNames()); + assertFalse(subTypes.getTypeNames().isEmpty()); + assertTrue(subTypes.getTypeNames().contains("Table")); + } + + /** + * Tests upstream lineage aspect with multiple inputs. + * Verifies handling of multiple upstream datasets. + */ + @Test + void testBuildUpstreamLineageAspect_MultipleInputs() throws URISyntaxException { + // Add multiple inputs + validDatasetJson.add("inputs", JsonParser.parseString("[\"testdb.input_table1\", \"testdb.input_table2\"]").getAsJsonArray()); + + // Call the method under test + List> aspects = datasetMCPTransformerService.transformToMCP(validDatasetJson); + + // Get the upstream lineage aspect + UpstreamLineage upstreamLineage = (UpstreamLineage) aspects.get(0); + + // Verify multiple upstreams are created + assertNotNull(upstreamLineage.getUpstreams()); + assertEquals(2, upstreamLineage.getUpstreams().size()); + } + + /** + * Tests handling of edges with non-PROJECTION type. + * Verifies that only PROJECTION edges are processed for fine-grained lineage. + */ + @Test + void testTransformToMCP_NonProjectionEdges() throws URISyntaxException { + // Modify edge type to non-PROJECTION + JsonArray edges = datasetJsonWithEdges.getAsJsonArray("edges"); + JsonObject edge = edges.get(0).getAsJsonObject(); + edge.addProperty("edgeType", "DEPENDENCY"); + + // Call the method under test + List> aspects = datasetMCPTransformerService.transformToMCP(datasetJsonWithEdges); + + // Verify aspects are created + assertEquals(4, aspects.size()); + + // Verify upstream lineage aspect + UpstreamLineage upstreamLineage = (UpstreamLineage) aspects.get(0); + assertNotNull(upstreamLineage.getUpstreams()); + + // Fine-grained lineage should not be created for non-PROJECTION edges + if (upstreamLineage.hasFineGrainedLineages()) { + assertTrue(upstreamLineage.getFineGrainedLineages().isEmpty()); + } + } + + /** + * Tests handling of missing edges and vertices. + * Verifies graceful handling when fine-grained lineage data is missing. + */ + @Test + void testTransformToMCP_MissingEdgesAndVertices() throws URISyntaxException { + // Create dataset without edges and vertices + JsonObject datasetWithoutEdgesVertices = validDatasetJson.deepCopy(); + + // Call the method under test - should handle missing edges/vertices gracefully + List> aspects = datasetMCPTransformerService.transformToMCP(datasetWithoutEdgesVertices); + + // Verify that aspects are still created + assertNotNull(aspects); + assertEquals(4, aspects.size()); + + // Verify that the method produces valid aspects + assertTrue(aspects.get(0) instanceof UpstreamLineage); + assertTrue(aspects.get(1) instanceof DataPlatformInstance); + assertTrue(aspects.get(2) instanceof Status); + assertTrue(aspects.get(3) instanceof SubTypes); + + // Verify upstream lineage aspect + UpstreamLineage upstreamLineage = (UpstreamLineage) aspects.get(0); + assertNotNull(upstreamLineage.getUpstreams()); + + // Should not have fine-grained lineage without edges/vertices + if (upstreamLineage.hasFineGrainedLineages()) { + assertTrue(upstreamLineage.getFineGrainedLineages() == null || upstreamLineage.getFineGrainedLineages().isEmpty()); + } + } + + /** + * Tests handling of missing hash field. + * Verifies that upstream lineage is created without query URN when hash is missing. + */ + @Test + void testTransformToMCP_MissingHash() throws URISyntaxException { + // Remove hash field + validDatasetJson.remove("hash"); + + // Call the method under test + List> aspects = datasetMCPTransformerService.transformToMCP(validDatasetJson); + + // Verify that aspects are still created + assertEquals(4, aspects.size()); + + // Verify upstream lineage aspect + UpstreamLineage upstreamLineage = (UpstreamLineage) aspects.get(0); + assertNotNull(upstreamLineage.getUpstreams()); + assertFalse(upstreamLineage.getUpstreams().isEmpty()); + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/service/impl/QueryMCPTransformerServiceImplTest.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/service/impl/QueryMCPTransformerServiceImplTest.java new file mode 100644 index 00000000000000..fca1a45c1ce4e9 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/service/impl/QueryMCPTransformerServiceImplTest.java @@ -0,0 +1,310 @@ +package datahub.hive.consumer.service.impl; + +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; +import com.linkedin.data.DataMap; +import com.linkedin.data.template.DataTemplate; +import com.linkedin.query.QueryProperties; +import com.linkedin.query.QuerySubjects; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.InjectMocks; +import org.mockito.junit.jupiter.MockitoExtension; +import org.springframework.test.util.ReflectionTestUtils; + +import java.net.URISyntaxException; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.*; + +/** + * Unit tests for QueryMCPTransformerServiceImpl. + * Tests the transformation of query lineage JSON to MCP aspects. + */ +@ExtendWith(MockitoExtension.class) +public class QueryMCPTransformerServiceImplTest { + + @InjectMocks + private QueryMCPTransformerServiceImpl queryMCPTransformerService; + + private JsonObject validQueryJson; + + @BeforeEach + void setUp() { + // Set required properties using ReflectionTestUtils + ReflectionTestUtils.setField(queryMCPTransformerService, "environment", "DEV"); + ReflectionTestUtils.setField(queryMCPTransformerService, "serviceUser", "testuser"); + + // Create a valid query JSON for testing + validQueryJson = new JsonObject(); + validQueryJson.addProperty("version", "1.0"); + validQueryJson.addProperty("user", "testuser"); + validQueryJson.addProperty("timestamp", System.currentTimeMillis() / 1000); + validQueryJson.addProperty("duration", 1000); + validQueryJson.addProperty("engine", "hive"); + validQueryJson.addProperty("database", "testdb"); + validQueryJson.addProperty("hash", "testhash123"); + validQueryJson.addProperty("queryText", "CREATE TABLE testdb.output_table AS SELECT * FROM testdb.input_table"); + validQueryJson.addProperty("environment", "DEV"); + validQueryJson.addProperty("platformInstance", "urn:li:dataPlatformInstance:(urn:li:dataPlatform:hive,OPR.OCE.DEV)"); + + // Add inputs and outputs + validQueryJson.add("inputs", JsonParser.parseString("[\"testdb.input_table\"]").getAsJsonArray()); + validQueryJson.add("outputs", JsonParser.parseString("[\"testdb.output_table\"]").getAsJsonArray()); + } + + /** + * Tests the main transformToMCP method with valid query JSON. + * Verifies that all expected aspects are created. + */ + @Test + void testTransformToMCP_ValidQuery() throws URISyntaxException { + // Call the method under test + List> aspects = queryMCPTransformerService.transformToMCP(validQueryJson); + + // Verify that 2 aspects are created + assertEquals(2, aspects.size()); + + // Verify aspect types + assertTrue(aspects.get(0) instanceof QuerySubjects); + assertTrue(aspects.get(1) instanceof QueryProperties); + } + + /** + * Tests the query subjects aspect creation. + * Verifies that query subjects are correctly built from outputs. + */ + @Test + void testBuildQuerySubjectsAspect() throws URISyntaxException { + // Call the method under test + List> aspects = queryMCPTransformerService.transformToMCP(validQueryJson); + + // Get the query subjects aspect + QuerySubjects querySubjects = (QuerySubjects) aspects.get(0); + + // Verify subjects are set + assertNotNull(querySubjects.getSubjects()); + assertFalse(querySubjects.getSubjects().isEmpty()); + assertEquals(1, querySubjects.getSubjects().size()); + + // Verify the subject entity is set correctly + assertNotNull(querySubjects.getSubjects().get(0).getEntity()); + assertTrue(querySubjects.getSubjects().get(0).getEntity().toString().contains("output_table")); + } + + /** + * Tests the query properties aspect creation. + * Verifies that query properties are correctly built. + */ + @Test + void testBuildQueryPropertiesAspect() throws URISyntaxException { + // Call the method under test + List> aspects = queryMCPTransformerService.transformToMCP(validQueryJson); + + // Get the query properties aspect + QueryProperties queryProperties = (QueryProperties) aspects.get(1); + + // Verify statement is set + assertNotNull(queryProperties.getStatement()); + assertEquals("CREATE TABLE testdb.output_table AS SELECT * FROM testdb.input_table", + queryProperties.getStatement().getValue()); + assertEquals("SQL", queryProperties.getStatement().getLanguage().name()); + + // Verify timestamps are set + assertNotNull(queryProperties.getCreated()); + assertNotNull(queryProperties.getLastModified()); + assertTrue(queryProperties.getCreated().getTime() > 0); + assertTrue(queryProperties.getLastModified().getTime() > 0); + + // Verify actor is set + assertNotNull(queryProperties.getCreated().getActor()); + assertNotNull(queryProperties.getLastModified().getActor()); + assertTrue(queryProperties.getCreated().getActor().toString().contains("testuser")); + assertTrue(queryProperties.getLastModified().getActor().toString().contains("testuser")); + + // Verify source is set to MANUAL + assertEquals("MANUAL", queryProperties.getSource().name()); + } + + /** + * Tests transformToMCP with query JSON without outputs. + * Verifies handling of missing outputs. + */ + @Test + void testTransformToMCP_NoOutputs() throws URISyntaxException { + // Remove outputs from the query JSON + validQueryJson.remove("outputs"); + + // Call the method under test + List> aspects = queryMCPTransformerService.transformToMCP(validQueryJson); + + // Verify that aspects are still created + assertEquals(2, aspects.size()); + + // Verify query subjects aspect has no subjects + QuerySubjects querySubjects = (QuerySubjects) aspects.get(0); + if (querySubjects.hasSubjects()) { + assertTrue(querySubjects.getSubjects().isEmpty()); + } + } + + /** + * Tests transformToMCP with empty outputs array. + * Verifies handling of empty outputs. + */ + @Test + void testTransformToMCP_EmptyOutputs() throws URISyntaxException { + // Set empty outputs array + validQueryJson.add("outputs", JsonParser.parseString("[]").getAsJsonArray()); + + // Call the method under test + List> aspects = queryMCPTransformerService.transformToMCP(validQueryJson); + + // Verify that aspects are still created + assertEquals(2, aspects.size()); + + // Verify query subjects aspect has no subjects + QuerySubjects querySubjects = (QuerySubjects) aspects.get(0); + if (querySubjects.hasSubjects()) { + assertTrue(querySubjects.getSubjects().isEmpty()); + } + } + + /** + * Tests transformToMCP with multiple outputs. + * Verifies handling of multiple output datasets. + */ + @Test + void testTransformToMCP_MultipleOutputs() throws URISyntaxException { + // Add multiple outputs + validQueryJson.add("outputs", JsonParser.parseString("[\"testdb.output_table1\", \"testdb.output_table2\"]").getAsJsonArray()); + + // Call the method under test + List> aspects = queryMCPTransformerService.transformToMCP(validQueryJson); + + // Verify that aspects are created + assertEquals(2, aspects.size()); + + // Verify query subjects aspect - should only use the first output + QuerySubjects querySubjects = (QuerySubjects) aspects.get(0); + assertNotNull(querySubjects.getSubjects()); + assertEquals(1, querySubjects.getSubjects().size()); + assertTrue(querySubjects.getSubjects().get(0).getEntity().toString().contains("output_table1")); + } + + /** + * Tests transformToMCP with different query text. + * Verifies handling of various SQL statements. + */ + @Test + void testTransformToMCP_DifferentQueryText() throws URISyntaxException { + // Change query text to a SELECT statement + validQueryJson.addProperty("queryText", "SELECT col1, col2 FROM testdb.input_table WHERE col1 > 100"); + + // Call the method under test + List> aspects = queryMCPTransformerService.transformToMCP(validQueryJson); + + // Verify that aspects are created + assertEquals(2, aspects.size()); + + // Verify query properties aspect has the correct query text + QueryProperties queryProperties = (QueryProperties) aspects.get(1); + assertEquals("SELECT col1, col2 FROM testdb.input_table WHERE col1 > 100", + queryProperties.getStatement().getValue()); + } + + /** + * Tests transformToMCP with missing queryText field. + * Verifies handling of missing query text. + */ + @Test + void testTransformToMCP_MissingQueryText() { + // Remove queryText field + validQueryJson.remove("queryText"); + + // Call the method under test - should throw exception due to missing queryText + assertThrows(Exception.class, () -> { + queryMCPTransformerService.transformToMCP(validQueryJson); + }); + } + + /** + * Tests transformToMCP with null queryText field. + * Verifies handling of null query text. + */ + @Test + void testTransformToMCP_NullQueryText() { + // Set queryText to null + validQueryJson.add("queryText", null); + + // Call the method under test - should throw exception due to null queryText + assertThrows(Exception.class, () -> { + queryMCPTransformerService.transformToMCP(validQueryJson); + }); + } + + /** + * Tests transformToMCP with missing platformInstance field. + * Verifies handling of missing platform instance. + */ + @Test + void testTransformToMCP_MissingPlatformInstance() { + // Remove platformInstance field + validQueryJson.remove("platformInstance"); + + // Call the method under test - should throw exception due to missing platformInstance + assertThrows(Exception.class, () -> { + queryMCPTransformerService.transformToMCP(validQueryJson); + }); + } + + /** + * Tests query subjects aspect with complex dataset names. + * Verifies handling of dataset names with special characters. + */ + @Test + void testBuildQuerySubjectsAspect_ComplexDatasetNames() throws URISyntaxException { + // Set output with complex name + validQueryJson.add("outputs", JsonParser.parseString("[\"test_db.complex_table_name_123\"]").getAsJsonArray()); + + // Call the method under test + List> aspects = queryMCPTransformerService.transformToMCP(validQueryJson); + + // Get the query subjects aspect + QuerySubjects querySubjects = (QuerySubjects) aspects.get(0); + + // Verify subjects are set correctly + assertNotNull(querySubjects.getSubjects()); + assertEquals(1, querySubjects.getSubjects().size()); + assertTrue(querySubjects.getSubjects().get(0).getEntity().toString().contains("complex_table_name_123")); + } + + /** + * Tests query properties aspect with long query text. + * Verifies handling of large SQL statements. + */ + @Test + void testBuildQueryPropertiesAspect_LongQueryText() throws URISyntaxException { + // Create a long query text + StringBuilder longQuery = new StringBuilder(); + longQuery.append("SELECT "); + for (int i = 1; i <= 50; i++) { + longQuery.append("col").append(i); + if (i < 50) longQuery.append(", "); + } + longQuery.append(" FROM testdb.large_table WHERE col1 > 0 AND col2 < 1000"); + + validQueryJson.addProperty("queryText", longQuery.toString()); + + // Call the method under test + List> aspects = queryMCPTransformerService.transformToMCP(validQueryJson); + + // Get the query properties aspect + QueryProperties queryProperties = (QueryProperties) aspects.get(1); + + // Verify the long query text is preserved + assertEquals(longQuery.toString(), queryProperties.getStatement().getValue()); + assertTrue(queryProperties.getStatement().getValue().length() > 200); + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/util/MCPEmitterUtilTest.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/util/MCPEmitterUtilTest.java new file mode 100644 index 00000000000000..065e47dd0dc663 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/util/MCPEmitterUtilTest.java @@ -0,0 +1,359 @@ +package datahub.hive.consumer.util; + +import com.linkedin.common.Status; +import com.linkedin.data.DataMap; +import com.linkedin.data.template.DataTemplate; +import datahub.client.Callback; +import datahub.client.MetadataWriteResponse; +import datahub.client.kafka.KafkaEmitter; +import datahub.event.MetadataChangeProposalWrapper; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +import static org.junit.jupiter.api.Assertions.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.*; + +/** + * Comprehensive unit tests for MCPEmitterUtil. + * Tests the emission of Metadata Change Proposals (MCPs) to VDC via Kafka. + * Uses pure unit testing approach with complete mocking to avoid SSL configuration issues. + */ +@ExtendWith(MockitoExtension.class) +public class MCPEmitterUtilTest { + + @Mock + private KafkaEmitter mockKafkaEmitter; + + private DataTemplate testAspect; + + @Mock + private Future mockFuture; + + @Mock + private MetadataWriteResponse mockSuccessResponse; + + @Mock + private MetadataWriteResponse mockFailureResponse; + + private static final String ENTITY_URN = "urn:li:dataset:(urn:li:dataPlatform:hive,testdb.test_table,DEV)"; + private static final String ENTITY_TYPE = "dataset"; + private static final String FAILURE_MESSAGE = "Kafka broker rejected the message"; + + @BeforeEach + void setUp() { + // Reset all mocks before each test + reset(mockKafkaEmitter, mockFuture, mockSuccessResponse, mockFailureResponse); + + // Create a real DataHub aspect instead of mocking it + // Using Status aspect as it's simple and commonly used + testAspect = new Status().setRemoved(false); + } + + /** + * Test Case 1: Success Path + * Tests that when the Future completes successfully and response.isSuccess() returns true, + * no exception is thrown and the method completes normally. + */ + @Test + void emitMCP_ShouldCompleteSuccessfully_WhenEmitIsSuccessful() throws Exception { + // Arrange + when(mockKafkaEmitter.emit(any(MetadataChangeProposalWrapper.class), any(Callback.class))) + .thenReturn(mockFuture); + when(mockFuture.get()).thenReturn(mockSuccessResponse); + when(mockSuccessResponse.isSuccess()).thenReturn(true); + + // Act & Assert + assertDoesNotThrow(() -> MCPEmitterUtil.emitMCP(testAspect, ENTITY_URN, ENTITY_TYPE, mockKafkaEmitter)); + + // Verify interactions + verify(mockKafkaEmitter, times(1)).emit(any(MetadataChangeProposalWrapper.class), any(Callback.class)); + verify(mockFuture, times(1)).get(); + verify(mockSuccessResponse, times(1)).isSuccess(); // Called once in main method + } + + /** + * Test Case 2: Response Failure Path + * Tests that when Kafka accepts the message but reports failure in the response, + * an IOException is thrown with the appropriate error message. + */ + @Test + void emitMCP_ShouldThrowIOException_WhenResponseIsNotSuccess() throws Exception { + // Arrange + when(mockKafkaEmitter.emit(any(MetadataChangeProposalWrapper.class), any(Callback.class))) + .thenReturn(mockFuture); + when(mockFuture.get()).thenReturn(mockFailureResponse); + when(mockFailureResponse.isSuccess()).thenReturn(false); + when(mockFailureResponse.getResponseContent()).thenReturn(FAILURE_MESSAGE); + + // Act & Assert + IOException exception = assertThrows(IOException.class, () -> { + MCPEmitterUtil.emitMCP(testAspect, ENTITY_URN, ENTITY_TYPE, mockKafkaEmitter); + }); + + // Verify exception message + assertTrue(exception.getMessage().contains(FAILURE_MESSAGE)); + assertTrue(exception.getMessage().contains("Failed to emit MCP")); + + // Verify interactions + verify(mockKafkaEmitter, times(1)).emit(any(MetadataChangeProposalWrapper.class), any(Callback.class)); + verify(mockFuture, times(1)).get(); + verify(mockFailureResponse, times(1)).isSuccess(); + verify(mockFailureResponse, times(2)).getResponseContent(); // Called twice: once for logging, once for exception message + } + + /** + * Test Case 3: ExecutionException Path + * Tests that when future.get() throws ExecutionException, + * it's caught and wrapped in an IOException with the original exception as cause. + */ + @Test + void emitMCP_ShouldThrowIOException_WhenFutureGetThrowsExecutionException() throws Exception { + // Arrange + ExecutionException executionException = new ExecutionException("Connection timed out", new RuntimeException("Network error")); + when(mockKafkaEmitter.emit(any(MetadataChangeProposalWrapper.class), any(Callback.class))) + .thenReturn(mockFuture); + when(mockFuture.get()).thenThrow(executionException); + + // Act & Assert + IOException exception = assertThrows(IOException.class, () -> { + MCPEmitterUtil.emitMCP(testAspect, ENTITY_URN, ENTITY_TYPE, mockKafkaEmitter); + }); + + // Verify exception details + assertEquals("Error emitting MCP", exception.getMessage()); + assertEquals(executionException, exception.getCause()); + + // Verify interactions + verify(mockKafkaEmitter, times(1)).emit(any(MetadataChangeProposalWrapper.class), any(Callback.class)); + verify(mockFuture, times(1)).get(); + } + + /** + * Test Case 4: InterruptedException Path + * Tests that when future.get() throws InterruptedException, + * it's caught, thread is interrupted, and wrapped in an IOException. + */ + @Test + void emitMCP_ShouldThrowIOException_WhenFutureGetThrowsInterruptedException() throws Exception { + // Arrange + InterruptedException interruptedException = new InterruptedException("Thread was interrupted"); + when(mockKafkaEmitter.emit(any(MetadataChangeProposalWrapper.class), any(Callback.class))) + .thenReturn(mockFuture); + when(mockFuture.get()).thenThrow(interruptedException); + + // Act & Assert + IOException exception = assertThrows(IOException.class, () -> { + MCPEmitterUtil.emitMCP(testAspect, ENTITY_URN, ENTITY_TYPE, mockKafkaEmitter); + }); + + // Verify exception details + assertEquals("Error emitting MCP", exception.getMessage()); + assertEquals(interruptedException, exception.getCause()); + + // Verify thread interruption status + assertTrue(Thread.currentThread().isInterrupted()); + + // Clear the interrupted status for other tests + Thread.interrupted(); + + // Verify interactions + verify(mockKafkaEmitter, times(1)).emit(any(MetadataChangeProposalWrapper.class), any(Callback.class)); + verify(mockFuture, times(1)).get(); + } + + /** + * Test Case 5: Callback Success Behavior + * Tests that the callback's onCompletion method behaves correctly for successful responses. + * Uses ArgumentCaptor to capture and test the callback logic. + */ + @Test + void callback_ShouldLogSuccess_OnSuccessfulCompletion() throws Exception { + // Arrange + ArgumentCaptor callbackCaptor = ArgumentCaptor.forClass(Callback.class); + when(mockKafkaEmitter.emit(any(MetadataChangeProposalWrapper.class), any(Callback.class))) + .thenReturn(mockFuture); + when(mockFuture.get()).thenReturn(mockSuccessResponse); + when(mockSuccessResponse.isSuccess()).thenReturn(true); + + try (MockedStatic mockedTimeUtils = mockStatic(TimeUtils.class)) { + mockedTimeUtils.when(() -> TimeUtils.calculateDuration(anyLong())).thenReturn(150L); + + // Act + assertDoesNotThrow(() -> MCPEmitterUtil.emitMCP(testAspect, ENTITY_URN, ENTITY_TYPE, mockKafkaEmitter)); + + // Capture the callback + verify(mockKafkaEmitter).emit(any(MetadataChangeProposalWrapper.class), callbackCaptor.capture()); + Callback capturedCallback = callbackCaptor.getValue(); + + // Test callback success behavior + assertDoesNotThrow(() -> capturedCallback.onCompletion(mockSuccessResponse)); + + // Verify TimeUtils was called for duration calculation + mockedTimeUtils.verify(() -> TimeUtils.calculateDuration(anyLong()), times(1)); + } + } + + /** + * Test Case 6: Callback Failure Behavior + * Tests that the callback's onCompletion method behaves correctly for failed responses. + */ + @Test + void callback_ShouldLogFailure_OnFailedCompletion() throws Exception { + // Arrange + ArgumentCaptor callbackCaptor = ArgumentCaptor.forClass(Callback.class); + when(mockKafkaEmitter.emit(any(MetadataChangeProposalWrapper.class), any(Callback.class))) + .thenReturn(mockFuture); + when(mockFuture.get()).thenReturn(mockSuccessResponse); + when(mockSuccessResponse.isSuccess()).thenReturn(true); + + // Act + assertDoesNotThrow(() -> MCPEmitterUtil.emitMCP(testAspect, ENTITY_URN, ENTITY_TYPE, mockKafkaEmitter)); + + // Capture the callback + verify(mockKafkaEmitter).emit(any(MetadataChangeProposalWrapper.class), callbackCaptor.capture()); + Callback capturedCallback = callbackCaptor.getValue(); + + // Test callback failure behavior + when(mockFailureResponse.isSuccess()).thenReturn(false); + when(mockFailureResponse.getResponseContent()).thenReturn(FAILURE_MESSAGE); + + assertDoesNotThrow(() -> capturedCallback.onCompletion(mockFailureResponse)); + + // Verify failure response methods were called + verify(mockFailureResponse, times(1)).isSuccess(); + verify(mockFailureResponse, times(1)).getResponseContent(); + } + + /** + * Test Case 7: Callback Exception Behavior + * Tests that the callback's onFailure method behaves correctly when exceptions occur. + */ + @Test + void callback_ShouldLogException_OnFailure() throws Exception { + // Arrange + ArgumentCaptor callbackCaptor = ArgumentCaptor.forClass(Callback.class); + when(mockKafkaEmitter.emit(any(MetadataChangeProposalWrapper.class), any(Callback.class))) + .thenReturn(mockFuture); + when(mockFuture.get()).thenReturn(mockSuccessResponse); + when(mockSuccessResponse.isSuccess()).thenReturn(true); + + // Act + assertDoesNotThrow(() -> MCPEmitterUtil.emitMCP(testAspect, ENTITY_URN, ENTITY_TYPE, mockKafkaEmitter)); + + // Capture the callback + verify(mockKafkaEmitter).emit(any(MetadataChangeProposalWrapper.class), callbackCaptor.capture()); + Callback capturedCallback = callbackCaptor.getValue(); + + // Test callback exception behavior + Throwable testException = new RuntimeException("Network connection failed"); + assertDoesNotThrow(() -> capturedCallback.onFailure(testException)); + } + + /** + * Test Case 8: Null Parameter Validation + * Tests that the method handles null parameters appropriately. + */ + @Test + void emitMCP_ShouldThrowException_WhenParametersAreNull() { + // Test null aspect + assertThrows(Exception.class, () -> { + MCPEmitterUtil.emitMCP(null, ENTITY_URN, ENTITY_TYPE, mockKafkaEmitter); + }); + + // Test null entityUrn + assertThrows(Exception.class, () -> { + MCPEmitterUtil.emitMCP(testAspect, null, ENTITY_TYPE, mockKafkaEmitter); + }); + + // Test null entityType + assertThrows(Exception.class, () -> { + MCPEmitterUtil.emitMCP(testAspect, ENTITY_URN, null, mockKafkaEmitter); + }); + + // Test null kafkaEmitter + assertThrows(Exception.class, () -> { + MCPEmitterUtil.emitMCP(testAspect, ENTITY_URN, ENTITY_TYPE, null); + }); + } + + /** + * Test Case 9: Invalid URN Format + * Tests that the method handles invalid URN formats appropriately. + */ + @Test + void emitMCP_ShouldThrowException_WhenURNIsInvalid() { + String invalidUrn = "invalid-urn-format"; + + assertThrows(Exception.class, () -> { + MCPEmitterUtil.emitMCP(testAspect, invalidUrn, ENTITY_TYPE, mockKafkaEmitter); + }); + } + + /** + * Test Case 10: Empty String Parameters + * Tests that the method handles empty string parameters appropriately. + */ + @Test + void emitMCP_ShouldThrowException_WhenParametersAreEmpty() { + // Test empty entityUrn + assertThrows(Exception.class, () -> { + MCPEmitterUtil.emitMCP(testAspect, "", ENTITY_TYPE, mockKafkaEmitter); + }); + + // Test empty entityType + assertThrows(Exception.class, () -> { + MCPEmitterUtil.emitMCP(testAspect, ENTITY_URN, "", mockKafkaEmitter); + }); + } + + /** + * Test Case 11: Utility Class Constructor + * Tests that the utility class has a private constructor and cannot be instantiated. + */ + @Test + void testUtilityClassCannotBeInstantiated() throws Exception { + var constructor = MCPEmitterUtil.class.getDeclaredConstructor(); + assertTrue(java.lang.reflect.Modifier.isPrivate(constructor.getModifiers())); + + constructor.setAccessible(true); + assertDoesNotThrow(() -> constructor.newInstance()); + } + + /** + * Test Case 12: MetadataChangeProposalWrapper Creation + * Tests that the method properly creates MetadataChangeProposalWrapper with correct parameters. + */ + @Test + void emitMCP_ShouldCreateCorrectMCPWrapper() throws Exception { + // Arrange + ArgumentCaptor mcpCaptor = ArgumentCaptor.forClass(MetadataChangeProposalWrapper.class); + when(mockKafkaEmitter.emit(any(MetadataChangeProposalWrapper.class), any(Callback.class))) + .thenReturn(mockFuture); + when(mockFuture.get()).thenReturn(mockSuccessResponse); + when(mockSuccessResponse.isSuccess()).thenReturn(true); + + // Act + assertDoesNotThrow(() -> MCPEmitterUtil.emitMCP(testAspect, ENTITY_URN, ENTITY_TYPE, mockKafkaEmitter)); + + // Verify MCP wrapper creation + verify(mockKafkaEmitter).emit(mcpCaptor.capture(), any(Callback.class)); + MetadataChangeProposalWrapper capturedMCP = mcpCaptor.getValue(); + + // Verify MCP wrapper properties + assertNotNull(capturedMCP); + assertEquals(ENTITY_URN, capturedMCP.getEntityUrn()); + assertEquals(ENTITY_TYPE, capturedMCP.getEntityType()); + assertEquals(testAspect, capturedMCP.getAspect()); + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/util/TimeUtilsTest.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/util/TimeUtilsTest.java new file mode 100644 index 00000000000000..545465e524c8a5 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-consumer/src/test/java/datahub/hive/consumer/util/TimeUtilsTest.java @@ -0,0 +1,146 @@ +package datahub.hive.consumer.util; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.*; + +/** + * Unit tests for TimeUtils. + * Tests time-related utility operations. + */ +public class TimeUtilsTest { + + /** + * Tests calculateDuration method with valid start time. + * Verifies that duration is calculated correctly. + */ + @Test + void testCalculateDuration_ValidStartTime() { + // Get current time as start time + long startTime = System.currentTimeMillis(); + + // Add a small delay to ensure duration > 0 + try { + Thread.sleep(10); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + + // Calculate duration + long duration = TimeUtils.calculateDuration(startTime); + + // Verify duration is positive and reasonable + assertTrue(duration >= 0); + assertTrue(duration < 1000); // Should be less than 1 second for this test + } + + /** + * Tests calculateDuration method with past start time. + * Verifies that duration calculation works with older timestamps. + */ + @Test + void testCalculateDuration_PastStartTime() { + // Use a start time from 1 second ago + long startTime = System.currentTimeMillis() - 1000; + + // Calculate duration + long duration = TimeUtils.calculateDuration(startTime); + + // Verify duration is approximately 1 second (with some tolerance) + assertTrue(duration >= 1000); + assertTrue(duration < 2000); // Should be less than 2 seconds + } + + /** + * Tests calculateDuration method with future start time. + * Verifies that duration can handle future timestamps (negative duration). + */ + @Test + void testCalculateDuration_FutureStartTime() { + // Use a start time from 1 second in the future + long startTime = System.currentTimeMillis() + 1000; + + // Calculate duration + long duration = TimeUtils.calculateDuration(startTime); + + // Verify duration is negative + assertTrue(duration < 0); + assertTrue(duration > -2000); // Should be greater than -2 seconds + } + + /** + * Tests calculateDuration method with zero start time. + * Verifies that duration calculation works with epoch time. + */ + @Test + void testCalculateDuration_ZeroStartTime() { + // Use zero as start time (epoch) + long startTime = 0; + + // Calculate duration + long duration = TimeUtils.calculateDuration(startTime); + + // Verify duration is positive and represents current time + assertTrue(duration > 0); + + // Should be approximately current time in milliseconds + long currentTime = System.currentTimeMillis(); + assertTrue(Math.abs(duration - currentTime) < 1000); // Within 1 second tolerance + } + + /** + * Tests calculateDuration method with maximum long value. + * Verifies that duration calculation handles edge cases. + */ + @Test + void testCalculateDuration_MaxLongValue() { + // Use maximum long value as start time + long startTime = Long.MAX_VALUE; + + // Calculate duration + long duration = TimeUtils.calculateDuration(startTime); + + // Verify duration is negative (since start time is in far future) + assertTrue(duration < 0); + } + + /** + * Tests that TimeUtils class cannot be instantiated. + * Verifies that the utility class has a private constructor. + */ + @Test + void testUtilityClassCannotBeInstantiated() throws Exception { + // Access the private constructor + var constructor = TimeUtils.class.getDeclaredConstructor(); + constructor.setAccessible(true); + + // Should be able to instantiate (it's a utility class pattern) + assertDoesNotThrow(() -> { + constructor.newInstance(); + }); + } + + /** + * Tests calculateDuration method precision. + * Verifies that the method returns precise millisecond differences. + */ + @Test + void testCalculateDuration_Precision() { + long startTime = System.currentTimeMillis(); + + // Immediately calculate duration + long duration1 = TimeUtils.calculateDuration(startTime); + + // Wait a bit and calculate again + try { + Thread.sleep(5); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + + long duration2 = TimeUtils.calculateDuration(startTime); + + // Second duration should be greater than first + assertTrue(duration2 >= duration1); + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/README.md b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/README.md new file mode 100644 index 00000000000000..8b18a03576488a --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/README.md @@ -0,0 +1,491 @@ +# Hive Lineage Producer + +A Hive post-execution hook that captures lineage information from Hive queries and publishes it to Kafka for consumption by DataHub. + +## Overview + +The Hive Lineage Producer is a lightweight hook that integrates with Apache Hive to: +- Capture lineage information from Hive CTAS (CREATE TABLE AS SELECT) queries +- Extract column-level lineage (fine-grained lineage) from query execution plans +- Generate JSON lineage messages with source/target relationships +- Publish messages asynchronously to a Kafka topic + +## Architecture + +``` +Hive Query Execution + ↓ +HiveLineageLogger (Post-Execution Hook) + ↓ +Extract Lineage Info + ↓ +Build Lineage JSON + ↓ +KafkaProducerService + ↓ +Kafka Topic (HiveLineage_v1) +``` + +## Directory Structure + +``` +hive-lineage-producer/ +├── src/ +│ ├── main/ +│ │ └── java/ +│ │ └── datahub/hive/producer/ +│ │ ├── HiveLineageLogger.java # Main hook class +│ │ ├── KafkaProducerService.java # Kafka producer wrapper +│ │ └── TimeUtils.java # Utility for time calculations +│ └── test/ +│ └── java/ +│ └── datahub/hive/producer/ +│ ├── HiveLineageLoggerTest.java +│ ├── KafkaProducerServiceTest.java +│ └── TimeUtilsTest.java +├── build.gradle +└── README.md (this file) +``` + +## Components + +### 1. HiveLineageLogger + +The main hook class that: +- Implements `ExecuteWithHookContext` interface +- Runs after query execution completes +- Extracts lineage information +- Builds comprehensive lineage JSON including: + - Table-level lineage (inputs → outputs) + - Column-level lineage (fine-grained lineage) + - Query metadata (user, timestamp, duration, etc.) +- Submits lineage to Kafka asynchronously using thread pool + +### 2. KafkaProducerService + +Manages Kafka producer lifecycle: +- Creates and configures Kafka producer with SSL/TLS support +- Sends messages with configurable timeouts and retries +- Handles connection errors gracefully +- Provides proper cleanup on shutdown + +### 3. TimeUtils + +Utility class for time-related operations: +- Calculates duration between timestamps +- Used for performance monitoring + +## Building + +### Prerequisites + +- Java 17 or higher +- Gradle 8.7 or higher + +### Build Commands + +```bash +# Clean and build +./gradlew clean build + +# Build without tests +./gradlew clean build -x test + +# Build shadow JAR +./gradlew shadowJar + +# The JAR will be created at: +# build/libs/hive-lineage-producer.jar +``` + +**Output:** `hive-lineage-producer.jar` (~25-30 KB with only Kafka client dependencies) + +## Configuration + +### Configuration File + +The producer reads configuration from an XML file (default: `/etc/hive/conf/hive-lineage-config.xml`). + +**Example Configuration:** + +```xml + + + + + hive.lineage.environment + DEV + Environment name (e.g., DEV, QA, PROD) + + + + hive.lineage.platform.instance + your-hive-cluster-name + Platform instance identifier + + + + + hive.lineage.kafka.bootstrap.servers + kafka-broker1:9093,kafka-broker2:9093 + Kafka bootstrap servers + + + + hive.lineage.kafka.topic + HiveLineage_v1 + Kafka topic for lineage messages + + + + + hive.lineage.kafka.retries + 0 + Number of retries for failed Kafka sends + + + + hive.lineage.kafka.retry.backoff.ms + 100 + Backoff time in milliseconds between retry attempts + + + + hive.lineage.kafka.enable.idempotence + false + Enable idempotent producer to avoid duplicate messages + + + + hive.lineage.kafka.max.block.ms + 3000 + Maximum time in milliseconds to block waiting for Kafka metadata + + + + hive.lineage.kafka.request.timeout.ms + 3000 + Maximum time in milliseconds to wait for a Kafka request response + + + + hive.lineage.kafka.delivery.timeout.ms + 5000 + Maximum time in milliseconds for message delivery including retries + + + + hive.lineage.kafka.close.timeout.ms + 120000 + Timeout in milliseconds for closing Kafka producer gracefully + + + + + hive.lineage.kafka.ssl.truststore.location + /path/to/truststore.jks + Path to SSL truststore file + + + + hive.lineage.kafka.ssl.truststore.password + truststore-password + Truststore password + + + + hive.lineage.kafka.ssl.keystore.location + /path/to/keystore.jks + Path to SSL keystore file + + + + hive.lineage.kafka.ssl.keystore.password + keystore-password + Keystore password + + + + + hive.lineage.thread.max.pool.size + 100 + Maximum thread pool size multiplier + + + + hive.lineage.thread.queue.capacity + 500 + Thread pool queue capacity + + + + hive.lineage.thread.name + HiveLineageComputationThread- + Thread name prefix for lineage computation threads + + + + hive.lineage.thread.keep.alive.time + 60 + Keep-alive time in seconds for idle threads in the pool + + + + hive.lineage.executor.timeout.seconds + 30 + Timeout in seconds for executor service shutdown + + +``` + +### Hive Configuration + +Add the following to `hive-site.xml`: + +```xml + + + hive.exec.post.hooks + datahub.hive.producer.HiveLineageLogger + + + + + hive.lineage.hook.info.enabled + true + + + + + hive.lineage.custom.config.path + /custom/path/to/hive-lineage-config.xml + +``` + +## Deployment + +### Step 1: Copy JAR to Hive + +```bash +# Copy to Hive auxiliary library path +cp hive-lineage-producer.jar /path/to/hive/auxlib/ +``` + +### Step 2: Configure HIVE_AUX_JARS_PATH + +Add the following to your `hive-env.sh` template under Advanced hive-env configuration: + +```bash +# Adding hive lineage logger hook jar to Hive aux jars path if present +if [ -f "/path/to/hive/auxlib/hive-lineage-producer.jar" ]; then + export HIVE_AUX_JARS_PATH=${HIVE_AUX_JARS_PATH}:/path/to/hive/auxlib/hive-lineage-producer.jar +fi +``` + +This ensures the lineage JAR is available in Hive's classpath. + +### Step 3: Configure Hive + +1. Create `/etc/hive/conf/hive-lineage-config.xml` with your configuration +2. Update `hive-site.xml` with hook configuration (see Configuration section) +3. Verify Kafka connectivity and SSL certificates + +### Step 4: Restart Hive Services + +```bash +# Restart HiveServer2 +sudo systemctl restart hive-server2 + +# Or restart via Ambari/Cloudera Manager +``` + +## Testing + +### Running Unit Tests + +```bash +# Run all tests +./gradlew test + +# Run specific test class +./gradlew test --tests TimeUtilsTest +./gradlew test --tests KafkaProducerServiceTest +./gradlew test --tests HiveLineageLoggerTest + +# Run with detailed output +./gradlew test --info + +# Generate test coverage report +./gradlew jacocoTestReport +# Report available at: build/reports/tests/test/index.html +``` + +### Manual Testing + +Test the hook with a CTAS query: + +```sql +-- Enable lineage +SET hive.lineage.hook.info.enabled=true; + +-- Run a CTAS query +CREATE TABLE test_output AS +SELECT col1, col2, col3 +FROM test_input +WHERE col1 > 100; +``` + +### Verification + +1. **Check Hive Logs:** + ```bash + tail -f /var/log/hive/hiveserver2.log | grep "HiveLineageLogger" + ``` + + Look for messages like: + - "Lineage computation completed in X ms" + - "Successfully sent lineage message to Kafka" + + +2. **Verify Kafka Messages:** + ```bash + kafka-console-consumer --bootstrap-server localhost:9092 \ + --topic HiveLineage_v1 --from-beginning + ``` + +## Lineage Message Format + +The producer generates JSON messages with the following structure: + +```json +{ + "version": "1.0", + "user": "hive_user", + "timestamp": 1700000000, + "duration": 5432, + "jobIds": ["job_123456"], + "engine": "hive", + "database": "default", + "hash": "abc123def456", + "queryText": "CREATE TABLE output AS SELECT * FROM input", + "environment": "DEV", + "platformInstance": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:hive,cluster-name)", + "inputs": ["default.input_table"], + "outputs": ["default.output_table"], + "edges": [ + { + "sources": [0], + "targets": [1], + "edgeType": "PROJECTION", + "expression": "col1" + } + ], + "vertices": [ + { + "id": 0, + "vertexType": "COLUMN", + "vertexId": "input_table.col1" + }, + { + "id": 1, + "vertexType": "COLUMN", + "vertexId": "output_table.col1" + } + ] +} +``` + +## Monitoring + +### Key Metrics + +Monitor the following in Hive logs: + +- **Lineage computation time:** Time taken to extract and build lineage +- **Kafka send success/failure:** Success rate of Kafka message delivery +- **Thread pool utilization:** Active threads and queue size +- **Error rates:** Exceptions during lineage extraction or Kafka send + +## Troubleshooting + +### Hook Not Executing + +**Symptoms:** No lineage messages generated + +**Solutions:** +1. Verify JAR is in Hive classpath: + ```bash + ls -l /path/to/hive/auxlib/hive-lineage-producer.jar + ``` +2. Check `hive.exec.post.hooks` in hive-site.xml +3. Ensure `hive.lineage.hook.info.enabled=true` +4. Review HiveServer2 logs for initialization errors + +### Kafka Connection Errors + +**Symptoms:** "Failed to send lineage message" errors + +**Solutions:** +1. Verify Kafka bootstrap servers are reachable: + ```bash + telnet kafka-broker 9093 + ``` +2. Check SSL certificates and passwords +3. Verify network connectivity and firewall rules +4. Test Kafka producer manually + +### Thread Pool Exhaustion + +**Symptoms:** "Thread pool queue is full" warnings + +**Solutions:** +1. Increase `hive.lineage.thread.max.pool.size` +2. Increase `hive.lineage.thread.queue.capacity` + +### Missing Lineage Information + +**Symptoms:** Incomplete lineage in messages + +**Solutions:** +1. Ensure query is a supported CTAS operation +2. Check that source tables exist and are accessible +3. Verify `hive.lineage.hook.info.enabled=true` +4. Review query for unsupported operations + +## Performance Considerations + +### Thread Pool Sizing + +- Default max pool size: 100 threads +- Queue capacity: 500 tasks +- Adjust based on: + - Query concurrency + - Lineage computation complexity + - Available system resources + +### Kafka Timeouts + +- Keep timeouts low to avoid blocking query execution +- Default: 3-5 seconds +- Failed sends are logged but don't fail the query + +### Memory Usage + +- Hook has minimal memory footprint +- Thread pool uses bounded queue to prevent memory issues +- Each lineage message is typically < 10KB + +## Dependencies + +### Runtime Dependencies + +- Apache Hive (provided by Hive environment) +- Apache Hadoop (provided by Hadoop environment) +- Kafka Client (included in shadow JAR) + +### Test Dependencies + +- JUnit 5 (Jupiter) +- Mockito +- Kafka Test Utils + diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/build.gradle b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/build.gradle new file mode 100644 index 00000000000000..60c6bed9edbff5 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/build.gradle @@ -0,0 +1,64 @@ +plugins { + id 'java' + id 'com.github.johnrengelman.shadow' version '8.1.1' +} + +group = 'datahub.hive' + +repositories { + mavenCentral() +} + +dependencies { + implementation 'com.google.guava:guava:33.3.1-jre' + implementation 'com.google.code.gson:gson:2.11.0' + implementation 'org.apache.commons:commons-collections4:4.4' + implementation 'commons-io:commons-io:2.16.1' + implementation 'commons-lang:commons-lang:2.6' + implementation 'org.apache.hive:hive-common:3.1.3.3.3.6.4-5' + implementation 'org.apache.hive:hive-metastore:3.1.3.3.3.6.4-5' + implementation 'org.apache.hive:hive-exec:3.1.3.3.3.6.4-5' + implementation 'org.slf4j:slf4j-api:2.0.16' + implementation 'org.slf4j:slf4j-log4j12:2.0.16' + implementation 'ch.qos.logback:logback-classic:1.5.15' + implementation 'org.apache.hadoop:hadoop-common:3.4.0' + implementation 'org.apache.kafka:kafka-clients:3.9.0' + testImplementation platform('org.junit:junit-bom:5.10.0') + testImplementation 'org.junit.jupiter:junit-jupiter' + testImplementation 'org.mockito:mockito-core:5.8.0' + testImplementation 'org.mockito:mockito-junit-jupiter:5.8.0' + testImplementation 'org.apache.kafka:kafka-clients:3.9.0:test' +} + +shadowJar { + archiveBaseName.set('hive-lineage-producer') + archiveClassifier.set('') + zip64 true + + dependencies { + include(dependency('org.apache.kafka:kafka-clients:3.9.0')) + } + + dependencies { + exclude(dependency('com.google.guava:guava')) + exclude(dependency('com.google.code.gson:gson')) + exclude(dependency('org.apache.commons:commons-collections4')) + exclude(dependency('commons-io:commons-io')) + exclude(dependency('commons-lang:commons-lang')) + exclude(dependency('org.apache.hive:hive-common')) + exclude(dependency('org.apache.hive:hive-metastore')) + exclude(dependency('org.apache.hive:hive-exec')) + exclude(dependency('org.slf4j:slf4j-api')) + exclude(dependency('org.slf4j:slf4j-log4j12')) + exclude(dependency('ch.qos.logback:logback-classic')) + exclude(dependency('org.apache.hadoop:hadoop-common')) + } +} + +tasks.build { + dependsOn shadowJar +} + +test { + useJUnitPlatform() +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/main/java/datahub/hive/producer/HiveLineageLogger.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/main/java/datahub/hive/producer/HiveLineageLogger.java new file mode 100644 index 00000000000000..bf48ad047618e5 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/main/java/datahub/hive/producer/HiveLineageLogger.java @@ -0,0 +1,768 @@ +package datahub.hive.producer; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.hash.Hasher; +import com.google.common.hash.Hashing; +import com.google.gson.stream.JsonWriter; +import org.apache.commons.collections4.SetUtils; +import org.apache.commons.io.output.StringBuilderWriter; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.hive.common.ObjectPair; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.exec.ColumnInfo; +import org.apache.hadoop.hive.ql.exec.SelectOperator; +import org.apache.hadoop.hive.ql.exec.TaskRunner; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.hooks.*; +import org.apache.hadoop.hive.ql.hooks.HookContext.HookType; +import org.apache.hadoop.hive.ql.hooks.LineageInfo.BaseColumnInfo; +import org.apache.hadoop.hive.ql.hooks.LineageInfo.Dependency; +import org.apache.hadoop.hive.ql.hooks.LineageInfo.Predicate; +import org.apache.hadoop.hive.ql.optimizer.lineage.LineageCtx.Index; +import org.apache.hadoop.hive.ql.plan.HiveOperation; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; +/** + * Implementation of a post execute hook that logs lineage info to a log file. + */ +public class HiveLineageLogger implements ExecuteWithHookContext { + + private static final Logger LOG = LoggerFactory.getLogger(HiveLineageLogger.class); + + private static final HashSet OPERATION_NAMES = new HashSet(); + + private static String CONFIG_PATH = "/etc/hive/conf/hive-lineage-config.xml"; + + private static final Configuration configuration = new Configuration(); + + private static final ExecutorService executorService; + + private static final KafkaProducerService kafkaProducerService; + + static { + ExecutorService tempExecutorService = null; + KafkaProducerService tempKafkaProducerService = null; + + try { + OPERATION_NAMES.add(HiveOperation.CREATETABLE_AS_SELECT.getOperationName()); + loadCustomConfiguration(); + tempExecutorService = getExecutorService(); + LOG.info("Successfully initialized executor service"); + + tempKafkaProducerService = getKafkaProducerService(); + LOG.info("Successfully initialized Kafka producer service"); + + callExecutorShutdownHook(); + } catch (Throwable t) { + LOG.warn("Failed to initialize hive lineage hook", t); + } + + executorService = tempExecutorService; + kafkaProducerService = tempKafkaProducerService; + } + + + /** + * An edge in lineage. + */ + @VisibleForTesting + public static final class Edge { + + /** + * The types of Edge. + */ + public static enum Type { + PROJECTION, PREDICATE + } + + private Set sources; + private Set targets; + private String expr; + private Type type; + + Edge(Set sources, Set targets, String expr, Type type) { + this.sources = sources; + this.targets = targets; + this.expr = expr; + this.type = type; + } + } + + /** + * A vertex in lineage. + */ + @VisibleForTesting + public static final class Vertex { + + /** + * A type in lineage. + */ + public static enum Type { + COLUMN, TABLE + } + private Type type; + private String label; + private int id; + + Vertex(String label) { + this(label, Type.COLUMN); + } + + Vertex(String label, Type type) { + this.label = label; + this.type = type; + } + + @Override + public int hashCode() { + return label.hashCode() + type.hashCode() * 3; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof Vertex)) { + return false; + } + Vertex vertex = (Vertex) obj; + return label.equals(vertex.label) && type == vertex.type; + } + + @VisibleForTesting + public Type getType() { + return type; + } + + @VisibleForTesting + public String getLabel() { + return label; + } + + @VisibleForTesting + public int getId() { + return id; + } + } + + /** + * A custom thread factory to create threads with a specific naming pattern. + */ + private static class CustomThreadFactory implements ThreadFactory { + private final String baseName; + private final AtomicInteger threadNumber = new AtomicInteger(0); + + public CustomThreadFactory(String baseName) { + this.baseName = baseName; + } + + @Override + public Thread newThread(Runnable r) { + return new Thread(r, baseName + threadNumber.incrementAndGet()); + } + } + + /** + * Initialize the HiveLineageLogger. + * This method is called by Hive to register the hook. + */ + @Override + public void run(HookContext hookContext) { + try{ + if (executorService == null || kafkaProducerService == null) { + LOG.info("Executor or Kafka producer service not initialized, skipping hive lineage computation"); + return; + } + assert(hookContext.getHookType() == HookType.POST_EXEC_HOOK); + QueryPlan plan = hookContext.getQueryPlan(); + Index index = hookContext.getIndex(); + SessionState ss = SessionState.get(); + if (ss != null && index != null && OPERATION_NAMES.contains(plan.getOperationName()) && !plan.isExplain() + && Boolean.parseBoolean(ss.getConf().get("hive.lineage.hook.info.enabled"))) { + CompletableFuture.runAsync(() -> computeHiveLineage(plan, index, ss, hookContext), executorService); + } + } + catch(Throwable t){ + LOG.warn("Failed to initialize HiveLineageLogger", t); + } + } + + /** + * Compute the Hive lineage and send it to Kafka. + * This method is executed asynchronously in a separate thread. + */ + private void computeHiveLineage(QueryPlan plan, Index index, SessionState ss, HookContext hookContext) { + long startTime = System.currentTimeMillis(); + + try { + StringBuilderWriter out = new StringBuilderWriter(1024); + long lineageStartTime = System.currentTimeMillis(); + JsonWriter writer = new JsonWriter(out); + + PlatformConfig platformConfig = getPlatformConfig(); + String environment = platformConfig.environment(); + String platformInstance = platformConfig.platformInstance(); + String formatVersion = platformConfig.formatVersion(); + + HashSet inputs = plan.getInputs(); + HashSet outputs = plan.getOutputs(); + String queryStr = plan.getQueryStr().trim(); + writer.beginObject(); + writer.name("version").value(formatVersion); + HiveConf conf = ss.getConf(); + boolean testMode = conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST); + if (!testMode) { + // Don't emit user/timestamp info in test mode, + // so that the test golden output file is fixed. + long queryTime = plan.getQueryStartTime().longValue(); + if (queryTime == 0) queryTime = System.currentTimeMillis(); + long duration = TimeUtils.calculateDuration(queryTime); + writer.name("user").value(hookContext.getUgi().getUserName()); + writer.name("timestamp").value(queryTime/1000); + writer.name("duration").value(duration); + writer.name("jobIds"); + writer.beginArray(); + List tasks = hookContext.getCompleteTaskList(); + if (tasks != null && !tasks.isEmpty()) { + for (TaskRunner task: tasks) { + String jobId = task.getTask().getJobID(); + if (jobId != null) { + writer.value(jobId); + } + } + } + writer.endArray(); + } + writer.name("engine").value( + HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE)); + writer.name("database").value(ss.getCurrentDatabase()); + writer.name("hash").value(getQueryHash(queryStr)); + writer.name("queryText").value(queryStr); + + writer.name("environment").value(environment); + writer.name("platformInstance").value(platformInstance); + + writeEntitiesToJson(writer, inputs, "inputs"); + writeEntitiesToJson(writer, outputs, "outputs"); + + List edges = getEdges(plan, index); + Set vertices = getVertices(edges); + writeEdges(writer, edges, hookContext.getConf()); + writeVertices(writer, vertices); + writer.endObject(); + writer.close(); + + long lineageDuration = TimeUtils.calculateDuration(lineageStartTime); + LOG.info("Time taken for lineage calculation: {} ms", lineageDuration); + // Logger the lineage info + String lineage = out.toString(); + + if (testMode) { + // Logger to console + log(lineage); + } else { + // In non-test mode, emit to a log file, + // which can be different from the normal hive.log. + // For example, using NoDeleteRollingFileAppender to + // log to some file with different rolling policy. + LOG.info("Lineage message (size: {} bytes): {}", lineage.getBytes(java.nio.charset.StandardCharsets.UTF_8).length, lineage); + } + LOG.info("Sending lineage on Kafka topic"); + kafkaProducerService.sendMessage(platformInstance, lineage); + + } catch (Throwable t) { + // Don't fail the query just because of any lineage issue. + log("Failed to log lineage graph, query is not affected\n" + + org.apache.hadoop.util.StringUtils.stringifyException(t)); + } finally{ + long duration = TimeUtils.calculateDuration(startTime); + LOG.info("Total time taken for lineage calculation and Kafka message sending: {} ms", duration); + } + } + + /** + * Write out a collection of entities to JSON. + * Only write out table entities. + */ + private void writeEntitiesToJson(JsonWriter writer, Collection entities, String fieldName) + throws IOException { + writer.name(fieldName); + writer.beginArray(); + for (Entity entity : entities) { + if (entity.getType() == Entity.Type.TABLE) { + writer.value(entity.getTable().getFullyQualifiedName()); + } + } + writer.endArray(); + } + + /** + * Logger an error to console if available. + */ + private static void log(String error) { + LogHelper console = SessionState.getConsole(); + if (console != null) { + console.printError(error); + } + } + + /** + * Based on the final select operator, find out all the target columns. + * For each target column, find out its sources based on the dependency index. + */ + @VisibleForTesting + public static List getEdges(QueryPlan plan, Index index) { + LinkedHashMap> finalSelOps = index.getFinalSelectOps(); + Map vertexCache = new LinkedHashMap(); + List edges = new ArrayList(); + for (ObjectPair pair: finalSelOps.values()) { + List fieldSchemas = plan.getResultSchema().getFieldSchemas(); + SelectOperator finalSelOp = pair.getFirst(); + org.apache.hadoop.hive.ql.metadata.Table t = pair.getSecond(); + String destTableName = null; + List colNames = null; + if (t != null) { + destTableName = t.getFullyQualifiedName(); + fieldSchemas = t.getCols(); + } else { + // Based on the plan outputs, find out the target table name and column names. + for (WriteEntity output : plan.getOutputs()) { + Entity.Type entityType = output.getType(); + if (entityType == Entity.Type.TABLE + || entityType == Entity.Type.PARTITION) { + t = output.getTable(); + destTableName = t.getFullyQualifiedName(); + List cols = t.getCols(); + if (cols != null && !cols.isEmpty()) { + colNames = Utilities.getColumnNamesFromFieldSchema(cols); + } + break; + } + } + } + Map colMap = index.getDependencies(finalSelOp); + List dependencies = colMap != null ? Lists.newArrayList(colMap.values()) : null; + int fields = fieldSchemas.size(); + if (t != null && colMap != null && fields < colMap.size()) { + // Dynamic partition keys should be added to field schemas. + List partitionKeys = t.getPartitionKeys(); + int dynamicKeyCount = colMap.size() - fields; + int keyOffset = partitionKeys.size() - dynamicKeyCount; + if (keyOffset >= 0) { + fields += dynamicKeyCount; + for (int i = 0; i < dynamicKeyCount; i++) { + FieldSchema field = partitionKeys.get(keyOffset + i); + fieldSchemas.add(field); + if (colNames != null) { + colNames.add(field.getName()); + } + } + } + } + if (dependencies == null || dependencies.size() != fields) { + log("Result schema has " + fields + + " fields, but we don't get as many dependencies"); + } else { + // Go through each target column, generate the lineage edges. + Set targets = new LinkedHashSet(); + for (int i = 0; i < fields; i++) { + Vertex target = getOrCreateVertex(vertexCache, + getTargetFieldName(i, destTableName, colNames, fieldSchemas), + Vertex.Type.COLUMN); + targets.add(target); + Dependency dep = dependencies.get(i); + addEdge(vertexCache, edges, dep.getBaseCols(), target, + dep.getExpr(), Edge.Type.PROJECTION); + } + Set conds = index.getPredicates(finalSelOp); + if (conds != null && !conds.isEmpty()) { + for (Predicate cond: conds) { + addEdge(vertexCache, edges, cond.getBaseCols(), + new LinkedHashSet(targets), cond.getExpr(), + Edge.Type.PREDICATE); + } + } + } + } + return edges; + } + + private static void addEdge(Map vertexCache, List edges, + Set srcCols, Vertex target, String expr, Edge.Type type) { + Set targets = new LinkedHashSet(); + targets.add(target); + addEdge(vertexCache, edges, srcCols, targets, expr, type); + } + + /** + * Find an edge from all edges that has the same source vertices. + * If found, add the more targets to this edge's target vertex list. + * Otherwise, create a new edge and add to edge list. + */ + private static void addEdge(Map vertexCache, List edges, + Set srcCols, Set targets, String expr, Edge.Type type) { + Set sources = createSourceVertices(vertexCache, srcCols); + Edge edge = findSimilarEdgeBySources(edges, sources, expr, type); + if (edge == null) { + edges.add(new Edge(sources, targets, expr, type)); + } else { + edge.targets.addAll(targets); + } + } + + /** + * Convert a list of columns to a set of vertices. + * Use cached vertices if possible. + */ + private static Set createSourceVertices( + Map vertexCache, Collection baseCols) { + Set sources = new LinkedHashSet(); + if (baseCols != null && !baseCols.isEmpty()) { + for(BaseColumnInfo col: baseCols) { + Table table = col.getTabAlias().getTable(); + if (table.isTemporary()) { + // Ignore temporary tables + continue; + } + Vertex.Type type = Vertex.Type.TABLE; + String tableName = Warehouse.getQualifiedName(table); + FieldSchema fieldSchema = col.getColumn(); + String label = tableName; + if (fieldSchema != null) { + type = Vertex.Type.COLUMN; + label = tableName + "." + fieldSchema.getName(); + } + sources.add(getOrCreateVertex(vertexCache, label, type)); + } + } + return sources; + } + + /** + * Find a vertex from a cache, or create one if not. + */ + private static Vertex getOrCreateVertex( + Map vertices, String label, Vertex.Type type) { + Vertex vertex = vertices.get(label); + if (vertex == null) { + vertex = new Vertex(label, type); + vertices.put(label, vertex); + } + return vertex; + } + + /** + * Find an edge that has the same type, expression, and sources. + */ + private static Edge findSimilarEdgeBySources( + List edges, Set sources, String expr, Edge.Type type) { + for (Edge edge: edges) { + if (edge.type == type && StringUtils.equals(edge.expr, expr) + && SetUtils.isEqualSet(edge.sources, sources)) { + return edge; + } + } + return null; + } + + /** + * Generate normalized name for a given target column. + */ + private static String getTargetFieldName(int fieldIndex, + String destTableName, List colNames, List fieldSchemas) { + String fieldName = fieldSchemas.get(fieldIndex).getName(); + String[] parts = fieldName.split("\\."); + if (destTableName != null) { + String colName = parts[parts.length - 1]; + if (colNames != null && !colNames.contains(colName)) { + colName = colNames.get(fieldIndex); + } + return destTableName + "." + colName; + } + if (parts.length == 2 && parts[0].startsWith("_u")) { + return parts[1]; + } + return fieldName; + } + + /** + * Get all the vertices of all edges. Targets at first, + * then sources. Assign id to each vertex. + */ + @VisibleForTesting + public static Set getVertices(List edges) { + Set vertices = new LinkedHashSet(); + for (Edge edge: edges) { + vertices.addAll(edge.targets); + } + for (Edge edge: edges) { + vertices.addAll(edge.sources); + } + + // Assign ids to all vertices, + // targets at first, then sources. + int id = 0; + for (Vertex vertex: vertices) { + vertex.id = id++; + } + return vertices; + } + + /** + * Write out an JSON array of edges. + */ + private void writeEdges(JsonWriter writer, List edges, HiveConf conf) + throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException { + writer.name("edges"); + writer.beginArray(); + for (Edge edge: edges) { + writer.beginObject(); + writer.name("sources"); + writer.beginArray(); + for (Vertex vertex: edge.sources) { + writer.value(vertex.id); + } + writer.endArray(); + writer.name("targets"); + writer.beginArray(); + for (Vertex vertex: edge.targets) { + writer.value(vertex.id); + } + writer.endArray(); + if (edge.expr != null) { + writer.name("expression").value(HookUtils.redactLogString(conf, edge.expr)); + } + writer.name("edgeType").value(edge.type.name()); + writer.endObject(); + } + writer.endArray(); + } + + /** + * Write out an JSON array of vertices. + */ + private void writeVertices(JsonWriter writer, Set vertices) throws IOException { + writer.name("vertices"); + writer.beginArray(); + for (Vertex vertex: vertices) { + writer.beginObject(); + writer.name("id").value(vertex.id); + writer.name("vertexType").value(vertex.type.name()); + writer.name("vertexId").value(vertex.label); + writer.endObject(); + } + writer.endArray(); + } + + /** + * Generate query string sha256 hash. + */ + private String getQueryHash(String queryStr) { + Hasher hasher = Hashing.sha256().newHasher(); + hasher.putBytes(queryStr.getBytes(Charset.defaultCharset())); + return hasher.hash().toString(); + } + + /** + * Load custom configuration from a specified path. + * If the path is not set, it will fall back to the default configuration path. + */ + private static void loadCustomConfiguration() { + try { + if (SessionState.get() != null && SessionState.get().getConf() != null) { + String customConfigPath = SessionState.get().getConf().get("hive.lineage.custom.config.path"); + + if (customConfigPath != null) { + CONFIG_PATH = customConfigPath; + LOG.info("Loading configuration from custom path: {}", CONFIG_PATH); + configuration.addResource(new Path(CONFIG_PATH)); + } + } + } catch (Exception e) { + LOG.warn("Failed to load configuration from custom path: {}", e.getMessage()); + LOG.info("Continuing with configuration from default path"); + loadDefaultConfiguration(); + } + } + + /** + * Load the default configuration file. + * This is used when no custom configuration path is provided. + */ + private static void loadDefaultConfiguration() { + try { + LOG.info("Loading configuration from default path: {}", CONFIG_PATH); + configuration.addResource(new Path(CONFIG_PATH)); + } catch (Exception e) { + LOG.warn("Failed to load configuration from default path {}: {}", CONFIG_PATH, e.getMessage()); + } + } + + /** + * Get an ExecutorService with a custom thread pool configuration. + * The thread pool is configured based on the available processors and custom settings. + */ + private static ExecutorService getExecutorService() { + ThreadConfig threadConfig = getThreadConfig(); + return new ThreadPoolExecutor( + threadConfig.corePoolSize(), + threadConfig.maxPoolSize(), + threadConfig.keepAliveTime(), + TimeUnit.SECONDS, + new LinkedBlockingQueue(threadConfig.queueCapacity()), + new CustomThreadFactory(threadConfig.threadName()) + ); + } + + /** + * Register a shutdown hook to gracefully shut down the executor service and Kafka producer. + */ + private static void callExecutorShutdownHook() { + ThreadConfig threadConfig = getThreadConfig(); + // Adding shutdown hook to gracefully shutdown executor when JVM exits + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + kafkaProducerService.close(); + executorService.shutdown(); + try { + if (!executorService.awaitTermination(threadConfig.executorServiceTimeout(), TimeUnit.SECONDS)) { + LOG.info("Executor service did not terminate within the timeout. Initiating forceful shutdown."); + List awaitingTasks = executorService.shutdownNow(); + LOG.info("Tasks that never commenced execution: {}", awaitingTasks); + if (!executorService.awaitTermination(threadConfig.executorServiceTimeout(), TimeUnit.SECONDS)) + LOG.warn("Executor service did not terminate even after forceful shutdown."); + } + } catch (InterruptedException ie) { + List awaitingTasks = executorService.shutdownNow(); + LOG.warn("Tasks that never commenced execution after interruption: {}", awaitingTasks); + Thread.currentThread().interrupt(); + } + }, "HiveLineageShutdownHook")); + } + + /** + * Configuration for the thread pool used by the HiveLineageLogger. + */ + private record ThreadConfig ( + String threadName, + int corePoolSize, + int maxPoolSize, + long keepAliveTime, + int queueCapacity, + int executorServiceTimeout + ){} + + /** + * Get the thread configuration for the HiveLineageLogger. + * This includes core pool size, max pool size, keep alive time, queue capacity, and executor service timeout. + */ + private static ThreadConfig getThreadConfig() { + int corePoolSize = Runtime.getRuntime().availableProcessors(); + int maxPoolSize = Integer.parseInt(configuration.get("hive.lineage.thread.max.pool.size", "100")) * corePoolSize; + return new ThreadConfig( + configuration.get("hive.lineage.thread.name", "HiveLineageComputationThread-"), + corePoolSize, + maxPoolSize, + Integer.parseInt(configuration.get("hive.lineage.thread.keep.alive.time", "60")), + Integer.parseInt(configuration.get("hive.lineage.thread.queue.capacity", "500")), + Integer.parseInt(configuration.get("hive.lineage.executor.timeout.seconds", "30")) + ); + } + + /** + * Get the Kafka producer service configured with the necessary properties. + * This service is used to send lineage information to a Kafka topic. + */ + private static KafkaProducerService getKafkaProducerService() { + KafkaProducerConfig kafkaProducerConfig = getKafkaProducerConfig(); + return new KafkaProducerService(kafkaProducerConfig); + } + + /** + * Configuration for the Kafka producer used to send lineage information. + * This includes properties like bootstrap servers, truststore and keystore locations, topic name, and various timeout settings. + */ + public record KafkaProducerConfig( + String bootstrapServers, + String truststoreLocation, + String truststorePassword, + String keystoreLocation, + String keystorePassword, + String kafkaTopic, + int retries, // Controls how many times the Kafka producer will retry sending a record if the initial send fails due to a transient error (network issues, leader not available, etc.) + int retryBackoffMs, // Amount of time (in milliseconds) that the producer will wait between retry attempts + boolean enableIdempotence, // Enables idempotent message production, guarantees that even with retries, no duplicate messages will be written to a Kafka topic + int maxBlockMs, // Maximum amount of time (in milliseconds) that the send() and partitionsFor() methods will block if the producer's buffer is full or metadata is unavailable + int requestTimeoutMs, // Maximum amount of time (in milliseconds) the producer will wait for a response from the Kafka broker for a request (like sending a message) + int deliveryTimeoutMs, // Maximum time to wait for a record to be successfully sent (acknowledged by Kafka), including retries, before giving up and failing the send + int closeTimeoutMs // Maximum time (in milliseconds) to wait for the producer to close gracefully + ) {} + + /** + * Get the Kafka producer configuration based on the properties defined in the Hive configuration. + * This includes environment, platform instance, bootstrap servers, truststore and keystore locations, topic name, and various timeout settings. + */ + private static KafkaProducerConfig getKafkaProducerConfig() { + return new KafkaProducerConfig( + configuration.get("hive.lineage.kafka.bootstrap.servers"), + configuration.get("hive.lineage.kafka.ssl.truststore.location"), + configuration.get("hive.lineage.kafka.ssl.truststore.password"), + configuration.get("hive.lineage.kafka.ssl.keystore.location"), + configuration.get("hive.lineage.kafka.ssl.keystore.password"), + configuration.get("hive.lineage.kafka.topic", "HiveLineage_v1"), + Integer.parseInt(configuration.get("hive.lineage.kafka.retries", "0")), + Integer.parseInt(configuration.get("hive.lineage.kafka.retry.backoff.ms", "100")), + Boolean.parseBoolean(configuration.get("hive.lineage.kafka.enable.idempotence", "false")), + Integer.parseInt(configuration.get("hive.lineage.kafka.max.block.ms", "3000")), + Integer.parseInt(configuration.get("hive.lineage.kafka.request.timeout.ms", "3000")), + Integer.parseInt(configuration.get("hive.lineage.kafka.delivery.timeout.ms", "5000")), + Integer.parseInt(configuration.get("hive.lineage.kafka.close.timeout.ms", "30000")) + ); + } + + /** + * Configuration for the platform instance, including environment and format version. + * This is used to provide context for the lineage information being logged. + */ + private record PlatformConfig( + String environment, + String platformInstance, + String formatVersion + ) {} + + /** + * Get the platform configuration based on the properties defined in the Hive configuration. + * This includes environment, platform instance, and format version. + */ + private static PlatformConfig getPlatformConfig() { + return new PlatformConfig( + configuration.get("hive.lineage.environment"), + configuration.get("hive.lineage.platform.instance"), + configuration.get("hive.lineage.format.version", "1.0") + ); + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/main/java/datahub/hive/producer/KafkaProducerService.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/main/java/datahub/hive/producer/KafkaProducerService.java new file mode 100644 index 00000000000000..7316f3c2dedfa9 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/main/java/datahub/hive/producer/KafkaProducerService.java @@ -0,0 +1,73 @@ +package datahub.hive.producer; + +import datahub.hive.producer.HiveLineageLogger.KafkaProducerConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.serialization.StringSerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.Properties; +import java.util.concurrent.Future; + +public class KafkaProducerService { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaProducerService.class); + private final KafkaProducer producer; + private final KafkaProducerConfig kafkaProducerConfig; + + public KafkaProducerService(KafkaProducerConfig kafkaProducerConfig) { + this.kafkaProducerConfig = kafkaProducerConfig; + + Properties props = new Properties(); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaProducerConfig.bootstrapServers()); + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + + props.put(ProducerConfig.RETRIES_CONFIG, kafkaProducerConfig.retries()); + props.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, kafkaProducerConfig.retryBackoffMs()); + props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, kafkaProducerConfig.enableIdempotence()); + + props.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, kafkaProducerConfig.maxBlockMs()); + props.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, kafkaProducerConfig.requestTimeoutMs()); + props.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, kafkaProducerConfig.deliveryTimeoutMs()); + + props.put("security.protocol", "SSL"); + props.put("ssl.truststore.location", kafkaProducerConfig.truststoreLocation()); + props.put("ssl.truststore.password", kafkaProducerConfig.truststorePassword()); + props.put("ssl.keystore.location", kafkaProducerConfig.keystoreLocation()); + props.put("ssl.keystore.password", kafkaProducerConfig.keystorePassword()); + + this.producer = new KafkaProducer<>(props); + } + + public void sendMessage(String key, String value) { + long startTime = System.currentTimeMillis(); + ProducerRecord record = new ProducerRecord<>(kafkaProducerConfig.kafkaTopic(), key, value); + try { + Future future = producer.send(record); + RecordMetadata metadata = future.get(); + long duration = TimeUtils.calculateDuration(startTime); + LOG.info("Sent message with key: {}, value: {} to partition: {}, offset: {}. Time taken: {} ms", + key, value, metadata.partition(), metadata.offset(), duration); + } catch (Exception e) { + long duration = TimeUtils.calculateDuration(startTime); + LOG.warn("Failed to send message with key: {}, value: {}. Time taken before failure: {} ms", + key, value, duration, e); + } + } + + public void close() { + if (producer != null) { + try { + producer.close(Duration.ofMillis(kafkaProducerConfig.closeTimeoutMs())); + LOG.info("Kafka producer closed successfully with timeout: {} ms", kafkaProducerConfig.closeTimeoutMs()); + } catch (Exception e) { + LOG.warn("Error closing Kafka producer", e); + } + } + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/main/java/datahub/hive/producer/TimeUtils.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/main/java/datahub/hive/producer/TimeUtils.java new file mode 100644 index 00000000000000..a620f25d6092bf --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/main/java/datahub/hive/producer/TimeUtils.java @@ -0,0 +1,14 @@ +package datahub.hive.producer; + +/** + * Utility class for time-related operations. + */ +public class TimeUtils { + + /** + * Calculates and returns the duration in milliseconds between a start time and the current time. + */ + public static long calculateDuration(long startTime) { + return System.currentTimeMillis() - startTime; + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/test/java/datahub/hive/producer/HiveLineageLoggerTest.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/test/java/datahub/hive/producer/HiveLineageLoggerTest.java new file mode 100644 index 00000000000000..dfc7c30fa3960f --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/test/java/datahub/hive/producer/HiveLineageLoggerTest.java @@ -0,0 +1,318 @@ +package datahub.hive.producer; + +import org.apache.hadoop.hive.ql.hooks.LineageInfo; +import org.apache.hadoop.hive.ql.optimizer.lineage.LineageCtx.Index; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.BeforeEach; + +import java.util.*; + +import static org.junit.jupiter.api.Assertions.*; + +/** + * Unit tests for HiveLineageLogger class. + * Tests focus on the public static methods and inner classes that are marked as @VisibleForTesting. + */ +class HiveLineageLoggerTest { + + @BeforeEach + void setUp() { + // Setup before each test + } + + @Test + void testVertex_creation() { + // Given & When + HiveLineageLogger.Vertex vertex = new HiveLineageLogger.Vertex("db.table.column"); + + // Then + assertNotNull(vertex); + assertEquals("db.table.column", vertex.getLabel()); + assertEquals(HiveLineageLogger.Vertex.Type.COLUMN, vertex.getType()); + } + + @Test + void testVertex_creationWithType() { + // Given & When + HiveLineageLogger.Vertex vertex = new HiveLineageLogger.Vertex("db.table", HiveLineageLogger.Vertex.Type.TABLE); + + // Then + assertNotNull(vertex); + assertEquals("db.table", vertex.getLabel()); + assertEquals(HiveLineageLogger.Vertex.Type.TABLE, vertex.getType()); + } + + @Test + void testVertex_equality() { + // Given + HiveLineageLogger.Vertex vertex1 = new HiveLineageLogger.Vertex("db.table.column"); + HiveLineageLogger.Vertex vertex2 = new HiveLineageLogger.Vertex("db.table.column"); + HiveLineageLogger.Vertex vertex3 = new HiveLineageLogger.Vertex("db.table.other"); + + // Then + assertEquals(vertex1, vertex2); + assertNotEquals(vertex1, vertex3); + assertEquals(vertex1.hashCode(), vertex2.hashCode()); + } + + @Test + void testVertex_equalityWithDifferentTypes() { + // Given + HiveLineageLogger.Vertex vertex1 = new HiveLineageLogger.Vertex("db.table", HiveLineageLogger.Vertex.Type.COLUMN); + HiveLineageLogger.Vertex vertex2 = new HiveLineageLogger.Vertex("db.table", HiveLineageLogger.Vertex.Type.TABLE); + + // Then + assertNotEquals(vertex1, vertex2); + } + + @Test + void testVertex_equalsSameObject() { + // Given + HiveLineageLogger.Vertex vertex = new HiveLineageLogger.Vertex("db.table.column"); + + // Then + assertEquals(vertex, vertex); + } + + @Test + void testVertex_equalsNull() { + // Given + HiveLineageLogger.Vertex vertex = new HiveLineageLogger.Vertex("db.table.column"); + + // Then + assertNotEquals(vertex, null); + } + + @Test + void testVertex_equalsDifferentClass() { + // Given + HiveLineageLogger.Vertex vertex = new HiveLineageLogger.Vertex("db.table.column"); + String notAVertex = "db.table.column"; + + // Then + assertNotEquals(vertex, notAVertex); + } + + @Test + void testEdge_creation() { + // Given + Set sources = new LinkedHashSet<>(); + sources.add(new HiveLineageLogger.Vertex("source.table.col1")); + + Set targets = new LinkedHashSet<>(); + targets.add(new HiveLineageLogger.Vertex("target.table.col1")); + + String expression = "col1 + 1"; + + // When + HiveLineageLogger.Edge edge = new HiveLineageLogger.Edge( + sources, + targets, + expression, + HiveLineageLogger.Edge.Type.PROJECTION + ); + + // Then + assertNotNull(edge); + } + + @Test + void testEdge_projectionType() { + // Given + Set sources = new LinkedHashSet<>(); + Set targets = new LinkedHashSet<>(); + + // When + HiveLineageLogger.Edge edge = new HiveLineageLogger.Edge( + sources, + targets, + "expression", + HiveLineageLogger.Edge.Type.PROJECTION + ); + + // Then + assertNotNull(edge); + } + + @Test + void testEdge_predicateType() { + // Given + Set sources = new LinkedHashSet<>(); + Set targets = new LinkedHashSet<>(); + + // When + HiveLineageLogger.Edge edge = new HiveLineageLogger.Edge( + sources, + targets, + "WHERE condition", + HiveLineageLogger.Edge.Type.PREDICATE + ); + + // Then + assertNotNull(edge); + } + + @Test + void testGetVertices_emptyEdges() { + // Given + List edges = new ArrayList<>(); + + // When + Set vertices = HiveLineageLogger.getVertices(edges); + + // Then + assertNotNull(vertices); + assertTrue(vertices.isEmpty()); + } + + @Test + void testGetVertices_singleEdge() { + // Given + HiveLineageLogger.Vertex source = new HiveLineageLogger.Vertex("source.table.col1"); + HiveLineageLogger.Vertex target = new HiveLineageLogger.Vertex("target.table.col1"); + + Set sources = new LinkedHashSet<>(); + sources.add(source); + + Set targets = new LinkedHashSet<>(); + targets.add(target); + + List edges = new ArrayList<>(); + edges.add(new HiveLineageLogger.Edge(sources, targets, "expr", HiveLineageLogger.Edge.Type.PROJECTION)); + + // When + Set vertices = HiveLineageLogger.getVertices(edges); + + // Then + assertNotNull(vertices); + assertEquals(2, vertices.size()); + assertTrue(vertices.contains(source)); + assertTrue(vertices.contains(target)); + } + + @Test + void testGetVertices_multipleEdges() { + // Given + HiveLineageLogger.Vertex source1 = new HiveLineageLogger.Vertex("source1.table.col1"); + HiveLineageLogger.Vertex source2 = new HiveLineageLogger.Vertex("source2.table.col2"); + HiveLineageLogger.Vertex target1 = new HiveLineageLogger.Vertex("target.table.col1"); + HiveLineageLogger.Vertex target2 = new HiveLineageLogger.Vertex("target.table.col2"); + + Set sources1 = new LinkedHashSet<>(); + sources1.add(source1); + + Set targets1 = new LinkedHashSet<>(); + targets1.add(target1); + + Set sources2 = new LinkedHashSet<>(); + sources2.add(source2); + + Set targets2 = new LinkedHashSet<>(); + targets2.add(target2); + + List edges = new ArrayList<>(); + edges.add(new HiveLineageLogger.Edge(sources1, targets1, "expr1", HiveLineageLogger.Edge.Type.PROJECTION)); + edges.add(new HiveLineageLogger.Edge(sources2, targets2, "expr2", HiveLineageLogger.Edge.Type.PROJECTION)); + + // When + Set vertices = HiveLineageLogger.getVertices(edges); + + // Then + assertNotNull(vertices); + assertEquals(4, vertices.size()); + } + + @Test + void testGetVertices_duplicateVertices() { + // Given + HiveLineageLogger.Vertex source = new HiveLineageLogger.Vertex("source.table.col1"); + HiveLineageLogger.Vertex target = new HiveLineageLogger.Vertex("target.table.col1"); + + Set sources1 = new LinkedHashSet<>(); + sources1.add(source); + + Set targets1 = new LinkedHashSet<>(); + targets1.add(target); + + Set sources2 = new LinkedHashSet<>(); + sources2.add(source); // Same source vertex + + Set targets2 = new LinkedHashSet<>(); + targets2.add(target); // Same target vertex + + List edges = new ArrayList<>(); + edges.add(new HiveLineageLogger.Edge(sources1, targets1, "expr1", HiveLineageLogger.Edge.Type.PROJECTION)); + edges.add(new HiveLineageLogger.Edge(sources2, targets2, "expr2", HiveLineageLogger.Edge.Type.PROJECTION)); + + // When + Set vertices = HiveLineageLogger.getVertices(edges); + + // Then + assertNotNull(vertices); + assertEquals(2, vertices.size()); // Should only have 2 unique vertices + } + + @Test + void testGetVertices_assignsIds() { + // Given + HiveLineageLogger.Vertex source = new HiveLineageLogger.Vertex("source.table.col1"); + HiveLineageLogger.Vertex target = new HiveLineageLogger.Vertex("target.table.col1"); + + Set sources = new LinkedHashSet<>(); + sources.add(source); + + Set targets = new LinkedHashSet<>(); + targets.add(target); + + List edges = new ArrayList<>(); + edges.add(new HiveLineageLogger.Edge(sources, targets, "expr", HiveLineageLogger.Edge.Type.PROJECTION)); + + // When + Set vertices = HiveLineageLogger.getVertices(edges); + + // Then + assertNotNull(vertices); + List vertexList = new ArrayList<>(vertices); + assertEquals(0, vertexList.get(0).getId()); // First vertex should have id 0 + assertEquals(1, vertexList.get(1).getId()); // Second vertex should have id 1 + } + + @Test + void testGetVertices_targetsBeforeSources() { + // Given + HiveLineageLogger.Vertex source = new HiveLineageLogger.Vertex("source.table.col1"); + HiveLineageLogger.Vertex target = new HiveLineageLogger.Vertex("target.table.col1"); + + Set sources = new LinkedHashSet<>(); + sources.add(source); + + Set targets = new LinkedHashSet<>(); + targets.add(target); + + List edges = new ArrayList<>(); + edges.add(new HiveLineageLogger.Edge(sources, targets, "expr", HiveLineageLogger.Edge.Type.PROJECTION)); + + // When + Set vertices = HiveLineageLogger.getVertices(edges); + + // Then - targets should come before sources in the set + List vertexList = new ArrayList<>(vertices); + assertEquals(target, vertexList.get(0)); // Target should be first + assertEquals(source, vertexList.get(1)); // Source should be second + } + + @Test + void testVertexTypes_enum() { + // Test that enum values exist + assertEquals(HiveLineageLogger.Vertex.Type.COLUMN, HiveLineageLogger.Vertex.Type.valueOf("COLUMN")); + assertEquals(HiveLineageLogger.Vertex.Type.TABLE, HiveLineageLogger.Vertex.Type.valueOf("TABLE")); + } + + @Test + void testEdgeTypes_enum() { + // Test that enum values exist + assertEquals(HiveLineageLogger.Edge.Type.PROJECTION, HiveLineageLogger.Edge.Type.valueOf("PROJECTION")); + assertEquals(HiveLineageLogger.Edge.Type.PREDICATE, HiveLineageLogger.Edge.Type.valueOf("PREDICATE")); + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/test/java/datahub/hive/producer/KafkaProducerServiceTest.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/test/java/datahub/hive/producer/KafkaProducerServiceTest.java new file mode 100644 index 00000000000000..27807c11ade90f --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/test/java/datahub/hive/producer/KafkaProducerServiceTest.java @@ -0,0 +1,220 @@ +package datahub.hive.producer; + +import datahub.hive.producer.HiveLineageLogger.KafkaProducerConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.MockedConstruction; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.*; + +/** + * Unit tests for KafkaProducerService class. + */ +@ExtendWith(MockitoExtension.class) +class KafkaProducerServiceTest { + + private KafkaProducerConfig config; + + @Mock + private KafkaProducer mockProducer; + + @BeforeEach + void setUp() { + config = new KafkaProducerConfig( + "localhost:9092", + "/path/to/truststore", + "truststore-password", + "/path/to/keystore", + "keystore-password", + "test-topic", + 3, + 100, + true, + 3000, + 3000, + 5000, + 30000 + ); + } + + @AfterEach + void tearDown() { + // Clean up after each test + } + + @Test + void testSendMessage_success() throws Exception { + // Given + String key = "test-key"; + String value = "{\"test\":\"value\"}"; + + RecordMetadata metadata = new RecordMetadata( + new TopicPartition("test-topic", 0), + 0L, + 0L, + System.currentTimeMillis(), + 0L, + 0, + 0 + ); + + Future future = CompletableFuture.completedFuture(metadata); + + try (MockedConstruction mockedProducer = mockConstruction(KafkaProducer.class, + (mock, context) -> { + when(mock.send(any(ProducerRecord.class))).thenReturn(future); + })) { + + // When + KafkaProducerService service = new KafkaProducerService(config); + service.sendMessage(key, value); + + // Then + KafkaProducer producer = mockedProducer.constructed().get(0); + verify(producer, times(1)).send(any(ProducerRecord.class)); + } + } + + @Test + void testSendMessage_failure() throws Exception { + // Given + String key = "test-key"; + String value = "{\"test\":\"value\"}"; + + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(new RuntimeException("Kafka send failed")); + + try (MockedConstruction mockedProducer = mockConstruction(KafkaProducer.class, + (mock, context) -> { + when(mock.send(any(ProducerRecord.class))).thenReturn(future); + })) { + + // When + KafkaProducerService service = new KafkaProducerService(config); + service.sendMessage(key, value); + + // Then - should not throw exception, just log warning + KafkaProducer producer = mockedProducer.constructed().get(0); + verify(producer, times(1)).send(any(ProducerRecord.class)); + } + } + + @Test + void testClose_success() throws Exception { + // Given + try (MockedConstruction mockedProducer = mockConstruction(KafkaProducer.class, + (mock, context) -> { + doNothing().when(mock).close(any()); + })) { + + // When + KafkaProducerService service = new KafkaProducerService(config); + service.close(); + + // Then + KafkaProducer producer = mockedProducer.constructed().get(0); + verify(producer, times(1)).close(any()); + } + } + + @Test + void testClose_withException() throws Exception { + // Given + try (MockedConstruction mockedProducer = mockConstruction(KafkaProducer.class, + (mock, context) -> { + doThrow(new RuntimeException("Close failed")).when(mock).close(any()); + })) { + + // When + KafkaProducerService service = new KafkaProducerService(config); + service.close(); + + // Then - should not throw exception, just log warning + KafkaProducer producer = mockedProducer.constructed().get(0); + verify(producer, times(1)).close(any()); + } + } + + @Test + void testMultipleSendMessages() throws Exception { + // Given + String key1 = "key1"; + String value1 = "{\"test\":\"value1\"}"; + String key2 = "key2"; + String value2 = "{\"test\":\"value2\"}"; + + RecordMetadata metadata = new RecordMetadata( + new TopicPartition("test-topic", 0), + 0L, + 0L, + System.currentTimeMillis(), + 0L, + 0, + 0 + ); + + Future future = CompletableFuture.completedFuture(metadata); + + try (MockedConstruction mockedProducer = mockConstruction(KafkaProducer.class, + (mock, context) -> { + when(mock.send(any(ProducerRecord.class))).thenReturn(future); + })) { + + // When + KafkaProducerService service = new KafkaProducerService(config); + service.sendMessage(key1, value1); + service.sendMessage(key2, value2); + + // Then + KafkaProducer producer = mockedProducer.constructed().get(0); + verify(producer, times(2)).send(any(ProducerRecord.class)); + } + } + + @Test + void testKafkaProducerConfig_values() { + // Given & When + KafkaProducerConfig testConfig = new KafkaProducerConfig( + "broker1:9092,broker2:9092", + "/custom/truststore", + "custom-trust-pass", + "/custom/keystore", + "custom-key-pass", + "custom-topic", + 5, + 200, + false, + 5000, + 5000, + 10000, + 60000 + ); + + // Then + assert testConfig.bootstrapServers().equals("broker1:9092,broker2:9092"); + assert testConfig.truststoreLocation().equals("/custom/truststore"); + assert testConfig.truststorePassword().equals("custom-trust-pass"); + assert testConfig.keystoreLocation().equals("/custom/keystore"); + assert testConfig.keystorePassword().equals("custom-key-pass"); + assert testConfig.kafkaTopic().equals("custom-topic"); + assert testConfig.retries() == 5; + assert testConfig.retryBackoffMs() == 200; + assert !testConfig.enableIdempotence(); + assert testConfig.maxBlockMs() == 5000; + assert testConfig.requestTimeoutMs() == 5000; + assert testConfig.deliveryTimeoutMs() == 10000; + assert testConfig.closeTimeoutMs() == 60000; + } +} diff --git a/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/test/java/datahub/hive/producer/TimeUtilsTest.java b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/test/java/datahub/hive/producer/TimeUtilsTest.java new file mode 100644 index 00000000000000..4d927bec3b4e96 --- /dev/null +++ b/metadata-integration/java/acryl-hive-lineage/hive-lineage-producer/src/test/java/datahub/hive/producer/TimeUtilsTest.java @@ -0,0 +1,79 @@ +package datahub.hive.producer; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.*; + +/** + * Unit tests for TimeUtils class. + */ +class TimeUtilsTest { + + @Test + void testCalculateDuration_withValidStartTime() { + // Given + long startTime = System.currentTimeMillis() - 1000; // 1 second ago + + // When + long duration = TimeUtils.calculateDuration(startTime); + + // Then + assertTrue(duration >= 1000, "Duration should be at least 1000ms"); + assertTrue(duration < 2000, "Duration should be less than 2000ms"); + } + + @Test + void testCalculateDuration_withCurrentTime() { + // Given + long startTime = System.currentTimeMillis(); + + // When + long duration = TimeUtils.calculateDuration(startTime); + + // Then + assertTrue(duration >= 0, "Duration should be non-negative"); + assertTrue(duration < 100, "Duration should be very small for current time"); + } + + @Test + void testCalculateDuration_withMultipleInvocations() throws InterruptedException { + // Given + long startTime = System.currentTimeMillis(); + Thread.sleep(100); // Sleep for 100ms + + // When + long duration1 = TimeUtils.calculateDuration(startTime); + Thread.sleep(100); // Sleep for another 100ms + long duration2 = TimeUtils.calculateDuration(startTime); + + // Then + assertTrue(duration2 > duration1, "Second duration should be greater than first"); + assertTrue(duration2 - duration1 >= 100, "Difference should be at least 100ms"); + } + + @Test + void testCalculateDuration_withOldStartTime() { + // Given + long startTime = System.currentTimeMillis() - 5000; // 5 seconds ago + + // When + long duration = TimeUtils.calculateDuration(startTime); + + // Then + assertTrue(duration >= 5000, "Duration should be at least 5000ms"); + assertTrue(duration < 6000, "Duration should be less than 6000ms"); + } + + @Test + void testCalculateDuration_precision() { + // Given + long startTime = System.currentTimeMillis(); + + // When - call immediately + long duration = TimeUtils.calculateDuration(startTime); + + // Then - should return a precise value + assertNotNull(duration); + assertTrue(duration >= 0, "Duration should be non-negative"); + } +}