diff --git a/.circleci/config.yml b/.circleci/config.yml index 258fb2da1feea..f4a7596db644d 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -245,7 +245,7 @@ jobs: - v1-integration-spark-{{ .Branch }}-{{ .Revision }} - v1-integration-spark-{{ .Branch }} - run: (cd ./../../client/java/ && ./gradlew --no-daemon --stacktrace publishToMavenLocal) - - run: ./gradlew --no-daemon --stacktrace integrationTest -Pspark.version=<< parameters.spark-version >> + - run: ./gradlew --no-daemon --info integrationTest -Pspark.version=<< parameters.spark-version >> - run: when: on_fail command: cat integration/spark/build/test-results/integrationTest/TEST-*.xml diff --git a/CHANGELOG.md b/CHANGELOG.md index d0a0bff01560a..24db8f97be417 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,6 +10,7 @@ * dbt: allow parsing artifacts with version higher than officially supported [@mobuchowski](https://github.com/mobuchowski) * dbt: dbt build command is supported [@mobuchowski](https://github.com/mobuchowski) * dbt: fix crash when build command is used with seeds in dbt 1.0.0rc3 [@mobuchowski](https://github.com/mobuchowski) +* spark: increase logical plan visitor coverage [@mobuchowski](https://github.com/mobuchowski) ## [0.3.1](https://github.com/OpenLineage/OpenLineage/releases/tag/0.3.1) - 2021-10-21 diff --git a/integration/spark/build.gradle b/integration/spark/build.gradle index f2a32eaaa017b..7272c76145f30 100644 --- a/integration/spark/build.gradle +++ b/integration/spark/build.gradle @@ -66,8 +66,9 @@ archivesBaseName='openlineage-spark' ext { assertjVersion = '3.20.2' + bigqueryVersion = '0.21.1' junit5Version = '5.7.2' - sparkVersion = '2.4.7' + sparkVersion = '2.4.8' jacksonVersion = '2.6.7' jacksonVersionSpark3 = '2.10.0' jacksonModuleScalaVersion = '2.6.7.1' @@ -126,40 +127,45 @@ dependencies { testCompile "org.assertj:assertj-core:${assertjVersion}" testCompile "org.junit.jupiter:junit-jupiter:${junit5Version}" testCompile "org.mockito:mockito-core:${mockitoVersion}" + testCompile "org.mockito:mockito-inline:${mockitoVersion}" testCompile "org.mockito:mockito-junit-jupiter:${mockitoVersion}" - testCompileOnly "org.projectlombok:lombok:${lombokVersion}" + testCompile "org.projectlombok:lombok:${lombokVersion}" testCompileOnly "org.apache.spark:spark-core_2.11:${sparkVersion}" testCompileOnly "org.apache.spark:spark-sql_2.11:${sparkVersion}" - testCompileOnly 'com.google.cloud.spark:spark-bigquery-with-dependencies_2.11:0.21.1' + testCompileOnly "org.apache.spark:spark-hive_2.11:${sparkVersion}" + testCompileOnly "com.google.cloud.spark:spark-bigquery-with-dependencies_2.11:${bigqueryVersion}" spark3Test "com.fasterxml.jackson.core:jackson-databind:${jacksonVersionSpark3}" spark3Test "com.fasterxml.jackson.core:jackson-core:${jacksonVersionSpark3}" spark3Test "com.fasterxml.jackson.module:jackson-module-scala_2.12:${jacksonVersionSpark3}" - spark3Test 'com.google.cloud.spark:spark-bigquery-with-dependencies_2.12:0.21.1' + spark3Test "com.google.cloud.spark:spark-bigquery-with-dependencies_2.12:${bigqueryVersion}" spark3Test "org.apache.spark:spark-core_2.12:${spark3Version}" spark3Test "org.apache.spark:spark-sql_2.12:${spark3Version}" spark3Test "org.apache.spark:spark-hive_2.12:${spark3Version}" spark3Test "org.apache.spark:spark-catalyst_2.12:${spark3Version}" spark3Test kafkaPackage('2.12', spark3Version) + spark3Test "org.apache.iceberg:iceberg-spark3-runtime:0.12.1" + spark3Test "io.delta:delta-core_2.12:1.0.0" spark2Test "com.fasterxml.jackson.core:jackson-databind:${jacksonDatabindVersion}" spark2Test "com.fasterxml.jackson.core:jackson-core:${jacksonVersion}" spark2Test "com.fasterxml.jackson.module:jackson-module-scala_2.11:${jacksonVersion}" - spark2Test 'com.google.cloud.spark:spark-bigquery-with-dependencies_2.11:0.21.1' + spark2Test "com.google.cloud.spark:spark-bigquery-with-dependencies_2.11:${bigqueryVersion}" spark2Test "org.apache.spark:spark-core_2.11:${sparkVersion}" - spark2Test "org.apache.spark:spark-sql_2.11:${sparkVersion}" spark2Test "org.apache.spark:spark-hive_2.11:${sparkVersion}" + spark2Test "org.apache.spark:spark-sql_2.11:${sparkVersion}" spark2Test kafkaPackage('2.11', sparkVersion) spark3 "org.projectlombok:lombok:${lombokVersion}" - spark3 'io.openlineage:openlineage-java:0.0.1-SNAPSHOT' spark3 "com.fasterxml.jackson.module:jackson-module-scala_2.12:${jacksonVersionSpark3}" - spark3 "com.google.cloud.spark:spark-bigquery_2.12:0.21.1" - spark3 "com.google.cloud.spark:spark-bigquery-with-dependencies_2.12:0.21.1" + spark3 "com.google.cloud.spark:spark-bigquery_2.12:${bigqueryVersion}" + spark3 "com.google.cloud.spark:spark-bigquery-with-dependencies_2.12:${bigqueryVersion}" spark3 "org.apache.spark:spark-core_2.12:${spark3Version}" spark3 "org.apache.spark:spark-sql_2.12:${spark3Version}" spark3 "org.apache.spark:spark-hive_2.12:${spark3Version}" spark3 kafkaPackage('2.12', spark3Version) + spark3 "org.apache.iceberg:iceberg-spark3-runtime:0.12.1" + spark3 "io.delta:delta-core_2.12:1.0.0" lombok "org.projectlombok:lombok:${lombokVersion}" testAnnotationProcessor "org.projectlombok:lombok:${lombokVersion}" @@ -167,7 +173,7 @@ dependencies { sourceSets { main.java.srcDirs = ["src/main/common/java", "src/main/spark2/java"] - test.java.srcDirs = ["src/test/common/java"] + test.java.srcDirs = ["src/test/common/java", "src/test/spark2/java"] spark3{ java { @@ -180,7 +186,7 @@ sourceSets { spark3Test { java { - srcDirs = ["src/test/spark3/java"] + srcDirs = ["src/test/common/java", "src/test/spark3/java"] compileClasspath = configurations.spark3Test + sourceSets.main.output annotationProcessorPath = configurations.lombok destinationDirectory.set(file("$buildDir/classes/java/test/")) @@ -232,6 +238,7 @@ test { useJUnitPlatform { excludeTags 'integration-test' } + if (isSpark3) exclude 'io/openlineage/spark2/**' if (!isSpark3) exclude 'io/openlineage/spark3/**' dependsOn copyDependencies } @@ -241,9 +248,13 @@ task integrationTest(type: Test) { useJUnitPlatform { includeTags "integration-test" } + if (isSpark3) exclude 'io/openlineage/spark2/**' + if (!isSpark3) exclude 'io/openlineage/spark3/**' dependsOn copyDependencies, shadowJar } +integrationTest.outputs.upToDateWhen { false } + spotless { def disallowWildcardImports = { String text = it diff --git a/integration/spark/gradle.properties b/integration/spark/gradle.properties index c82832b0c8699..0b59e3d75892f 100644 --- a/integration/spark/gradle.properties +++ b/integration/spark/gradle.properties @@ -1,3 +1,4 @@ jdk8.build=true version=0.4.0-SNAPSHOT spark.version=3.1.2 +org.gradle.jvmargs=-Xmx1G \ No newline at end of file diff --git a/integration/spark/integrations/container/pysparkAlterTableAddColumnsEnd.json b/integration/spark/integrations/container/pysparkAlterTableAddColumnsEnd.json new file mode 100644 index 0000000000000..db9529f33b57a --- /dev/null +++ b/integration/spark/integrations/container/pysparkAlterTableAddColumnsEnd.json @@ -0,0 +1,33 @@ +{ + "eventType" : "COMPLETE", + "job" : { + "namespace" : "testAlterTable", + "name" : "open_lineage_integration_alter_table.execute_alter_table_add_columns_command" + }, + "inputs" : [ ], + "outputs" : [ { + "namespace" : "file", + "name" : "/tmp/alter_test/alter_table_test", + "facets" : { + "dataSource" : { + "name" : "file", + "uri" : "file" + }, + "schema" : { + "fields" : [ { + "name" : "a", + "type" : "string" + }, { + "name" : "b", + "type" : "string" + }, { + "name" : "c", + "type" : "string" + }, { + "name" : "d", + "type" : "string" + } ] + } + } + } ] +} \ No newline at end of file diff --git a/integration/spark/integrations/container/pysparkAlterTableRenameEnd.json b/integration/spark/integrations/container/pysparkAlterTableRenameEnd.json new file mode 100644 index 0000000000000..7efbc37df59dd --- /dev/null +++ b/integration/spark/integrations/container/pysparkAlterTableRenameEnd.json @@ -0,0 +1,37 @@ +{ + "eventType" : "COMPLETE", + "job" : { + "namespace" : "testAlterTable", + "name" : "open_lineage_integration_alter_table.execute_alter_table_rename_command" + }, + "inputs" : [ ], + "outputs" : [ { + "namespace" : "file", + "name" : "/tmp/alter_test/alter_table_test_new", + "facets" : { + "dataSource" : { + "name" : "file", + "uri" : "file" + }, + "schema" : { + "fields" : [ { + "name" : "a", + "type" : "string" + }, { + "name" : "b", + "type" : "string" + }, { + "name" : "c", + "type" : "string" + }, { + "name" : "d", + "type" : "string" + } ] + }, + "previousTableName" : { + "previousTableURI" : "/tmp/alter_test/alter_table_test", + "currentTableURI" : "/tmp/alter_test/alter_table_test_new" + } + } + } ] +} \ No newline at end of file diff --git a/integration/spark/integrations/container/pysparkCTASEnd.json b/integration/spark/integrations/container/pysparkCTASEnd.json new file mode 100644 index 0000000000000..61b98c6482e47 --- /dev/null +++ b/integration/spark/integrations/container/pysparkCTASEnd.json @@ -0,0 +1,27 @@ +{ + "eventType" : "COMPLETE", + "job" : { + "namespace" : "testCreateAsSelectAndLoad", + "name" : "open_lineage_integration_cta_s_load.execute_create_hive_table_as_select_command" + }, + "inputs" : [ ], + "outputs" : [ { + "namespace" : "file", + "name" : "/tmp/ctas_load/tbl1", + "facets" : { + "dataSource" : { + "name" : "file", + "uri" : "file" + }, + "schema" : { + "fields" : [ { + "name" : "a", + "type" : "long" + }, { + "name" : "b", + "type" : "long" + } ] + } + } + } ] +} \ No newline at end of file diff --git a/integration/spark/integrations/container/pysparkCTASStart.json b/integration/spark/integrations/container/pysparkCTASStart.json new file mode 100644 index 0000000000000..a960d14166e12 --- /dev/null +++ b/integration/spark/integrations/container/pysparkCTASStart.json @@ -0,0 +1,27 @@ +{ + "eventType" : "START", + "job" : { + "namespace" : "testCreateAsSelectAndLoad", + "name" : "open_lineage_integration_cta_s_load.execute_create_hive_table_as_select_command" + }, + "inputs" : [ ], + "outputs" : [ { + "namespace" : "file", + "name" : "/tmp/ctas_load/tbl1", + "facets" : { + "dataSource" : { + "name" : "file", + "uri" : "file" + }, + "schema" : { + "fields" : [ { + "name" : "a", + "type" : "long" + }, { + "name" : "b", + "type" : "long" + } ] + } + } + } ] +} \ No newline at end of file diff --git a/integration/spark/integrations/container/pysparkCreateTableCompleteEvent.json b/integration/spark/integrations/container/pysparkCreateTableCompleteEvent.json new file mode 100644 index 0000000000000..0ff99d7bf4217 --- /dev/null +++ b/integration/spark/integrations/container/pysparkCreateTableCompleteEvent.json @@ -0,0 +1,32 @@ +{ + "eventType": "COMPLETE", + "job": { + "namespace": "testCreateTable", + "name": "open_lineage_integration_create_table.execute_create_table_command" + }, + "inputs": [], + "outputs": [ + { + "namespace": "file", + "name": "/tmp/create_test/create_table_test", + "facets": { + "dataSource": { + "name": "file", + "uri": "file" + }, + "schema": { + "fields": [ + { + "name": "a", + "type": "string" + }, + { + "name": "b", + "type": "string" + } + ] + } + } + } + ] +} diff --git a/integration/spark/integrations/container/pysparkCreateTableStartEvent.json b/integration/spark/integrations/container/pysparkCreateTableStartEvent.json new file mode 100644 index 0000000000000..158ad71c5c7a6 --- /dev/null +++ b/integration/spark/integrations/container/pysparkCreateTableStartEvent.json @@ -0,0 +1,32 @@ +{ + "eventType": "START", + "job": { + "namespace": "testCreateTable", + "name": "open_lineage_integration_create_table.execute_create_table_command" + }, + "inputs": [], + "outputs": [ + { + "namespace": "file", + "name": "/tmp/create_test/create_table_test", + "facets": { + "dataSource": { + "name": "file", + "uri": "file" + }, + "schema": { + "fields": [ + { + "name": "a", + "type": "string" + }, + { + "name": "b", + "type": "string" + } + ] + } + } + } + ] +} diff --git a/integration/spark/integrations/container/pysparkDeltaCTASComplete.json b/integration/spark/integrations/container/pysparkDeltaCTASComplete.json new file mode 100644 index 0000000000000..85e0e902c648b --- /dev/null +++ b/integration/spark/integrations/container/pysparkDeltaCTASComplete.json @@ -0,0 +1,33 @@ +{ + "eventType": "COMPLETE", + "inputs": [], + "job": { + "name": "open_lineage_integration_delta.atomic_create_table_as_select", + "namespace": "testCTASDelta" + }, + "outputs": [ + { + "facets": { + "dataSource": { + "name": "file", + "uri": "file" + }, + "schema": { + "fields": [ + { + "name": "a", + "type": "long" + }, + { + "name": "b", + "type": "long" + } + ] + } + }, + "name": "/tmp/delta", + "namespace": "file" + } + ], + "run": {} +} \ No newline at end of file diff --git a/integration/spark/integrations/container/pysparkDeltaViewComplete.json b/integration/spark/integrations/container/pysparkDeltaViewComplete.json new file mode 100644 index 0000000000000..66d1b675daaa0 --- /dev/null +++ b/integration/spark/integrations/container/pysparkDeltaViewComplete.json @@ -0,0 +1,7 @@ +{ + "eventType": "COMPLETE", + "job": { + "namespace": "testCreateAsSelectAndLoad", + "name": "open_lineage_integration_delta.execute_create_view_command" + } +} \ No newline at end of file diff --git a/integration/spark/integrations/container/pysparkDeltaViewStart.json b/integration/spark/integrations/container/pysparkDeltaViewStart.json new file mode 100644 index 0000000000000..5209eba6785e0 --- /dev/null +++ b/integration/spark/integrations/container/pysparkDeltaViewStart.json @@ -0,0 +1,7 @@ +{ + "eventType": "START", + "job": { + "namespace": "testCreateAsSelectAndLoad", + "name": "open_lineage_integration_delta.execute_create_view_command" + } +} \ No newline at end of file diff --git a/integration/spark/integrations/container/pysparkDropTableStartEvent.json b/integration/spark/integrations/container/pysparkDropTableStartEvent.json new file mode 100644 index 0000000000000..fb602b44662ca --- /dev/null +++ b/integration/spark/integrations/container/pysparkDropTableStartEvent.json @@ -0,0 +1,23 @@ +{ + "eventType": "START", + "job": { + "namespace": "testDropTable", + "name": "open_lineage_integration_drop_table.execute_drop_table_command" + }, + "inputs": [], + "outputs": [ + { + "namespace": "file", + "name": "/tmp/drop_test/drop_table_test", + "facets": { + "dataSource": { + "name": "file", + "uri": "file" + }, + "tableStateChange": { + "stateChange": "drop" + } + } + } + ] +} diff --git a/integration/spark/integrations/container/pysparkLoadComplete.json b/integration/spark/integrations/container/pysparkLoadComplete.json new file mode 100644 index 0000000000000..63742164cfd1c --- /dev/null +++ b/integration/spark/integrations/container/pysparkLoadComplete.json @@ -0,0 +1,27 @@ +{ + "eventType" : "COMPLETE", + "job" : { + "namespace" : "testCreateAsSelectAndLoad", + "name" : "open_lineage_integration_cta_s_load.execute_load_data_command" + }, + "inputs" : [ ], + "outputs" : [ { + "namespace" : "file", + "name" : "/tmp/ctas_load/tbl1", + "facets" : { + "dataSource" : { + "name" : "file", + "uri" : "file" + }, + "schema" : { + "fields" : [ { + "name" : "a", + "type" : "long" + }, { + "name" : "b", + "type" : "long" + } ] + } + } + } ] +} \ No newline at end of file diff --git a/integration/spark/integrations/container/pysparkLoadStart.json b/integration/spark/integrations/container/pysparkLoadStart.json new file mode 100644 index 0000000000000..adbdf09e3128b --- /dev/null +++ b/integration/spark/integrations/container/pysparkLoadStart.json @@ -0,0 +1,27 @@ +{ + "eventType" : "START", + "job" : { + "namespace" : "testCreateAsSelectAndLoad", + "name" : "open_lineage_integration_cta_s_load.execute_load_data_command" + }, + "inputs" : [ ], + "outputs" : [ { + "namespace" : "file", + "name" : "/tmp/ctas_load/tbl1", + "facets" : { + "dataSource" : { + "name" : "file", + "uri" : "file" + }, + "schema" : { + "fields" : [ { + "name" : "a", + "type" : "long" + }, { + "name" : "b", + "type" : "long" + } ] + } + } + } ] +} \ No newline at end of file diff --git a/integration/spark/integrations/container/pysparkOCTASEnd.json b/integration/spark/integrations/container/pysparkOCTASEnd.json new file mode 100644 index 0000000000000..f87b7f0361885 --- /dev/null +++ b/integration/spark/integrations/container/pysparkOCTASEnd.json @@ -0,0 +1,27 @@ +{ + "eventType" : "COMPLETE", + "job" : { + "namespace" : "testOptimizedCreateAsSelectAndLoad", + "name" : "open_lineage_integration_octa_s_load.execute_optimized_create_hive_table_as_select_command" + }, + "inputs" : [ ], + "outputs" : [ { + "namespace" : "file", + "name" : "/tmp/ctas_load/tbl2", + "facets" : { + "dataSource" : { + "name" : "file", + "uri" : "file" + }, + "schema" : { + "fields" : [ { + "name" : "a", + "type" : "long" + }, { + "name" : "b", + "type" : "long" + } ] + } + } + } ] +} \ No newline at end of file diff --git a/integration/spark/integrations/container/pysparkOCTASStart.json b/integration/spark/integrations/container/pysparkOCTASStart.json new file mode 100644 index 0000000000000..794da92e44ea8 --- /dev/null +++ b/integration/spark/integrations/container/pysparkOCTASStart.json @@ -0,0 +1,27 @@ +{ + "eventType" : "START", + "job" : { + "namespace" : "testOptimizedCreateAsSelectAndLoad", + "name" : "open_lineage_integration_octa_s_load.execute_optimized_create_hive_table_as_select_command" + }, + "inputs" : [ ], + "outputs" : [ { + "namespace" : "file", + "name" : "/tmp/ctas_load/tbl2", + "facets" : { + "dataSource" : { + "name" : "file", + "uri" : "file" + }, + "schema" : { + "fields" : [ { + "name" : "a", + "type" : "long" + }, { + "name" : "b", + "type" : "long" + } ] + } + } + } ] +} \ No newline at end of file diff --git a/integration/spark/integrations/sparkrdd/1.json b/integration/spark/integrations/sparkrdd/1.json index 04a3163c37b9c..72ee0d7466809 100644 --- a/integration/spark/integrations/sparkrdd/1.json +++ b/integration/spark/integrations/sparkrdd/1.json @@ -5,15 +5,11 @@ "runId" : "fake_run_id", "facets" : { "parent" : { - "_producer" : "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "run" : { - "runId" : "8d99e33e-2a1c-4254-9600-18f23435fc3b" - }, + "run" : {}, "job" : { "namespace" : "ns_name", "name" : "job_name" - }, - "_schemaURL" : "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/ParentRunFacet" + } }, "spark_version": { } @@ -27,7 +23,5 @@ "namespace" : "gs.bucket", "name" : "gs://bucket/data.txt" } ], - "outputs" : [ ], - "producer" : "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "schemaURL" : "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/RunEvent" + "outputs" : [ ] } \ No newline at end of file diff --git a/integration/spark/integrations/sparkrdd/2.json b/integration/spark/integrations/sparkrdd/2.json index 5a31b3ec575d4..4e42272178f77 100644 --- a/integration/spark/integrations/sparkrdd/2.json +++ b/integration/spark/integrations/sparkrdd/2.json @@ -5,15 +5,12 @@ "runId" : "fake_run_id", "facets" : { "parent" : { - "_producer" : "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", "run" : { - "runId" : "8d99e33e-2a1c-4254-9600-18f23435fc3b" }, "job" : { "namespace" : "ns_name", "name" : "job_name" - }, - "_schemaURL" : "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/ParentRunFacet" + } }, "spark_version": { } @@ -27,7 +24,5 @@ "namespace" : "gs.bucket", "name" : "gs://bucket/data.txt" } ], - "outputs" : [ ], - "producer" : "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "schemaURL" : "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/RunEvent" + "outputs" : [ ] } \ No newline at end of file diff --git a/integration/spark/integrations/sparksql/1.json b/integration/spark/integrations/sparksql/1.json index 43958a12e2c82..3318093ad3550 100644 --- a/integration/spark/integrations/sparksql/1.json +++ b/integration/spark/integrations/sparksql/1.json @@ -2,22 +2,15 @@ "eventType": "START", "eventTime": "2021-01-01T00:00:00Z", "run": { - "runId": "fake_run_id", "facets": { "parent": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/ParentRunFacet", - "run": { - "runId": "ea445b5c-22eb-457a-8007-01c7c52b6e54" - }, + "run": {}, "job": { "namespace": "ns_name", "name": "job_name" } }, "spark_unknown": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/CustomFacet", "output": { "description": { "@class": "org.apache.spark.sql.catalyst.plans.logical.Aggregate", @@ -29,47 +22,14 @@ "explicitMetadata": { "map": {} }, - "origin": { - "line": null, - "startPosition": null - }, "deterministic": true, "resolved": true } ], - "origin": { - "line": null, - "startPosition": null - }, - "schema": [ - { - "name": "count", - "dataType": { - "numeric": {}, - "integral": {}, - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], "allAttributes": { "attrs": [] }, "resolved": true, - "constraints": [ - { - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - ], "statsCache": null, "traceEnabled": false, "streaming": false, @@ -87,8 +47,6 @@ "inputs": [] }, "spark.logicalPlan": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/CustomFacet", "plan": [ { "class": "org.apache.spark.sql.catalyst.plans.logical.Aggregate", @@ -202,16 +160,13 @@ }, "job": { "namespace": "ns_name", - "name": "word_count.hash_aggregate" + "name": "test_spark_sql.hash_aggregate" }, "inputs": [ { "namespace": "file", - "name": "/path/to/data", "facets": { "schema": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/SchemaDatasetFacet", "fields": [ { "name": "value", @@ -220,15 +175,11 @@ ] }, "dataSource": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/DatasourceDatasetFacet", "uri": "file", "name": "file" } } } ], - "outputs": [], - "producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/RunEvent" + "outputs": [] } \ No newline at end of file diff --git a/integration/spark/integrations/sparksql/2.json b/integration/spark/integrations/sparksql/2.json index 8955f74b75b40..a4cab1a55d69b 100644 --- a/integration/spark/integrations/sparksql/2.json +++ b/integration/spark/integrations/sparksql/2.json @@ -2,22 +2,15 @@ "eventType": "COMPLETE", "eventTime": "2021-01-01T00:00:00Z", "run": { - "runId": "fake_run_id", "facets": { "parent": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/ParentRunFacet", - "run": { - "runId": "ea445b5c-22eb-457a-8007-01c7c52b6e54" - }, + "run": {}, "job": { "namespace": "ns_name", "name": "job_name" } }, "spark_unknown": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/CustomFacet", "output": { "description": { "@class": "org.apache.spark.sql.catalyst.plans.logical.Aggregate", @@ -29,47 +22,14 @@ "explicitMetadata": { "map": {} }, - "origin": { - "line": null, - "startPosition": null - }, "deterministic": true, "resolved": true } ], - "origin": { - "line": null, - "startPosition": null - }, - "schema": [ - { - "name": "count", - "dataType": { - "numeric": {}, - "integral": {}, - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], "allAttributes": { "attrs": [] }, "resolved": true, - "constraints": [ - { - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - ], "statsCache": null, "traceEnabled": false, "streaming": false, @@ -87,8 +47,6 @@ "inputs": [] }, "spark.logicalPlan": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/CustomFacet", "plan": [ { "class": "org.apache.spark.sql.catalyst.plans.logical.Aggregate", @@ -219,16 +177,13 @@ }, "job": { "namespace": "ns_name", - "name": "word_count.hash_aggregate" + "name": "test_spark_sql.hash_aggregate" }, "inputs": [ { "namespace": "file", - "name": "/path/to/data", "facets": { "schema": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/SchemaDatasetFacet", "fields": [ { "name": "value", @@ -237,15 +192,11 @@ ] }, "dataSource": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/DatasourceDatasetFacet", "uri": "file", "name": "file" } } } ], - "outputs": [], - "producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/RunEvent" + "outputs": [] } \ No newline at end of file diff --git a/integration/spark/integrations/sparksql/3.json b/integration/spark/integrations/sparksql/3.json index d9a9f334fd8f4..55f617eedabf6 100644 --- a/integration/spark/integrations/sparksql/3.json +++ b/integration/spark/integrations/sparksql/3.json @@ -2,22 +2,15 @@ "eventType": "START", "eventTime": "2021-01-01T00:00:00Z", "run": { - "runId": "fake_run_id", "facets": { "parent": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/ParentRunFacet", - "run": { - "runId": "ea445b5c-22eb-457a-8007-01c7c52b6e54" - }, + "run": {}, "job": { "namespace": "ns_name", "name": "job_name" } }, "spark_unknown": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/CustomFacet", "output": { "description": { "@class": "org.apache.spark.sql.catalyst.plans.logical.Aggregate", @@ -29,47 +22,14 @@ "explicitMetadata": { "map": {} }, - "origin": { - "line": null, - "startPosition": null - }, "deterministic": true, "resolved": true } ], - "origin": { - "line": null, - "startPosition": null - }, - "schema": [ - { - "name": "count", - "dataType": { - "numeric": {}, - "integral": {}, - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], "allAttributes": { "attrs": [] }, "resolved": true, - "constraints": [ - { - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - ], "statsCache": null, "traceEnabled": false, "streaming": false, @@ -87,8 +47,6 @@ "inputs": [] }, "spark.logicalPlan": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/CustomFacet", "plan": [ { "class": "org.apache.spark.sql.catalyst.plans.logical.Aggregate", @@ -201,16 +159,13 @@ }, "job": { "namespace": "ns_name", - "name": "word_count.hash_aggregate" + "name": "test_spark_sql.hash_aggregate" }, "inputs": [ { "namespace": "file", - "name": "/path/to/data", "facets": { "schema": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/SchemaDatasetFacet", "fields": [ { "name": "value", @@ -219,15 +174,11 @@ ] }, "dataSource": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/DatasourceDatasetFacet", "uri": "file", "name": "file" } } } ], - "outputs": [], - "producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/RunEvent" + "outputs": [] } \ No newline at end of file diff --git a/integration/spark/integrations/sparksql/4.json b/integration/spark/integrations/sparksql/4.json index 2707611eb4eca..171ecb1c2de1a 100644 --- a/integration/spark/integrations/sparksql/4.json +++ b/integration/spark/integrations/sparksql/4.json @@ -2,22 +2,15 @@ "eventType": "COMPLETE", "eventTime": "2021-01-01T00:00:00Z", "run": { - "runId": "fake_run_id", "facets": { "parent": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/ParentRunFacet", - "run": { - "runId": "ea445b5c-22eb-457a-8007-01c7c52b6e54" - }, + "run": {}, "job": { "namespace": "ns_name", "name": "job_name" } }, "spark_unknown": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/CustomFacet", "output": { "description": { "@class": "org.apache.spark.sql.catalyst.plans.logical.Aggregate", @@ -29,47 +22,14 @@ "explicitMetadata": { "map": {} }, - "origin": { - "line": null, - "startPosition": null - }, "deterministic": true, "resolved": true } ], - "origin": { - "line": null, - "startPosition": null - }, - "schema": [ - { - "name": "count", - "dataType": { - "numeric": {}, - "integral": {}, - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], "allAttributes": { "attrs": [] }, "resolved": true, - "constraints": [ - { - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - ], "statsCache": null, "traceEnabled": false, "streaming": false, @@ -87,8 +47,6 @@ "inputs": [] }, "spark.logicalPlan": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/CustomFacet", "plan": [ { "class": "org.apache.spark.sql.catalyst.plans.logical.Aggregate", @@ -218,16 +176,13 @@ }, "job": { "namespace": "ns_name", - "name": "word_count.hash_aggregate" + "name": "test_spark_sql.hash_aggregate" }, "inputs": [ { "namespace": "file", - "name": "/path/to/data", "facets": { "schema": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/SchemaDatasetFacet", "fields": [ { "name": "value", @@ -236,15 +191,11 @@ ] }, "dataSource": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/DatasourceDatasetFacet", "uri": "file", "name": "file" } } } ], - "outputs": [], - "producer": "https://github.com/OpenLineage/OpenLineage/tree/$VERSION/integration/spark", - "schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/RunEvent" + "outputs": [] } \ No newline at end of file diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/OpenLineageSparkListener.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/OpenLineageSparkListener.java index 45db7103486af..b9ee797f2e8d5 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/OpenLineageSparkListener.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/OpenLineageSparkListener.java @@ -167,6 +167,7 @@ public void onJobStart(SparkListenerJobStart jobStart) { } else { context = getExecutionContext(job.jobId()); } + context.setActiveJob(job); context.start(jobStart); }); diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/facets/PreviousTableNameFacet.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/facets/PreviousTableNameFacet.java new file mode 100644 index 0000000000000..9b78f852fded2 --- /dev/null +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/facets/PreviousTableNameFacet.java @@ -0,0 +1,30 @@ +package io.openlineage.spark.agent.facets; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.client.OpenLineageClient; +import lombok.Getter; + +/** + * Facet used to indicate previous table name, when table was renamed as a result of a spark job + * execution + */ +@Getter +public class PreviousTableNameFacet extends OpenLineage.DefaultDatasetFacet { + + @JsonProperty("previousTableURI") + private String previousTableUri; + + @JsonProperty("currentTableURI") + private String currentTableUri; + + /** + * @param previousTableUri previous table name + * @param currentTableUri current table name + */ + public PreviousTableNameFacet(String previousTableUri, String currentTableUri) { + super(OpenLineageClient.OPEN_LINEAGE_CLIENT_URI); + this.previousTableUri = previousTableUri; + this.currentTableUri = currentTableUri; + } +} diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/facets/TableStateChangeFacet.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/facets/TableStateChangeFacet.java new file mode 100644 index 0000000000000..9bd6eac4b282e --- /dev/null +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/facets/TableStateChangeFacet.java @@ -0,0 +1,32 @@ +package io.openlineage.spark.agent.facets; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonValue; +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.client.OpenLineageClient; +import lombok.Getter; + +/** Facet used to notify state change performed on table like "CREATE", "DROP" or "TRUNCATE". */ +@Getter +public class TableStateChangeFacet extends OpenLineage.DefaultDatasetFacet { + + public enum StateChange { + CREATE, + DROP, + TRUNCATE; + + @JsonValue + @Override + public String toString() { + return name().toLowerCase(); + } + } + + @JsonProperty("stateChange") + private StateChange stateChange; + + public TableStateChangeFacet(StateChange stateChange) { + super(OpenLineageClient.OPEN_LINEAGE_CLIENT_URI); + this.stateChange = stateChange; + } +} diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/BaseVisitorFactory.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/BaseVisitorFactory.java index 9c6be756698cb..1f6651bed1d7f 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/BaseVisitorFactory.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/BaseVisitorFactory.java @@ -1,9 +1,16 @@ package io.openlineage.spark.agent.lifecycle; import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.lifecycle.plan.AlterTableAddColumnsCommandVisitor; +import io.openlineage.spark.agent.lifecycle.plan.AlterTableRenameCommandVisitor; import io.openlineage.spark.agent.lifecycle.plan.AppendDataVisitor; import io.openlineage.spark.agent.lifecycle.plan.BigQueryNodeVisitor; import io.openlineage.spark.agent.lifecycle.plan.CommandPlanVisitor; +import io.openlineage.spark.agent.lifecycle.plan.CreateDataSourceTableAsSelectCommandVisitor; +import io.openlineage.spark.agent.lifecycle.plan.CreateDataSourceTableCommandVisitor; +import io.openlineage.spark.agent.lifecycle.plan.CreateHiveTableAsSelectCommandVisitor; +import io.openlineage.spark.agent.lifecycle.plan.CreateTableCommandVisitor; +import io.openlineage.spark.agent.lifecycle.plan.DropTableCommandVisitor; import io.openlineage.spark.agent.lifecycle.plan.InsertIntoDataSourceDirVisitor; import io.openlineage.spark.agent.lifecycle.plan.InsertIntoDataSourceVisitor; import io.openlineage.spark.agent.lifecycle.plan.InsertIntoDirVisitor; @@ -11,8 +18,10 @@ import io.openlineage.spark.agent.lifecycle.plan.InsertIntoHiveDirVisitor; import io.openlineage.spark.agent.lifecycle.plan.InsertIntoHiveTableVisitor; import io.openlineage.spark.agent.lifecycle.plan.KafkaRelationVisitor; +import io.openlineage.spark.agent.lifecycle.plan.LoadDataCommandVisitor; import io.openlineage.spark.agent.lifecycle.plan.LogicalRDDVisitor; import io.openlineage.spark.agent.lifecycle.plan.LogicalRelationVisitor; +import io.openlineage.spark.agent.lifecycle.plan.OptimizedCreateHiveTableAsSelectCommandVisitor; import io.openlineage.spark.agent.lifecycle.plan.QueryPlanVisitor; import io.openlineage.spark.agent.lifecycle.plan.SaveIntoDataSourceCommandVisitor; import io.openlineage.spark.agent.lifecycle.plan.wrapper.InputDatasetVisitor; @@ -64,12 +73,26 @@ public List> getOutputV list.add( new OutputDatasetVisitor( new SaveIntoDataSourceCommandVisitor(sqlContext, allCommonVisitors))); + list.add(new OutputDatasetVisitor(new CreateDataSourceTableAsSelectCommandVisitor())); list.add(new OutputDatasetVisitor(new AppendDataVisitor(allCommonVisitors))); - list.add(new OutputDatasetVisitor(new InsertIntoDirVisitor(sqlContext))); + list.add(new OutputDatasetVisitor(new InsertIntoDirVisitor())); if (InsertIntoHiveTableVisitor.hasHiveClasses()) { - list.add(new OutputDatasetVisitor(new InsertIntoHiveTableVisitor(sqlContext.sparkContext()))); + list.add(new OutputDatasetVisitor(new InsertIntoHiveTableVisitor())); list.add(new OutputDatasetVisitor(new InsertIntoHiveDirVisitor())); + list.add(new OutputDatasetVisitor(new CreateHiveTableAsSelectCommandVisitor())); } + if (OptimizedCreateHiveTableAsSelectCommandVisitor.hasClasses()) { + list.add(new OutputDatasetVisitor(new OptimizedCreateHiveTableAsSelectCommandVisitor())); + } + list.add(new OutputDatasetVisitor(new CreateDataSourceTableCommandVisitor())); + list.add(new OutputDatasetVisitor(new LoadDataCommandVisitor(sqlContext.sparkSession()))); + list.add( + new OutputDatasetVisitor(new AlterTableRenameCommandVisitor(sqlContext.sparkSession()))); + list.add( + new OutputDatasetVisitor( + new AlterTableAddColumnsCommandVisitor(sqlContext.sparkSession()))); + list.add(new OutputDatasetVisitor(new CreateTableCommandVisitor())); + list.add(new OutputDatasetVisitor(new DropTableCommandVisitor(sqlContext.sparkSession()))); return list; } } diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/ContextFactory.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/ContextFactory.java index cb077e6bbb815..894b939f4203e 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/ContextFactory.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/ContextFactory.java @@ -13,6 +13,7 @@ import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.QueryExecution; import org.apache.spark.sql.execution.SQLExecution; import scala.PartialFunction; @@ -29,7 +30,8 @@ public RddExecutionContext createRddExecutionContext(int jobId) { } public SparkSQLExecutionContext createSparkSQLExecutionContext(long executionId) { - SQLContext sqlContext = SQLExecution.getQueryExecution(executionId).sparkPlan().sqlContext(); + QueryExecution queryExecution = SQLExecution.getQueryExecution(executionId); + SQLContext sqlContext = queryExecution.sparkPlan().sqlContext(); VisitorFactory visitorFactory = VisitorFactoryProvider.getInstance(SparkSession.active()); @@ -39,7 +41,8 @@ public SparkSQLExecutionContext createSparkSQLExecutionContext(long executionId) List> outputDatasets = visitorFactory.getOutputVisitors(sqlContext, sparkContext.getJobNamespace()); - return new SparkSQLExecutionContext(executionId, sparkContext, outputDatasets, inputDatasets); + return new SparkSQLExecutionContext( + executionId, sparkContext, queryExecution, outputDatasets, inputDatasets); } private List>> commonDatasetVisitors( diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/SparkSQLExecutionContext.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/SparkSQLExecutionContext.java index 4130db8a93bf3..1f44dadacc716 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/SparkSQLExecutionContext.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/SparkSQLExecutionContext.java @@ -25,6 +25,7 @@ import java.util.Objects; import java.util.Optional; import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.spark.SparkContext; @@ -36,7 +37,6 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; import org.apache.spark.sql.execution.QueryExecution; -import org.apache.spark.sql.execution.SQLExecution; import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.execution.WholeStageCodegenExec; import org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd; @@ -61,29 +61,64 @@ public class SparkSQLExecutionContext implements ExecutionContext { private final OpenLineage openLineage = new OpenLineage(OpenLineageClient.OPEN_LINEAGE_CLIENT_URI); + private AtomicBoolean started = new AtomicBoolean(false); + private AtomicBoolean finished = new AtomicBoolean(false); + private Optional jobId = Optional.empty(); + public SparkSQLExecutionContext( long executionId, OpenLineageContext sparkContext, + QueryExecution queryExecution, List> outputDatasetSupplier, List> inputDatasetSupplier) { this.executionId = executionId; this.sparkContext = sparkContext; - this.queryExecution = SQLExecution.getQueryExecution(executionId); + this.queryExecution = queryExecution; this.outputDatasetSupplier = outputDatasetSupplier; this.inputDatasetSupplier = inputDatasetSupplier; this.jobMetrics = JobMetricsHolder.getInstance(); } - public void start(SparkListenerSQLExecutionStart startEvent) {} + public void start(SparkListenerSQLExecutionStart startEvent) { + log.debug("SparkListenerSQLExecutionStart - executionId: {}", startEvent.executionId()); + startEvent(startEvent.time()); + } - public void end(SparkListenerSQLExecutionEnd endEvent) {} + public void end(SparkListenerSQLExecutionEnd endEvent) { + log.debug("SparkListenerSQLExecutionEnd - executionId: {}", endEvent.executionId()); + // TODO: can we get failed event here? + // If not, then we probably need to use this only for LogicalPlans that emit no Job events. + // Maybe use QueryExecutionListener? + endEvent(endEvent.time(), "COMPLETE", null); + } @Override public void setActiveJob(ActiveJob activeJob) {} @Override public void start(SparkListenerJobStart jobStart) { - log.info("Starting job as part of spark-sql:" + jobStart.jobId()); + log.debug("SparkListenerJobStart - executionId: " + executionId); + jobId = Optional.of(jobStart.jobId()); + startEvent(jobStart.time()); + } + + @Override + public void end(SparkListenerJobEnd jobEnd) { + log.debug("SparkListenerJobEnd - executionId: " + executionId); + Exception exception = null; + if (jobEnd.jobResult() instanceof JobFailed) { + exception = ((JobFailed) jobEnd.jobResult()).exception(); + } + jobId = Optional.of(jobEnd.jobId()); + endEvent(jobEnd.time(), getEventType(jobEnd.jobResult()), exception); + } + + void startEvent(Long time) { + if (!started.compareAndSet(false, true)) { + log.debug("Start event already emitted: returning"); + return; + } + if (queryExecution == null) { log.info("No execution info {}", queryExecution); return; @@ -106,7 +141,7 @@ public void start(SparkListenerJobStart jobStart) { OpenLineage.RunEvent event = openLineage .newRunEventBuilder() - .eventTime(toZonedTime(jobStart.time())) + .eventTime(toZonedTime(time)) .eventType("START") .inputs(inputDatasets) .outputs(outputDatasets) @@ -123,7 +158,7 @@ public void start(SparkListenerJobStart jobStart) { .job(buildJob(queryExecution)) .build(); - log.debug("Posting event for start {}: {}", jobStart, event); + log.debug("Posting event for start {}: {}", executionId, event); sparkContext.emit(event); } @@ -155,9 +190,12 @@ private Optional buildParentFacet() { runId, sparkContext.getParentJobName(), sparkContext.getJobNamespace())); } - @Override - public void end(SparkListenerJobEnd jobEnd) { - log.info("Ending job as part of spark-sql:" + jobEnd.jobId()); + void endEvent(Long time, String eventType, Exception exception) { + if (!finished.compareAndSet(false, true)) { + log.debug("Event already finished, returning"); + return; + } + if (queryExecution == null) { log.info("No execution info {}", queryExecution); return; @@ -167,6 +205,7 @@ public void end(SparkListenerJobEnd jobEnd) { log.debug("Traversing optimized plan {}", optimizedPlan.toJSON()); log.debug("Physical plan executed {}", queryExecution.executedPlan().toJSON()); } + PartialFunction> outputVisitor = merge(outputDatasetSupplier); PartialFunction> planTraversal = @@ -175,7 +214,9 @@ public void end(SparkListenerJobEnd jobEnd) { planTraversal.isDefinedAt(optimizedPlan) ? planTraversal.apply(optimizedPlan) : Collections.emptyList(); - outputDatasets = populateOutputMetrics(jobEnd.jobId(), outputDatasets); + if (jobId.isPresent()) { + outputDatasets = populateOutputMetrics(jobId.get(), outputDatasets); + } List inputDatasets = getInputDatasets(); UnknownEntryFacet unknownFacet = unknownEntryFacetListener.build(optimizedPlan).orElse(null); @@ -183,8 +224,8 @@ public void end(SparkListenerJobEnd jobEnd) { OpenLineage.RunEvent event = openLineage .newRunEventBuilder() - .eventTime(toZonedTime(jobEnd.time())) - .eventType(getEventType(jobEnd.jobResult())) + .eventTime(toZonedTime(time)) + .eventType(eventType) .inputs(inputDatasets) .outputs(outputDatasets) .run( @@ -194,14 +235,14 @@ public void end(SparkListenerJobEnd jobEnd) { new SimpleImmutableEntry( "spark.logicalPlan", buildLogicalPlanFacet(queryExecution.logical())), new SimpleImmutableEntry( - "spark.exception", buildJobErrorFacet(jobEnd.jobResult())), + "spark.exception", buildJobErrorFacet(eventType, exception)), new SimpleImmutableEntry("spark_unknown", unknownFacet), new SimpleImmutableEntry( "spark_version", new SparkVersionFacet(SparkSession.active()))))) .job(buildJob(queryExecution)) .build(); - log.debug("Posting event for end {}: {}", jobEnd, event); + log.debug("Posting event for end {}: {}", executionId, event); sparkContext.emit(event); } @@ -276,9 +317,9 @@ protected LogicalPlanFacet buildLogicalPlanFacet(LogicalPlan plan) { return LogicalPlanFacet.builder().plan(plan).build(); } - protected ErrorFacet buildJobErrorFacet(JobResult jobResult) { - if (jobResult instanceof JobFailed && ((JobFailed) jobResult).exception() != null) { - return ErrorFacet.builder().exception(((JobFailed) jobResult).exception()).build(); + protected ErrorFacet buildJobErrorFacet(String jobResult, Exception exception) { + if (jobResult.equals("FAIL") && exception != null) { + return ErrorFacet.builder().exception(exception).build(); } return null; } diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/AlterTableAddColumnsCommandVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/AlterTableAddColumnsCommandVisitor.java new file mode 100644 index 0000000000000..32f4697dc6571 --- /dev/null +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/AlterTableAddColumnsCommandVisitor.java @@ -0,0 +1,47 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.util.PathUtils; +import io.openlineage.spark.agent.util.PlanUtils; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import lombok.SneakyThrows; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.command.AlterTableAddColumnsCommand; +import org.apache.spark.sql.types.StructField; +import scala.collection.JavaConversions; + +public class AlterTableAddColumnsCommandVisitor + extends QueryPlanVisitor { + + private final SparkSession sparkSession; + + public AlterTableAddColumnsCommandVisitor(SparkSession sparkSession) { + this.sparkSession = sparkSession; + } + + @Override + @SneakyThrows + public List apply(LogicalPlan x) { + CatalogTable catalogTable = + sparkSession + .sessionState() + .catalog() + .getTableMetadata(((AlterTableAddColumnsCommand) x).table()); + + List tableColumns = Arrays.asList(catalogTable.schema().fields()); + List addedColumns = + JavaConversions.seqAsJavaList(((AlterTableAddColumnsCommand) x).colsToAdd()); + + if (tableColumns.containsAll(addedColumns)) { + return Collections.singletonList( + PlanUtils.getDataset(PathUtils.fromCatalogTable(catalogTable), catalogTable.schema())); + } else { + // apply triggered before applying the change - do not send an event + return Collections.emptyList(); + } + } +} diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/AlterTableRenameCommandVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/AlterTableRenameCommandVisitor.java new file mode 100644 index 0000000000000..fa18021d6940e --- /dev/null +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/AlterTableRenameCommandVisitor.java @@ -0,0 +1,63 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import static io.openlineage.spark.agent.util.PlanUtils.datasourceFacet; +import static io.openlineage.spark.agent.util.PlanUtils.schemaFacet; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.facets.PreviousTableNameFacet; +import io.openlineage.spark.agent.util.DatasetIdentifier; +import io.openlineage.spark.agent.util.PathUtils; +import io.openlineage.spark.agent.util.PlanUtils; +import java.util.Collections; +import java.util.List; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.catalog.SessionCatalog; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.command.AlterTableRenameCommand; + +@Slf4j +public class AlterTableRenameCommandVisitor + extends QueryPlanVisitor { + + private final SparkSession sparkSession; + + public AlterTableRenameCommandVisitor(SparkSession sparkSession) { + this.sparkSession = sparkSession; + } + + @SneakyThrows + @Override + public List apply(LogicalPlan x) { + SessionCatalog sessionCatalog = sparkSession.sessionState().catalog(); + CatalogTable table; + try { + table = sessionCatalog.getTableMetadata(((AlterTableRenameCommand) x).newName()); + } catch (NoSuchTableException e) { + log.info("NoSuchTableException caught"); + // apply method called before altering table - do not send an event + return Collections.emptyList(); + } + + DatasetIdentifier di = PathUtils.fromCatalogTable(table); + + AlterTableRenameCommand alterTableRenameCommand = (AlterTableRenameCommand) x; + String previousPath = + di.getName() + .replace( + alterTableRenameCommand.newName().table(), + alterTableRenameCommand.oldName().table()); + + return Collections.singletonList( + PlanUtils.getDataset( + di, + new OpenLineage.DatasetFacetsBuilder() + .schema(schemaFacet(table.schema())) + .dataSource(datasourceFacet(di.getNamespace())) + .put("previousTableName", new PreviousTableNameFacet(previousPath, di.getName())) + .build())); + } +} diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/AppendDataVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/AppendDataVisitor.java index 56f8bc798f522..ca1a782173aa5 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/AppendDataVisitor.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/AppendDataVisitor.java @@ -20,6 +20,7 @@ public AppendDataVisitor( @Override public List apply(LogicalPlan x) { + // Needs to cast to logical plan despite IntelliJ claiming otherwise. List datasets = PlanUtils.applyFirst(outputVisitors, (LogicalPlan) ((AppendData) x).table()); diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateDataSourceTableAsSelectCommandVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateDataSourceTableAsSelectCommandVisitor.java new file mode 100644 index 0000000000000..c5eccf80c09f2 --- /dev/null +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateDataSourceTableAsSelectCommandVisitor.java @@ -0,0 +1,26 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.util.PathUtils; +import io.openlineage.spark.agent.util.PlanUtils; +import java.util.Collections; +import java.util.List; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand; + +/** + * {@link LogicalPlan} visitor that matches an {@link CreateDataSourceTableAsSelectCommand} and + * extracts the output {@link OpenLineage.Dataset} being written. + */ +public class CreateDataSourceTableAsSelectCommandVisitor + extends QueryPlanVisitor { + + @Override + public List apply(LogicalPlan x) { + CreateDataSourceTableAsSelectCommand command = (CreateDataSourceTableAsSelectCommand) x; + CatalogTable catalogTable = command.table(); + return Collections.singletonList( + PlanUtils.getDataset(PathUtils.fromCatalogTable(catalogTable), catalogTable.schema())); + } +} diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateDataSourceTableCommandVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateDataSourceTableCommandVisitor.java new file mode 100644 index 0000000000000..e9ed87b611d6c --- /dev/null +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateDataSourceTableCommandVisitor.java @@ -0,0 +1,26 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.util.PathUtils; +import io.openlineage.spark.agent.util.PlanUtils; +import java.util.Collections; +import java.util.List; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.command.CreateDataSourceTableCommand; + +/** + * {@link LogicalPlan} visitor that matches an {@link CreateDataSourceTableCommand} and extracts the + * output {@link OpenLineage.Dataset} being written. + */ +public class CreateDataSourceTableCommandVisitor + extends QueryPlanVisitor { + + @Override + public List apply(LogicalPlan x) { + CreateDataSourceTableCommand command = (CreateDataSourceTableCommand) x; + CatalogTable catalogTable = command.table(); + return Collections.singletonList( + PlanUtils.getDataset(PathUtils.fromCatalogTable(catalogTable), catalogTable.schema())); + } +} diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateHiveTableAsSelectCommandVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateHiveTableAsSelectCommandVisitor.java new file mode 100644 index 0000000000000..4187ec1566c77 --- /dev/null +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateHiveTableAsSelectCommandVisitor.java @@ -0,0 +1,39 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.util.DatasetIdentifier; +import io.openlineage.spark.agent.util.PathUtils; +import io.openlineage.spark.agent.util.PlanUtils; +import io.openlineage.spark.agent.util.ScalaConversionUtils; +import java.util.Collections; +import java.util.List; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.expressions.Attribute; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +/** + * {@link LogicalPlan} visitor that matches an {@link CreateHiveTableAsSelectCommand} and extracts + * the output {@link OpenLineage.Dataset} being written. + */ +public class CreateHiveTableAsSelectCommandVisitor + extends QueryPlanVisitor { + + @Override + public List apply(LogicalPlan x) { + CreateHiveTableAsSelectCommand command = (CreateHiveTableAsSelectCommand) x; + CatalogTable table = command.tableDesc(); + DatasetIdentifier di = PathUtils.fromCatalogTable(table); + StructType schema = outputSchema(ScalaConversionUtils.fromSeq(command.outputColumns())); + return Collections.singletonList(PlanUtils.getDataset(di, schema)); + } + + private StructType outputSchema(List attrs) { + return new StructType( + attrs.stream() + .map(a -> new StructField(a.name(), a.dataType(), a.nullable(), a.metadata())) + .toArray(StructField[]::new)); + } +} diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateTableCommandVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateTableCommandVisitor.java new file mode 100644 index 0000000000000..438b53997905e --- /dev/null +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateTableCommandVisitor.java @@ -0,0 +1,27 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.util.PathUtils; +import io.openlineage.spark.agent.util.PlanUtils; +import java.util.Collections; +import java.util.List; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.command.CreateTableCommand; + +/** + * {@link LogicalPlan} visitor that matches an {@link CreateTableCommand} and extracts the output + * {@link OpenLineage.Dataset} being written. + */ +public class CreateTableCommandVisitor + extends QueryPlanVisitor { + + @Override + public List apply(LogicalPlan x) { + CreateTableCommand command = (CreateTableCommand) x; + CatalogTable catalogTable = command.table(); + + return Collections.singletonList( + PlanUtils.getDataset(PathUtils.fromCatalogTable(catalogTable), catalogTable.schema())); + } +} diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/DropTableCommandVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/DropTableCommandVisitor.java new file mode 100644 index 0000000000000..0934e899ed294 --- /dev/null +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/DropTableCommandVisitor.java @@ -0,0 +1,58 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import static io.openlineage.spark.agent.util.PlanUtils.datasourceFacet; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.facets.TableStateChangeFacet; +import io.openlineage.spark.agent.util.DatasetIdentifier; +import io.openlineage.spark.agent.util.PathUtils; +import io.openlineage.spark.agent.util.PlanUtils; +import java.util.Collections; +import java.util.List; +import lombok.SneakyThrows; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.command.DropTableCommand; + +/** + * {@link LogicalPlan} visitor that matches an {@link DropTableCommand} and extracts the output + * {@link OpenLineage.Dataset} being written. + */ +public class DropTableCommandVisitor + extends QueryPlanVisitor { + + private final SparkSession sparkSession; + + public DropTableCommandVisitor(SparkSession sparkSession) { + this.sparkSession = sparkSession; + } + + @Override + @SneakyThrows + public List apply(LogicalPlan x) { + DropTableCommand command = (DropTableCommand) x; + + if (sparkSession.sessionState().catalog().tableExists(command.tableName())) { + // prepare an event, table will be deleted soon + CatalogTable table = + sparkSession.sessionState().catalog().getTableMetadata(command.tableName()); + DatasetIdentifier datasetIdentifier = PathUtils.fromCatalogTable(table); + + return Collections.singletonList( + PlanUtils.getDataset( + datasetIdentifier, + new OpenLineage.DatasetFacetsBuilder() + .schema(null) + .dataSource(datasourceFacet(datasetIdentifier.getNamespace())) + .put( + "tableStateChange", + new TableStateChangeFacet(TableStateChangeFacet.StateChange.DROP)) + .build())); + + } else { + // already deleted, do nothing + return Collections.emptyList(); + } + } +} diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoDataSourceDirVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoDataSourceDirVisitor.java index d2f08dac371ed..fec504b61735d 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoDataSourceDirVisitor.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoDataSourceDirVisitor.java @@ -1,8 +1,9 @@ package io.openlineage.spark.agent.lifecycle.plan; import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.util.DatasetIdentifier; +import io.openlineage.spark.agent.util.PathUtils; import io.openlineage.spark.agent.util.PlanUtils; -import java.net.URI; import java.util.Collections; import java.util.List; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; @@ -19,13 +20,7 @@ public class InsertIntoDataSourceDirVisitor public List apply(LogicalPlan x) { InsertIntoDataSourceDirCommand command = (InsertIntoDataSourceDirCommand) x; // URI is required by the InsertIntoDataSourceDirCommand - URI outputPath = command.storage().locationUri().get(); - if (outputPath.getScheme() == null) { - outputPath = URI.create("file://" + outputPath); - } - String namespace = PlanUtils.namespaceUri(outputPath); - return Collections.singletonList( - PlanUtils.getDataset( - outputPath.getPath(), namespace, PlanUtils.datasetFacet(command.schema(), namespace))); + DatasetIdentifier di = PathUtils.fromURI(command.storage().locationUri().get(), "file"); + return Collections.singletonList(PlanUtils.getDataset(di, command.schema())); } } diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoDirVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoDirVisitor.java index 75a08e0d158d7..9529b6b3217b4 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoDirVisitor.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoDirVisitor.java @@ -1,13 +1,13 @@ package io.openlineage.spark.agent.lifecycle.plan; import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.util.PathUtils; import io.openlineage.spark.agent.util.PlanUtils; import io.openlineage.spark.agent.util.ScalaConversionUtils; +import java.net.URI; import java.util.Collections; import java.util.List; -import org.apache.hadoop.fs.Path; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat; +import java.util.Optional; import org.apache.spark.sql.catalyst.plans.logical.InsertIntoDir; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; @@ -16,26 +16,16 @@ * OpenLineage.Dataset} being written. */ public class InsertIntoDirVisitor extends QueryPlanVisitor { - private final SQLContext sqlContext; - - public InsertIntoDirVisitor(SQLContext sqlContext) { - this.sqlContext = sqlContext; - } @Override public List apply(LogicalPlan x) { InsertIntoDir cmd = (InsertIntoDir) x; - CatalogStorageFormat storage = cmd.storage(); - return ScalaConversionUtils.asJavaOptional(storage.locationUri()) + Optional optionalUri = ScalaConversionUtils.asJavaOptional(cmd.storage().locationUri()); + return optionalUri .map( - uri -> { - Path path = new Path(uri); - if (uri.getScheme() == null) { - path = new Path("file", null, uri.toString()); - } - return Collections.singletonList( - PlanUtils.getDataset(path.toUri(), cmd.child().schema())); - }) + uri -> + Collections.singletonList( + PlanUtils.getDataset(PathUtils.fromURI(uri, "file"), cmd.child().schema()))) .orElse(Collections.emptyList()); } } diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoHadoopFsRelationVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoHadoopFsRelationVisitor.java index 32bdbf50408d5..27d23055f9c7a 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoHadoopFsRelationVisitor.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoHadoopFsRelationVisitor.java @@ -1,8 +1,8 @@ package io.openlineage.spark.agent.lifecycle.plan; import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.util.PathUtils; import io.openlineage.spark.agent.util.PlanUtils; -import java.net.URI; import java.util.Collections; import java.util.List; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; @@ -18,12 +18,8 @@ public class InsertIntoHadoopFsRelationVisitor @Override public List apply(LogicalPlan x) { InsertIntoHadoopFsRelationCommand command = (InsertIntoHadoopFsRelationCommand) x; - URI outputPath = command.outputPath().toUri(); - String namespace = PlanUtils.namespaceUri(outputPath); return Collections.singletonList( PlanUtils.getDataset( - outputPath.getPath(), - namespace, - PlanUtils.datasetFacet(command.query().schema(), namespace))); + PathUtils.fromURI(command.outputPath().toUri(), "file"), command.query().schema())); } } diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoHiveDirVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoHiveDirVisitor.java index 0a79f20ff2ad7..e1bd6ead34afb 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoHiveDirVisitor.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoHiveDirVisitor.java @@ -1,12 +1,13 @@ package io.openlineage.spark.agent.lifecycle.plan; import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.util.PathUtils; import io.openlineage.spark.agent.util.PlanUtils; import io.openlineage.spark.agent.util.ScalaConversionUtils; +import java.net.URI; import java.util.Collections; import java.util.List; -import org.apache.hadoop.fs.Path; -import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat; +import java.util.Optional; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; import org.apache.spark.sql.hive.execution.InsertIntoHiveDirCommand; @@ -20,17 +21,12 @@ public class InsertIntoHiveDirVisitor @Override public List apply(LogicalPlan x) { InsertIntoHiveDirCommand cmd = (InsertIntoHiveDirCommand) x; - CatalogStorageFormat storage = cmd.storage(); - return ScalaConversionUtils.asJavaOptional(storage.locationUri()) + Optional optionalUri = ScalaConversionUtils.asJavaOptional(cmd.storage().locationUri()); + return optionalUri .map( - uri -> { - Path path = new Path(uri); - if (uri.getScheme() == null) { - path = new Path("file", null, uri.toString()); - } - return Collections.singletonList( - PlanUtils.getDataset(path.toUri(), cmd.query().schema())); - }) + uri -> + Collections.singletonList( + PlanUtils.getDataset(PathUtils.fromURI(uri, "file"), cmd.query().schema()))) .orElse(Collections.emptyList()); } } diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoHiveTableVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoHiveTableVisitor.java index ce7631051b976..9045a3d6307b3 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoHiveTableVisitor.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/InsertIntoHiveTableVisitor.java @@ -1,13 +1,10 @@ package io.openlineage.spark.agent.lifecycle.plan; import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.util.PathUtils; import io.openlineage.spark.agent.util.PlanUtils; -import io.openlineage.spark.agent.util.SparkConfUtils; import java.util.Collections; import java.util.List; -import org.apache.hadoop.fs.Path; -import org.apache.spark.SparkContext; -import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.catalyst.catalog.CatalogTable; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; import org.apache.spark.sql.hive.execution.InsertIntoHiveTable; @@ -19,15 +16,6 @@ public class InsertIntoHiveTableVisitor extends QueryPlanVisitor { - private final SparkContext context; - - private final String metastoreUriKey = "spark.sql.hive.metastore.uris"; - private final String metastoreHadoopUriKey = "spark.hadoop.hive.metastore.uris"; - - public InsertIntoHiveTableVisitor(SparkContext context) { - this.context = context; - } - public static boolean hasHiveClasses() { try { InsertIntoHiveTableVisitor.class @@ -44,26 +32,8 @@ public static boolean hasHiveClasses() { public List apply(LogicalPlan x) { InsertIntoHiveTable cmd = (InsertIntoHiveTable) x; CatalogTable table = cmd.table(); - Path path; - try { - path = new Path(table.location()); - if (table.location().getScheme() == null) { - path = new Path("file", null, table.location().toString()); - } - } catch (Exception e) { // Java does not recognize scala exception - if (e instanceof AnalysisException) { - - String authority = - SparkConfUtils.findSparkConfigKey(context.getConf(), metastoreUriKey) - .orElse( - SparkConfUtils.findSparkConfigKey(context.getConf(), metastoreHadoopUriKey) - .get()); - - path = new Path("hive", authority, table.qualifiedName()); - } - throw e; - } - return Collections.singletonList(PlanUtils.getDataset(path.toUri(), cmd.query().schema())); + return Collections.singletonList( + PlanUtils.getDataset(PathUtils.fromCatalogTable(table), table.schema())); } } diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/LoadDataCommandVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/LoadDataCommandVisitor.java new file mode 100644 index 0000000000000..c1473156bb4f9 --- /dev/null +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/LoadDataCommandVisitor.java @@ -0,0 +1,34 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.util.PathUtils; +import io.openlineage.spark.agent.util.PlanUtils; +import java.util.Collections; +import java.util.List; +import lombok.SneakyThrows; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.command.LoadDataCommand; + +/** + * {@link LogicalPlan} visitor that matches an {@link LoadDataCommandVisitor} and extracts the + * output {@link OpenLineage.Dataset} being written. + */ +public class LoadDataCommandVisitor extends QueryPlanVisitor { + + private final SparkSession sparkSession; + + public LoadDataCommandVisitor(SparkSession sparkSession) { + this.sparkSession = sparkSession; + } + + @SneakyThrows + @Override + public List apply(LogicalPlan x) { + LoadDataCommand command = (LoadDataCommand) x; + CatalogTable table = sparkSession.sessionState().catalog().getTableMetadata(command.table()); + return Collections.singletonList( + PlanUtils.getDataset(PathUtils.fromCatalogTable(table), table.schema())); + } +} diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/OptimizedCreateHiveTableAsSelectCommandVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/OptimizedCreateHiveTableAsSelectCommandVisitor.java new file mode 100644 index 0000000000000..3191952fe2956 --- /dev/null +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/OptimizedCreateHiveTableAsSelectCommandVisitor.java @@ -0,0 +1,58 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.util.DatasetIdentifier; +import io.openlineage.spark.agent.util.PathUtils; +import io.openlineage.spark.agent.util.PlanUtils; +import io.openlineage.spark.agent.util.ScalaConversionUtils; +import java.util.Collections; +import java.util.List; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.expressions.Attribute; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.hive.execution.OptimizedCreateHiveTableAsSelectCommand; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +/** + * {@link LogicalPlan} visitor that matches an {@link OptimizedCreateHiveTableAsSelectCommand} and + * extracts the output {@link OpenLineage.Dataset} being written. + */ +public class OptimizedCreateHiveTableAsSelectCommandVisitor + extends QueryPlanVisitor { + + // OptimizedCreateHiveTableAsSelectCommand has been added in Spark 2.4.8 + public static boolean hasClasses() { + try { + OptimizedCreateHiveTableAsSelectCommandVisitor.class + .getClassLoader() + .loadClass("org.apache.spark.sql.hive.execution.OptimizedCreateHiveTableAsSelectCommand"); + return true; + } catch (Exception e) { + // swallow- we don't care + } + return false; + } + + @Override + public boolean isDefinedAt(LogicalPlan plan) { + return (plan instanceof OptimizedCreateHiveTableAsSelectCommand); + } + + @Override + public List apply(LogicalPlan x) { + OptimizedCreateHiveTableAsSelectCommand command = (OptimizedCreateHiveTableAsSelectCommand) x; + CatalogTable table = command.tableDesc(); + DatasetIdentifier datasetIdentifier = PathUtils.fromCatalogTable(table); + StructType schema = outputSchema(ScalaConversionUtils.fromSeq(command.outputColumns())); + + return Collections.singletonList(PlanUtils.getDataset(datasetIdentifier, schema)); + } + + private StructType outputSchema(List attrs) { + return new StructType( + attrs.stream() + .map(a -> new StructField(a.name(), a.dataType(), a.nullable(), a.metadata())) + .toArray(StructField[]::new)); + } +} diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/SaveIntoDataSourceCommandVisitor.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/SaveIntoDataSourceCommandVisitor.java index bdf58899014cd..7c40b7c258512 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/SaveIntoDataSourceCommandVisitor.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/lifecycle/plan/SaveIntoDataSourceCommandVisitor.java @@ -4,10 +4,12 @@ import com.google.common.collect.ImmutableMap.Builder; import io.openlineage.client.OpenLineage; import io.openlineage.spark.agent.util.PlanUtils; +import java.sql.SQLException; import java.util.Collections; import java.util.List; import java.util.Optional; import java.util.stream.Collectors; +import lombok.extern.slf4j.Slf4j; import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; import org.apache.spark.sql.execution.datasources.LogicalRelation; @@ -23,6 +25,7 @@ * BaseRelation}, we wrap it with an artificial {@link LogicalRelation} so we can delegate to other * plan visitors. */ +@Slf4j public class SaveIntoDataSourceCommandVisitor extends QueryPlanVisitor { private final SQLContext sqlContext; @@ -59,12 +62,26 @@ public List apply(LogicalPlan x) { if (KafkaRelationVisitor.isKafkaSource(command.dataSource())) { return KafkaRelationVisitor.createKafkaDatasets( command.dataSource(), command.options(), command.mode(), command.schema()); - } else if (command.dataSource() instanceof RelationProvider) { - RelationProvider p = (RelationProvider) command.dataSource(); - relation = p.createRelation(sqlContext, command.options()); - } else { - SchemaRelationProvider p = (SchemaRelationProvider) command.dataSource(); - relation = p.createRelation(sqlContext, command.options(), x.schema()); + } + try { + if (command.dataSource() instanceof RelationProvider) { + RelationProvider p = (RelationProvider) command.dataSource(); + relation = p.createRelation(sqlContext, command.options()); + } else { + SchemaRelationProvider p = (SchemaRelationProvider) command.dataSource(); + relation = p.createRelation(sqlContext, command.options(), x.schema()); + } + } catch (Exception ex) { + // Bad detection of errors in scala + if (ex instanceof SQLException) { + // This can happen on SparkListenerSQLExecutionStart for example for sqlite, when database + // does not exist yet - it will be created as command execution + // Still, we can just ignore it on start, because it will work on end + // see SparkReadWriteIntegTest.testReadFromFileWriteToJdbc + log.warn("Can't create relation: ", ex); + return Collections.emptyList(); + } + throw ex; } return Optional.ofNullable( PlanUtils.applyFirst( diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/DatasetIdentifier.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/DatasetIdentifier.java new file mode 100644 index 0000000000000..9eaf32973e2cf --- /dev/null +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/DatasetIdentifier.java @@ -0,0 +1,9 @@ +package io.openlineage.spark.agent.util; + +import lombok.Value; + +@Value +public class DatasetIdentifier { + String name; + String namespace; +} diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/PathUtils.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/PathUtils.java new file mode 100644 index 0000000000000..2c497e90c6f5e --- /dev/null +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/PathUtils.java @@ -0,0 +1,137 @@ +package io.openlineage.spark.agent.util; + +import java.net.URI; +import java.util.Optional; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.fs.Path; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.internal.StaticSQLConf; + +@Slf4j +public class PathUtils { + + private static final String DEFAULT_SCHEME = "file"; + + private static Optional sparkConf = Optional.empty(); + + public static DatasetIdentifier fromPath(Path path) { + return PathUtils.fromPath(path, DEFAULT_SCHEME); + } + + public static DatasetIdentifier fromPath(Path path, String defaultScheme) { + if (path.isAbsoluteAndSchemeAuthorityNull()) { + return new DatasetIdentifier(path.toString(), defaultScheme); + } + URI uri = path.toUri(); + String namespace = + Optional.ofNullable(uri.getAuthority()) + .map(a -> String.format("%s://%s", uri.getScheme(), a)) + .orElseGet(() -> (uri.getScheme() != null) ? uri.getScheme() : defaultScheme); + String name = removeFirstSlashIfSingleSlashInString(uri.getPath()); + return new DatasetIdentifier(name, namespace); + } + + public static DatasetIdentifier fromURI(URI location, String defaultScheme) { + return fromPath(new Path(location), defaultScheme); + } + + public static DatasetIdentifier fromCatalogTable(CatalogTable catalogTable) { + return fromCatalogTable(catalogTable, loadSparkConf()); + } + + /** + * Create DatasetIdentifier from CatalogTable, using storage's locationURI if it exists. In other + * way, use defaultTablePath. + */ + @SneakyThrows + public static DatasetIdentifier fromCatalogTable( + CatalogTable catalogTable, Optional sparkConf) { + Optional metastoreUri = extractMetastoreUri(sparkConf); + if (metastoreUri.isPresent() && metastoreUri.get() != null) { + // dealing with Hive tables + return prepareHiveDatasetIdentifier(catalogTable, metastoreUri.get()); + } else { + if (catalogTable.storage() != null && catalogTable.storage().locationUri().isDefined()) { + // location is present -> use it for dataset identifier with `file:/` scheme + return PathUtils.fromURI(catalogTable.storage().locationUri().get(), "file"); + } + + try { + // read it from default table path + return prepareDatasetIdentifierFromDefaultTablePath(catalogTable); + } catch (IllegalStateException e) { + // session inactive - no way to find DatasetProvider + throw new IllegalArgumentException( + "Unable to extract DatasetIdentifier from a CatalogTable", e); + } + } + } + + @SneakyThrows + private static DatasetIdentifier prepareDatasetIdentifierFromDefaultTablePath( + CatalogTable catalogTable) { + String path = + SparkSession.active() + .sessionState() + .catalog() + .defaultTablePath(catalogTable.identifier()) + .getPath(); + + return PathUtils.fromURI(new URI("file", null, path, null, null), "file"); + } + + @SneakyThrows + private static DatasetIdentifier prepareHiveDatasetIdentifier( + CatalogTable catalogTable, URI metastoreUri) { + String qualifiedName = catalogTable.qualifiedName(); + if (!qualifiedName.startsWith("/")) { + qualifiedName = String.format("/%s", qualifiedName); + } + return PathUtils.fromPath( + new Path(enrichHiveMetastoreURIWithTableName(metastoreUri, qualifiedName))); + } + + @SneakyThrows + public static URI enrichHiveMetastoreURIWithTableName(URI metastoreUri, String qualifiedName) { + return new URI( + "hive", null, metastoreUri.getHost(), metastoreUri.getPort(), qualifiedName, null, null); + } + + /** + * SparkConf does not change through job lifetime but it can get lost once session is closed. It's + * good to have it set in case of SPARK-29046 + */ + private static Optional loadSparkConf() { + if (!sparkConf.isPresent() && SparkSession.getActiveSession().isDefined()) { + sparkConf = Optional.of(SparkSession.getActiveSession().get().sparkContext().getConf()); + } + return sparkConf; + } + + private static Optional extractMetastoreUri(Optional sparkConf) { + // make sure SparkConf is present + if (!sparkConf.isPresent()) { + return Optional.empty(); + } + + // make sure enableHiveSupport is called + Optional setting = + SparkConfUtils.findSparkConfigKey( + sparkConf.get(), StaticSQLConf.CATALOG_IMPLEMENTATION().key()); + if (!setting.isPresent() || !setting.get().equals("hive")) { + return Optional.empty(); + } + + return SparkConfUtils.getMetastoreUri(sparkConf.get()); + } + + private static String removeFirstSlashIfSingleSlashInString(String name) { + if (name.chars().filter(x -> x == '/').count() == 1 && name.startsWith("/")) { + return name.substring(1); + } + return name; + } +} diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/PlanUtils.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/PlanUtils.java index b9b89256e85b0..8dc7eba4e66e0 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/PlanUtils.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/PlanUtils.java @@ -97,20 +97,6 @@ public static String namespaceUri(URI outputPath) { .orElse(outputPath.getScheme()); } - /** - * Given a {@link URI}, construct a valid {@link OpenLineage.Dataset} following the expected - * naming conventions. - * - * @param outputPath - * @param schema - * @return - */ - public static OpenLineage.Dataset getDataset(URI outputPath, StructType schema) { - String namespace = namespaceUri(outputPath); - OpenLineage.DatasetFacets datasetFacet = datasetFacet(schema, namespace); - return getDataset(outputPath.getPath(), namespace, datasetFacet); - } - /** * Construct a dataset {@link OpenLineage.Dataset} given a name, namespace, and preconstructed * {@link OpenLineage.DatasetFacets}. @@ -140,6 +126,30 @@ public OpenLineage.DatasetFacets getFacets() { }; } + /** + * Given a {@link URI}, construct a valid {@link OpenLineage.Dataset} following the expected + * naming conventions. + * + * @param outputPath + * @param schema + * @return + */ + public static OpenLineage.Dataset getDataset(URI outputPath, StructType schema) { + String namespace = namespaceUri(outputPath); + OpenLineage.DatasetFacets datasetFacet = datasetFacet(schema, namespace); + return getDataset(outputPath.getPath(), namespace, datasetFacet); + } + + public static OpenLineage.Dataset getDataset(DatasetIdentifier ident, StructType schema) { + OpenLineage.DatasetFacets datasetFacet = datasetFacet(schema, ident.getNamespace()); + return getDataset(ident.getName(), ident.getNamespace(), datasetFacet); + } + + public static OpenLineage.Dataset getDataset( + DatasetIdentifier ident, OpenLineage.DatasetFacets datasetFacet) { + return getDataset(ident.getName(), ident.getNamespace(), datasetFacet); + } + /** * Construct a {@link OpenLineage.DatasetFacets} given a schema and a namespace. * diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/ScalaConversionUtils.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/ScalaConversionUtils.java index a0ff9c7506b76..0727da378c57d 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/ScalaConversionUtils.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/ScalaConversionUtils.java @@ -1,6 +1,7 @@ package io.openlineage.spark.agent.util; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.function.Function; import java.util.function.Supplier; @@ -55,6 +56,17 @@ public static List fromSeq(Seq seq) { return JavaConverters.bufferAsJavaListConverter(seq.toBuffer()).asJava(); } + /** + * Convert a {@link scala.collection.immutable.Map} to a Java {@link java.util.Map}. + * + * @param map + * @param + * @return + */ + public static Map fromMap(scala.collection.immutable.Map map) { + return JavaConverters.mapAsJavaMapConverter(map).asJava(); + } + /** * Convert a Scala {@link Option} to a Java {@link Optional}. * diff --git a/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/SparkConfUtils.java b/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/SparkConfUtils.java index e68c428945f86..c9f51446fb584 100644 --- a/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/SparkConfUtils.java +++ b/integration/spark/src/main/common/java/io/openlineage/spark/agent/util/SparkConfUtils.java @@ -1,19 +1,54 @@ package io.openlineage.spark.agent.util; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.Optional; import org.apache.spark.SparkConf; +import scala.Option; public class SparkConfUtils { + private static final String metastoreUriKey = "spark.sql.hive.metastore.uris"; + private static final String metastoreHadoopUriKey = "spark.hadoop.hive.metastore.uris"; + public static String findSparkConfigKey(SparkConf conf, String name, String defaultValue) { return findSparkConfigKey(conf, name).orElse(defaultValue); } public static Optional findSparkConfigKey(SparkConf conf, String name) { - return ScalaConversionUtils.asJavaOptional( - conf.getOption(name) - .getOrElse(ScalaConversionUtils.toScalaFn(() -> conf.getOption("spark." + name)))); + Option opt = conf.getOption(name); + if (opt.isDefined()) { + return Optional.of(opt.get()); + } + opt = conf.getOption("spark." + name); + if (opt.isDefined()) { + return Optional.of(opt.get()); + } + return Optional.empty(); + } + + public static Optional getMetastoreUri(SparkConf conf) { + return Optional.ofNullable( + SparkConfUtils.findSparkConfigKey(conf, metastoreUriKey) + .orElse( + SparkConfUtils.findSparkConfigKey(conf, metastoreHadoopUriKey).orElse(null))) + .map( + key -> { + if (key.contains(",")) { + return Arrays.stream(key.split(",")).findFirst().get(); + } + return key; + }) + .map( + uri -> { + try { + return new URI(uri); + } catch (URISyntaxException e) { + return null; + } + }); } public static Optional> findSparkUrlParams(SparkConf conf, String prefix) { diff --git a/integration/spark/src/main/spark2/java/io/openlineage/spark/agent/lifecycle/Spark2VisitorFactoryImpl.java b/integration/spark/src/main/spark2/java/io/openlineage/spark/agent/lifecycle/Spark2VisitorFactoryImpl.java index cd38decab9581..3e82dd1c7a0e1 100644 --- a/integration/spark/src/main/spark2/java/io/openlineage/spark/agent/lifecycle/Spark2VisitorFactoryImpl.java +++ b/integration/spark/src/main/spark2/java/io/openlineage/spark/agent/lifecycle/Spark2VisitorFactoryImpl.java @@ -4,6 +4,7 @@ import io.openlineage.client.OpenLineage; import io.openlineage.spark.agent.lifecycle.plan.QueryPlanVisitor; import io.openlineage.spark.agent.lifecycle.plan.wrapper.OutputDatasetVisitor; +import io.openlineage.spark2.agent.lifecycle.plan.CreateTableLikeCommandVisitor; import io.openlineage.spark2.agent.lifecycle.plan.DatasetSourceVisitor; import java.util.List; import org.apache.spark.sql.SQLContext; @@ -17,6 +18,7 @@ public List> getOutputV return ImmutableList.>builder() .addAll(super.getOutputVisitors(sqlContext, jobNamespace)) .add(new OutputDatasetVisitor(new DatasetSourceVisitor())) + .add(new OutputDatasetVisitor(new CreateTableLikeCommandVisitor(sqlContext.sparkSession()))) .build(); } diff --git a/integration/spark/src/main/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateTableLikeCommandVisitor.java b/integration/spark/src/main/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateTableLikeCommandVisitor.java new file mode 100644 index 0000000000000..d18d25af898d8 --- /dev/null +++ b/integration/spark/src/main/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateTableLikeCommandVisitor.java @@ -0,0 +1,50 @@ +package io.openlineage.spark2.agent.lifecycle.plan; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.lifecycle.plan.QueryPlanVisitor; +import io.openlineage.spark.agent.util.DatasetIdentifier; +import io.openlineage.spark.agent.util.PathUtils; +import io.openlineage.spark.agent.util.PlanUtils; +import java.net.URI; +import java.util.Collections; +import java.util.List; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.catalog.SessionCatalog; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.command.CreateTableLikeCommand; + +/** + * {@link LogicalPlan} visitor that matches an {@link CreateTableLikeCommand} and extracts the + * output {@link OpenLineage.Dataset} being written. + */ +@Slf4j +public class CreateTableLikeCommandVisitor + extends QueryPlanVisitor { + + private final SparkSession sparkSession; + + public CreateTableLikeCommandVisitor(SparkSession sparkSession) { + this.sparkSession = sparkSession; + } + + @SneakyThrows + @Override + public List apply(LogicalPlan x) { + CreateTableLikeCommand command = (CreateTableLikeCommand) x; + SessionCatalog catalog = sparkSession.sessionState().catalog(); + + CatalogTable source = catalog.getTempViewOrPermanentTableMetadata(command.sourceTable()); + URI location; + if (command.location().isEmpty()) { + location = catalog.defaultTablePath(command.targetTable()); + } else { + location = new URI(command.location().get()); + } + + DatasetIdentifier di = PathUtils.fromURI(location, "file"); + return Collections.singletonList(PlanUtils.getDataset(di, source.schema())); + } +} diff --git a/integration/spark/src/main/spark3/java/io/openlineage/spark/agent/lifecycle/Spark3VisitorFactoryImpl.java b/integration/spark/src/main/spark3/java/io/openlineage/spark/agent/lifecycle/Spark3VisitorFactoryImpl.java index 25558eb226d3c..f1b1738f3fd7a 100644 --- a/integration/spark/src/main/spark3/java/io/openlineage/spark/agent/lifecycle/Spark3VisitorFactoryImpl.java +++ b/integration/spark/src/main/spark3/java/io/openlineage/spark/agent/lifecycle/Spark3VisitorFactoryImpl.java @@ -4,6 +4,8 @@ import io.openlineage.client.OpenLineage; import io.openlineage.spark.agent.lifecycle.plan.QueryPlanVisitor; import io.openlineage.spark.agent.lifecycle.plan.wrapper.OutputDatasetVisitor; +import io.openlineage.spark3.agent.lifecycle.plan.CreateTableAsSelectVisitor; +import io.openlineage.spark3.agent.lifecycle.plan.CreateTableLikeCommandVisitor; import io.openlineage.spark3.agent.lifecycle.plan.DataSourceV2RelationVisitor; import java.util.List; import org.apache.spark.sql.SQLContext; @@ -16,7 +18,10 @@ public List> getOutputV SQLContext sqlContext, String jobNamespace) { return ImmutableList.>builder() .addAll(super.getOutputVisitors(sqlContext, jobNamespace)) + .add(new OutputDatasetVisitor(new CreateTableAsSelectVisitor(sqlContext.sparkSession()))) .add(new OutputDatasetVisitor(new DataSourceV2RelationVisitor())) + .add(new OutputDatasetVisitor(new CreateTableAsSelectVisitor(sqlContext.sparkSession()))) + .add(new OutputDatasetVisitor(new CreateTableLikeCommandVisitor(sqlContext.sparkSession()))) .build(); } @@ -24,7 +29,6 @@ public List> getCom SQLContext sqlContext, String jobNamespace) { return ImmutableList.>builder() .addAll(super.getBaseCommonVisitors(sqlContext, jobNamespace)) - .add(new DataSourceV2RelationVisitor()) .build(); } } diff --git a/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/CreateTableAsSelectVisitor.java b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/CreateTableAsSelectVisitor.java new file mode 100644 index 0000000000000..6ee7d4bc0c343 --- /dev/null +++ b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/CreateTableAsSelectVisitor.java @@ -0,0 +1,54 @@ +package io.openlineage.spark3.agent.lifecycle.plan; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.lifecycle.plan.QueryPlanVisitor; +import io.openlineage.spark.agent.util.DatasetIdentifier; +import io.openlineage.spark.agent.util.PlanUtils; +import io.openlineage.spark.agent.util.ScalaConversionUtils; +import io.openlineage.spark3.agent.lifecycle.plan.catalog.CatalogUtils3; +import io.openlineage.spark3.agent.lifecycle.plan.catalog.UnsupportedCatalogException; +import java.util.Collections; +import java.util.List; +import lombok.extern.slf4j.Slf4j; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.plans.logical.CreateTableAsSelect; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; + +/** + * {@link LogicalPlan} visitor that matches an {@link CreateTableAsSelect} and extracts the output + * {@link OpenLineage.Dataset} being written. + */ +@Slf4j +public class CreateTableAsSelectVisitor + extends QueryPlanVisitor { + + private final SparkSession sparkSession; + + public CreateTableAsSelectVisitor(SparkSession sparkSession) { + this.sparkSession = sparkSession; + } + + @Override + public List apply(LogicalPlan x) { + CreateTableAsSelect command = (CreateTableAsSelect) x; + + DatasetIdentifier datasetIdentifier; + try { + datasetIdentifier = + CatalogUtils3.getDatasetIdentifier( + sparkSession, + command.catalog(), + command.tableName(), + ScalaConversionUtils.fromMap(command.properties())); + } catch (UnsupportedCatalogException ex) { + log.error(String.format("Catalog %s is unsupported", ex.getMessage()), ex); + return Collections.emptyList(); + } + + return Collections.singletonList( + PlanUtils.getDataset( + datasetIdentifier.getName(), + datasetIdentifier.getNamespace(), + PlanUtils.datasetFacet(command.tableSchema(), datasetIdentifier.getNamespace()))); + } +} diff --git a/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/CreateTableLikeCommandVisitor.java b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/CreateTableLikeCommandVisitor.java new file mode 100644 index 0000000000000..9c3bc8d29beff --- /dev/null +++ b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/CreateTableLikeCommandVisitor.java @@ -0,0 +1,49 @@ +package io.openlineage.spark3.agent.lifecycle.plan; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.lifecycle.plan.QueryPlanVisitor; +import io.openlineage.spark.agent.util.DatasetIdentifier; +import io.openlineage.spark.agent.util.PathUtils; +import io.openlineage.spark.agent.util.PlanUtils; +import io.openlineage.spark.agent.util.ScalaConversionUtils; +import java.net.URI; +import java.util.Collections; +import java.util.List; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.catalog.SessionCatalog; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.command.CreateTableLikeCommand; + +/** + * {@link LogicalPlan} visitor that matches an {@link CreateTableLikeCommand} and extracts the + * output {@link OpenLineage.Dataset} being written. + */ +@Slf4j +public class CreateTableLikeCommandVisitor + extends QueryPlanVisitor { + + private final SparkSession sparkSession; + + public CreateTableLikeCommandVisitor(SparkSession sparkSession) { + this.sparkSession = sparkSession; + } + + @SneakyThrows + @Override + public List apply(LogicalPlan x) { + CreateTableLikeCommand command = (CreateTableLikeCommand) x; + SessionCatalog catalog = sparkSession.sessionState().catalog(); + + CatalogTable source = catalog.getTempViewOrPermanentTableMetadata(command.sourceTable()); + URI defaultLocation = catalog.defaultTablePath(command.targetTable()); + + URI location = + ScalaConversionUtils.asJavaOptional(command.fileFormat().locationUri()) + .orElse(defaultLocation); + DatasetIdentifier di = PathUtils.fromURI(location, "file"); + return Collections.singletonList(PlanUtils.getDataset(di, source.schema())); + } +} diff --git a/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/DataSourceV2RelationVisitor.java b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/DataSourceV2RelationVisitor.java index 3c44395e5f0f8..5d9aef7800f67 100644 --- a/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/DataSourceV2RelationVisitor.java +++ b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/DataSourceV2RelationVisitor.java @@ -12,6 +12,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import lombok.extern.slf4j.Slf4j; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; import org.apache.spark.sql.connector.catalog.Table; import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation; @@ -24,31 +25,50 @@ * org.apache.spark.sql.catalyst.analysis.NamedRelation}, the returned name is that of the source, * not the specific dataset (e.g., "bigquery" not the table). */ +@Slf4j public class DataSourceV2RelationVisitor extends QueryPlanVisitor { - private static final String ICEBERG = "iceberg"; + private enum Provider { + ICEBERG, + DELTA, + UNKNOWN; + } @Override public boolean isDefinedAt(LogicalPlan logicalPlan) { - return findDatasetProvider(logicalPlan).equals(ICEBERG); + return logicalPlan instanceof DataSourceV2Relation + && !findDatasetProvider(logicalPlan).equals(Provider.UNKNOWN); } - private String findDatasetProvider(LogicalPlan plan) { + private Provider findDatasetProvider(LogicalPlan plan) { return Optional.of(plan) .filter(x -> x instanceof DataSourceV2Relation) .map(x -> (DataSourceV2Relation) x) .map(DataSourceV2Relation::table) .map(Table::properties) .map(properties -> properties.get("provider")) - .map(String::toLowerCase) - .orElse("unknown"); + .map(String::toUpperCase) + .map( + provider -> { + try { + return Provider.valueOf(provider); + } catch (IllegalArgumentException e) { + return Provider.UNKNOWN; + } + }) + .orElse(Provider.UNKNOWN); } private OpenLineage.Dataset findDatasetForIceberg(DataSourceV2Relation relation) { Map properties = relation.table().properties(); - String namespace = properties.getOrDefault("location", "unknown"); + String namespace = properties.getOrDefault("location", null); + String format = properties.getOrDefault("format", null); + // Should not happen - we're inside proper iceberg table. + if (namespace == null || format == null) { + return null; + } namespace = namespace.startsWith("/") ? "file://" + namespace : namespace; return PlanUtils.getDataset( relation.table().name(), @@ -59,16 +79,52 @@ private OpenLineage.Dataset findDatasetForIceberg(DataSourceV2Relation relation) .put( "table_provider", new TableProviderFacet( - ICEBERG, properties.getOrDefault("format", "unknown").replace("iceberg/", ""))) + Provider.ICEBERG.name().toLowerCase(), format.replace("iceberg/", ""))) + .build()); + } + + private OpenLineage.Dataset findDatasetForDelta(DataSourceV2Relation relation) { + Map properties = relation.table().properties(); + + String namespace = properties.getOrDefault("location", null); + String format = properties.getOrDefault("format", null); + // Should not happen - we're inside proper delta table. + if (namespace == null || format == null) { + return null; + } + + return PlanUtils.getDataset( + relation.table().name(), + namespace, + new OpenLineage.DatasetFacetsBuilder() + .schema(schemaFacet(relation.schema())) + .dataSource(datasourceFacet(namespace)) + .put( + "table_provider", + new TableProviderFacet( + Provider.DELTA.name().toLowerCase(), "parquet")) // Delta is always parquet .build()); } @Override public List apply(LogicalPlan logicalPlan) { - if (findDatasetProvider(logicalPlan).equals(ICEBERG)) { - return Collections.singletonList(findDatasetForIceberg((DataSourceV2Relation) logicalPlan)); + Provider provider = findDatasetProvider(logicalPlan); + DataSourceV2Relation x = (DataSourceV2Relation) logicalPlan; + + switch (provider) { + case ICEBERG: + return nullableSingletonList(findDatasetForIceberg(x)); + case DELTA: + return nullableSingletonList(findDatasetForDelta(x)); + default: + throw new RuntimeException("Couldn't find provider for dataset in plan " + logicalPlan); } + } - throw new RuntimeException("Couldn't find DatasetSource in plan " + logicalPlan); + private List nullableSingletonList(T singleton) { + if (singleton == null) { + return Collections.emptyList(); + } + return Collections.singletonList(singleton); } } diff --git a/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/CatalogHandler.java b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/CatalogHandler.java new file mode 100644 index 0000000000000..c8ef1ef0ad27f --- /dev/null +++ b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/CatalogHandler.java @@ -0,0 +1,19 @@ +package io.openlineage.spark3.agent.lifecycle.plan.catalog; + +import io.openlineage.spark.agent.util.DatasetIdentifier; +import java.util.Map; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; + +public interface CatalogHandler { + boolean hasClasses(); + + boolean isClass(TableCatalog tableCatalog); + + DatasetIdentifier getDatasetIdentifier( + SparkSession session, + TableCatalog tableCatalog, + Identifier identifier, + Map properties); +} diff --git a/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/CatalogUtils3.java b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/CatalogUtils3.java new file mode 100644 index 0000000000000..c14a8adb27f0e --- /dev/null +++ b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/CatalogUtils3.java @@ -0,0 +1,38 @@ +package io.openlineage.spark3.agent.lifecycle.plan.catalog; + +import io.openlineage.spark.agent.util.DatasetIdentifier; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; + +public class CatalogUtils3 { + + private static List parsers = getHandlers(); + + private static List getHandlers() { + List handlers = + Arrays.asList( + new IcebergHandler(), + new DeltaHandler(), + new JdbcHandler(), + new V2SessionCatalogHandler()); + return handlers.stream().filter(CatalogHandler::hasClasses).collect(Collectors.toList()); + } + + public static DatasetIdentifier getDatasetIdentifier( + SparkSession session, + TableCatalog catalog, + Identifier identifier, + Map properties) { + for (CatalogHandler parser : parsers) { + if (parser.isClass(catalog)) { + return parser.getDatasetIdentifier(session, catalog, identifier, properties); + } + } + throw new UnsupportedCatalogException(catalog.getClass().getCanonicalName()); + } +} diff --git a/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/DeltaHandler.java b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/DeltaHandler.java new file mode 100644 index 0000000000000..442b9b436b6b1 --- /dev/null +++ b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/DeltaHandler.java @@ -0,0 +1,68 @@ +package io.openlineage.spark3.agent.lifecycle.plan.catalog; + +import io.openlineage.spark.agent.util.DatasetIdentifier; +import io.openlineage.spark.agent.util.PathUtils; +import java.util.Arrays; +import java.util.Map; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.fs.Path; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.delta.catalog.DeltaCatalog; +import scala.Option; + +@Slf4j +public class DeltaHandler implements CatalogHandler { + public boolean hasClasses() { + try { + DeltaHandler.class + .getClassLoader() + .loadClass("org.apache.spark.sql.delta.catalog.DeltaCatalog"); + return true; + } catch (Exception e) { + // swallow- we don't care + } + return false; + } + + @Override + public boolean isClass(TableCatalog tableCatalog) { + return tableCatalog instanceof DeltaCatalog; + } + + @Override + public DatasetIdentifier getDatasetIdentifier( + SparkSession session, + TableCatalog tableCatalog, + Identifier identifier, + Map properties) { + DeltaCatalog catalog = (DeltaCatalog) tableCatalog; + + Optional location; + if (catalog.isPathIdentifier(identifier)) { + location = Optional.of(identifier.name()); + } else { + location = Optional.ofNullable(properties.get("location")); + } + // Delta uses spark2 catalog when location isn't specified. + Path path = + new Path( + location.orElse( + session + .sessionState() + .catalog() + .defaultTablePath( + TableIdentifier.apply( + identifier.name(), + Option.apply( + Arrays.stream(identifier.namespace()) + .reduce((x, y) -> y) + .orElse(null)))) + .toString())); + log.warn(path.toString()); + return PathUtils.fromPath(path, "file"); + } +} diff --git a/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/IcebergHandler.java b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/IcebergHandler.java new file mode 100644 index 0000000000000..9e70b54ed691e --- /dev/null +++ b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/IcebergHandler.java @@ -0,0 +1,92 @@ +package io.openlineage.spark3.agent.lifecycle.plan.catalog; + +import io.openlineage.spark.agent.util.DatasetIdentifier; +import io.openlineage.spark.agent.util.PathUtils; +import io.openlineage.spark.agent.util.ScalaConversionUtils; +import io.openlineage.spark.agent.util.SparkConfUtils; +import java.net.URI; +import java.util.Map; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; + +@Slf4j +public class IcebergHandler implements CatalogHandler { + @Override + public boolean hasClasses() { + try { + IcebergHandler.class.getClassLoader().loadClass("org.apache.iceberg.catalog.Catalog"); + return true; + } catch (Exception e) { + // swallow- we don't care + } + return false; + } + + @Override + public boolean isClass(TableCatalog tableCatalog) { + return tableCatalog instanceof SparkCatalog; + } + + @Override + public DatasetIdentifier getDatasetIdentifier( + SparkSession session, + TableCatalog tableCatalog, + Identifier identifier, + Map properties) { + String catalogName = ((SparkCatalog) tableCatalog).name(); + String prefix = String.format("spark.sql.catalog.%s", catalogName); + Map conf = + ScalaConversionUtils.fromMap(session.conf().getAll()); + log.warn(conf.toString()); + Map catalogConf = + conf.entrySet().stream() + .filter(x -> x.getKey().startsWith(prefix)) + .collect( + Collectors.toMap( + x -> x.getKey().substring(prefix.length() + 1), // handle dot after prefix + Map.Entry::getValue)); + + log.warn(catalogConf.toString()); + if (catalogConf.isEmpty() || !catalogConf.containsKey("type")) { + throw new UnsupportedCatalogException(catalogName); + } + log.warn(catalogConf.get("type")); + switch (catalogConf.get("type")) { + case "hadoop": + return getHadoopIdentifier(catalogConf, identifier.toString()); + case "hive": + return getHiveIdentifier( + session, catalogConf.get(CatalogProperties.URI), identifier.toString()); + default: + throw new UnsupportedCatalogException(catalogConf.get("type")); + } + } + + private DatasetIdentifier getHadoopIdentifier(Map conf, String table) { + String warehouse = conf.get(CatalogProperties.WAREHOUSE_LOCATION); + return PathUtils.fromPath(new Path(warehouse, table)); + } + + @SneakyThrows + private DatasetIdentifier getHiveIdentifier( + SparkSession session, @Nullable String confUri, String table) { + table = String.format("/%s", table); + URI uri; + if (confUri == null) { + uri = + SparkConfUtils.getMetastoreUri(session.sparkContext().conf()) + .orElseThrow(() -> new UnsupportedCatalogException("hive")); + } else { + uri = new URI(confUri); + } + return PathUtils.fromPath(new Path(PathUtils.enrichHiveMetastoreURIWithTableName(uri, table))); + } +} diff --git a/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/JdbcHandler.java b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/JdbcHandler.java new file mode 100644 index 0000000000000..bffeb44588fee --- /dev/null +++ b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/JdbcHandler.java @@ -0,0 +1,44 @@ +package io.openlineage.spark3.agent.lifecycle.plan.catalog; + +import io.openlineage.spark.agent.util.DatasetIdentifier; +import io.openlineage.spark.agent.util.PlanUtils; +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import lombok.SneakyThrows; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions; +import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog; + +public class JdbcHandler implements CatalogHandler { + @Override + public boolean hasClasses() { + return true; + } + + @Override + public boolean isClass(TableCatalog tableCatalog) { + return tableCatalog instanceof JDBCTableCatalog; + } + + @SneakyThrows + @Override + public DatasetIdentifier getDatasetIdentifier( + SparkSession session, + TableCatalog tableCatalog, + Identifier identifier, + Map properties) { + JDBCTableCatalog catalog = (JDBCTableCatalog) tableCatalog; + JDBCOptions options = (JDBCOptions) FieldUtils.readField(catalog, "options", true); + + String name = + Stream.concat(Arrays.stream(identifier.namespace()), Stream.of(identifier.name())) + .collect(Collectors.joining(".")); + + return new DatasetIdentifier(name, PlanUtils.sanitizeJdbcUrl(options.url())); + } +} diff --git a/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/UnsupportedCatalogException.java b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/UnsupportedCatalogException.java new file mode 100644 index 0000000000000..724a050322743 --- /dev/null +++ b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/UnsupportedCatalogException.java @@ -0,0 +1,7 @@ +package io.openlineage.spark3.agent.lifecycle.plan.catalog; + +public class UnsupportedCatalogException extends RuntimeException { + public UnsupportedCatalogException(String catalog) { + super(catalog); + } +} diff --git a/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/V2SessionCatalogHandler.java b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/V2SessionCatalogHandler.java new file mode 100644 index 0000000000000..8ed4a33d25193 --- /dev/null +++ b/integration/spark/src/main/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/catalog/V2SessionCatalogHandler.java @@ -0,0 +1,31 @@ +package io.openlineage.spark3.agent.lifecycle.plan.catalog; + +import io.openlineage.spark.agent.util.DatasetIdentifier; +import java.util.Map; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog; + +public class V2SessionCatalogHandler implements CatalogHandler { + + @Override + public boolean hasClasses() { + return true; + } + + @Override + public boolean isClass(TableCatalog tableCatalog) { + return tableCatalog instanceof V2SessionCatalog; + } + + @Override + public DatasetIdentifier getDatasetIdentifier( + SparkSession session, + TableCatalog tableCatalog, + Identifier identifier, + Map properties) { + V2SessionCatalog catalog = (V2SessionCatalog) tableCatalog; + throw new UnsupportedCatalogException(V2SessionCatalog.class.getCanonicalName()); + } +} diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/OpenLineageRunEventTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/OpenLineageRunEventTest.java index 3cb57f361f0d0..34df11cb5ee37 100644 --- a/integration/spark/src/test/common/java/io/openlineage/spark/agent/OpenLineageRunEventTest.java +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/OpenLineageRunEventTest.java @@ -9,6 +9,7 @@ import io.openlineage.spark.agent.lifecycle.MatchesMapRecursively; import java.io.IOException; import java.net.URI; +import java.net.URISyntaxException; import java.nio.file.Path; import java.nio.file.Paths; import java.time.ZonedDateTime; @@ -24,11 +25,14 @@ public class OpenLineageRunEventTest { new TypeReference>() {}; @Test - public void testSerializeRunEvent() throws IOException { + public void testSerializeRunEvent() throws IOException, URISyntaxException { ObjectMapper mapper = OpenLineageClient.createMapper(); ZonedDateTime dateTime = ZonedDateTime.parse("2021-01-01T00:00:01.000000000+02:00[UTC]"); - OpenLineage ol = new OpenLineage(OpenLineageClient.OPEN_LINEAGE_CLIENT_URI); + OpenLineage ol = + new OpenLineage( + new URI( + "https://github.com/OpenLineage/OpenLineage/tree/0.2.3-SNAPSHOT/integration/spark")); UUID runId = UUID.fromString("5f24c93c-2ce9-49dc-82e7-95ab4915242f"); OpenLineage.RunFacets runFacets = diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/OpenLineageSparkListenerTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/OpenLineageSparkListenerTest.java new file mode 100644 index 0000000000000..dcb91c9d94ca0 --- /dev/null +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/OpenLineageSparkListenerTest.java @@ -0,0 +1,99 @@ +package io.openlineage.spark.agent; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.lifecycle.SparkSQLExecutionContext; +import io.openlineage.spark.agent.lifecycle.plan.InsertIntoHadoopFsRelationVisitor; +import io.openlineage.spark.agent.lifecycle.plan.wrapper.OutputDatasetVisitor; +import java.util.Collections; +import java.util.Properties; +import org.apache.hadoop.fs.Path; +import org.apache.spark.SparkContext; +import org.apache.spark.scheduler.SparkListenerJobStart; +import org.apache.spark.scheduler.StageInfo; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.QueryExecution; +import org.apache.spark.sql.execution.SparkPlan; +import org.apache.spark.sql.execution.SparkPlanInfo; +import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand; +import org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart; +import org.apache.spark.sql.types.IntegerType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import scala.Option; +import scala.collection.Map$; +import scala.collection.Seq$; + +@ExtendWith(SparkAgentTestExtension.class) +public class OpenLineageSparkListenerTest { + @Test + public void testSqlEventWithJobEventEmitsOnce(SparkSession sparkSession) { + OpenLineageContext context = mock(OpenLineageContext.class); + QueryExecution qe = mock(QueryExecution.class); + LogicalPlan query = mock(LogicalPlan.class); + SparkPlan plan = mock(SparkPlan.class); + + when(query.schema()) + .thenReturn( + new StructType( + new StructField[] { + new StructField("key", IntegerType$.MODULE$, false, Metadata.empty()) + })); + + when(qe.optimizedPlan()) + .thenReturn( + new InsertIntoHadoopFsRelationCommand( + new Path("file:///tmp/dir"), + null, + false, + Seq$.MODULE$.empty(), + Option.empty(), + null, + Map$.MODULE$.empty(), + query, + SaveMode.Overwrite, + Option.empty(), + Option.empty(), + Seq$.MODULE$.empty())); + + when(qe.executedPlan()).thenReturn(plan); + when(plan.sparkContext()).thenReturn(SparkContext.getOrCreate()); + when(plan.nodeName()).thenReturn("execute"); + + SparkSQLExecutionContext executionContext = + new SparkSQLExecutionContext( + 1L, + context, + qe, + Collections.singletonList( + new OutputDatasetVisitor(new InsertIntoHadoopFsRelationVisitor())), + Collections.emptyList()); + + executionContext.start( + new SparkListenerSQLExecutionStart( + 1L, + "", + "", + "", + new SparkPlanInfo( + "name", "string", Seq$.MODULE$.empty(), Map$.MODULE$.empty(), Seq$.MODULE$.empty()), + 1L)); + executionContext.start( + new SparkListenerJobStart(0, 2L, Seq$.MODULE$.empty(), new Properties())); + + ArgumentCaptor lineageEvent = + ArgumentCaptor.forClass(OpenLineage.RunEvent.class); + + verify(context, times(1)).emit(lineageEvent.capture()); + } +} diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/SparkContainerIntegrationTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/SparkContainerIntegrationTest.java index 3119f5493885b..ea2d5e3f9114e 100644 --- a/integration/spark/src/test/common/java/io/openlineage/spark/agent/SparkContainerIntegrationTest.java +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/SparkContainerIntegrationTest.java @@ -19,6 +19,7 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.EnabledIfSystemProperty; import org.mockserver.client.MockServerClient; import org.mockserver.matchers.MatchType; import org.slf4j.Logger; @@ -100,7 +101,8 @@ private static GenericContainer makePysparkContainer(String... command) { .withFileSystemBind("build/libs", "/opt/libs") .withFileSystemBind("build/dependencies", "/opt/dependencies") .withLogConsumer(SparkContainerIntegrationTest::consumeOutput) - .withStartupTimeout(Duration.of(2, ChronoUnit.MINUTES)) + .waitingFor(Wait.forLogMessage(".*ShutdownHookManager: Shutdown hook called.*", 1)) + .withStartupTimeout(Duration.of(5, ChronoUnit.MINUTES)) .dependsOn(openLineageClientMockContainer) .withReuse(true) .withCommand(command); @@ -165,7 +167,6 @@ public void testPysparkWordCountWithCliArgs() throws IOException, InterruptedExc pyspark = makePysparkContainerWithDefaultConf( "testPysparkWordCountWithCliArgs", "/opt/spark_scripts/spark_word_count.py"); - pyspark.setWaitStrategy(Wait.forLogMessage(".*ShutdownHookManager: Shutdown hook called.*", 1)); pyspark.start(); Path eventFolder = Paths.get("integrations/container/"); @@ -188,7 +189,6 @@ public void testPysparkRddToTable() throws IOException, InterruptedException { pyspark = makePysparkContainerWithDefaultConf( "testPysparkRddToTable", "/opt/spark_scripts/spark_rdd_to_table.py"); - pyspark.setWaitStrategy(Wait.forLogMessage(".*ShutdownHookManager: Shutdown hook called.*", 1)); pyspark.start(); Path eventFolder = Paths.get("integrations/container/"); @@ -228,8 +228,6 @@ public void testPysparkKafkaReadWrite() throws IOException { "--packages", System.getProperty("kafka.package.version"), "/opt/spark_scripts/spark_kafka.py"); - - pyspark.setWaitStrategy(Wait.forLogMessage(".*ShutdownHookManager: Shutdown hook called.*", 1)); pyspark.start(); Path eventFolder = Paths.get("integrations/container/"); @@ -296,7 +294,6 @@ public void testPysparkSQLHiveTest() throws IOException, InterruptedException { pyspark = makePysparkContainerWithDefaultConf( "testPysparkSQLHiveTest", "/opt/spark_scripts/spark_hive.py"); - pyspark.setWaitStrategy(Wait.forLogMessage(".*ShutdownHookManager: Shutdown hook called.*", 1)); pyspark.start(); Path eventFolder = Paths.get("integrations/container/"); @@ -318,7 +315,6 @@ public void testPysparkSQLOverwriteDirHiveTest() throws IOException, Interrupted pyspark = makePysparkContainerWithDefaultConf( "testPysparkSQLHiveOverwriteDirTest", "/opt/spark_scripts/spark_overwrite_hive.py"); - pyspark.setWaitStrategy(Wait.forLogMessage(".*ShutdownHookManager: Shutdown hook called.*", 1)); pyspark.start(); Path eventFolder = Paths.get("integrations/container/"); @@ -335,4 +331,150 @@ public void testPysparkSQLOverwriteDirHiveTest() throws IOException, Interrupted .withPath("/api/v1/lineage") .withBody(json(completeEvent, MatchType.ONLY_MATCHING_FIELDS))); } + + @Test + public void testCreateAsSelectAndLoad() throws IOException, InterruptedException { + pyspark = + makePysparkContainerWithDefaultConf( + "testCreateAsSelectAndLoad", "/opt/spark_scripts/spark_ctas_load.py"); + pyspark.start(); + + Path eventFolder = Paths.get("integrations/container/"); + + String startCTASEvent = new String(readAllBytes(eventFolder.resolve("pysparkCTASStart.json"))); + String completeCTASEvent = new String(readAllBytes(eventFolder.resolve("pysparkCTASEnd.json"))); + + String startLoadEvent = new String(readAllBytes(eventFolder.resolve("pysparkLoadStart.json"))); + String completeLoadEvent = + new String(readAllBytes(eventFolder.resolve("pysparkLoadComplete.json"))); + + mockServerClient.verify( + request() + .withPath("/api/v1/lineage") + .withBody(json(startCTASEvent, MatchType.ONLY_MATCHING_FIELDS)), + request() + .withPath("/api/v1/lineage") + .withBody(json(completeCTASEvent, MatchType.ONLY_MATCHING_FIELDS)), + request() + .withPath("/api/v1/lineage") + .withBody(json(startLoadEvent, MatchType.ONLY_MATCHING_FIELDS)), + request() + .withPath("/api/v1/lineage") + .withBody(json(completeLoadEvent, MatchType.ONLY_MATCHING_FIELDS))); + } + + @Test + @EnabledIfSystemProperty( + named = "spark.version", + matches = "(3.*)|(2\\.4\\.([8,9]|\\d\\d))") // Spark version >= 2.4.8 + public void testCTASDelta() throws IOException, InterruptedException { + pyspark = + makePysparkContainerWithDefaultConf( + "testCTASDelta", + "--packages", + "io.delta:delta-core_2.12:1.0.0", + "/opt/spark_scripts/spark_delta.py"); + pyspark.start(); + + Path eventFolder = Paths.get("integrations/container/"); + + String completeCTASEvent = + new String(readAllBytes(eventFolder.resolve("pysparkDeltaCTASComplete.json"))); + + mockServerClient.verify( + request() + .withPath("/api/v1/lineage") + .withBody(json(completeCTASEvent, MatchType.ONLY_MATCHING_FIELDS))); + } + + @Test + public void testCreateTable() throws IOException { + pyspark = + makePysparkContainerWithDefaultConf( + "testCreateTable", "/opt/spark_scripts/spark_create_table.py"); + pyspark.setWaitStrategy(Wait.forLogMessage(".*ShutdownHookManager: Shutdown hook called.*", 1)); + pyspark.start(); + + Path eventFolder = Paths.get("integrations/container/"); + + String startCreateTableEvent = + new String(readAllBytes(eventFolder.resolve("pysparkCreateTableStartEvent.json"))); + String completeCreateTableEvent = + new String(readAllBytes(eventFolder.resolve("pysparkCreateTableCompleteEvent.json"))); + + mockServerClient.verify( + request() + .withPath("/api/v1/lineage") + .withBody(json(startCreateTableEvent, MatchType.ONLY_MATCHING_FIELDS)), + request() + .withPath("/api/v1/lineage") + .withBody(json(completeCreateTableEvent, MatchType.ONLY_MATCHING_FIELDS))); + } + + @Test + public void testDropTable() throws IOException { + pyspark = + makePysparkContainerWithDefaultConf( + "testDropTable", "/opt/spark_scripts/spark_drop_table.py"); + pyspark.setWaitStrategy(Wait.forLogMessage(".*ShutdownHookManager: Shutdown hook called.*", 1)); + pyspark.start(); + + Path eventFolder = Paths.get("integrations/container/"); + + String startDropTableEvent = + new String(readAllBytes(eventFolder.resolve("pysparkDropTableStartEvent.json"))); + + mockServerClient.verify( + request() + .withPath("/api/v1/lineage") + .withBody(json(startDropTableEvent, MatchType.ONLY_MATCHING_FIELDS))); + } + + @Test + @EnabledIfSystemProperty( + named = "spark.version", + matches = "(3.*)|(2\\.4\\.([8,9]|\\d\\d))") // Spark version >= 2.4.8 + public void testOptimizedCreateAsSelectAndLoad() throws IOException, InterruptedException { + pyspark = + makePysparkContainerWithDefaultConf( + "testOptimizedCreateAsSelectAndLoad", "/opt/spark_scripts/spark_octas_load.py"); + pyspark.start(); + + Path eventFolder = Paths.get("integrations/container/"); + String startOCTASEvent = + new String(readAllBytes(eventFolder.resolve("pysparkOCTASStart.json"))); + String completeOCTASEvent = + new String(readAllBytes(eventFolder.resolve("pysparkOCTASEnd.json"))); + + mockServerClient.verify( + request() + .withPath("/api/v1/lineage") + .withBody(json(startOCTASEvent, MatchType.ONLY_MATCHING_FIELDS)), + request() + .withPath("/api/v1/lineage") + .withBody(json(completeOCTASEvent, MatchType.ONLY_MATCHING_FIELDS))); + } + + @Test + public void testAlterTable() throws IOException { + pyspark = + makePysparkContainerWithDefaultConf( + "testAlterTable", "/opt/spark_scripts/spark_alter_table.py"); + pyspark.start(); + + Path eventFolder = Paths.get("integrations/container/"); + + String completeAddColumnsEvent = + new String(readAllBytes(eventFolder.resolve("pysparkAlterTableAddColumnsEnd.json"))); + String completeRenameEvent = + new String(readAllBytes(eventFolder.resolve("pysparkAlterTableRenameEnd.json"))); + + mockServerClient.verify( + request() + .withPath("/api/v1/lineage") + .withBody(json(completeAddColumnsEvent, MatchType.ONLY_MATCHING_FIELDS)), + request() + .withPath("/api/v1/lineage") + .withBody(json(completeRenameEvent, MatchType.ONLY_MATCHING_FIELDS))); + } } diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/CatalogTableTestUtils.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/CatalogTableTestUtils.java new file mode 100644 index 0000000000000..633217d8e2f42 --- /dev/null +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/CatalogTableTestUtils.java @@ -0,0 +1,67 @@ +package io.openlineage.spark.agent.lifecycle; + +import java.lang.reflect.Method; +import java.net.URI; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import lombok.SneakyThrows; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.catalog.CatalogTableType; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import scala.Option; +import scala.collection.Seq$; +import scala.collection.immutable.HashMap; + +public class CatalogTableTestUtils { + + @SneakyThrows + public static CatalogTable getCatalogTable(TableIdentifier tableIdentifier) { + Method applyMethod = + Arrays.stream(CatalogTable.class.getDeclaredMethods()) + .filter(m -> m.getName().equals("apply")) + .findFirst() + .get(); + List params = new ArrayList<>(); + params.add(tableIdentifier); + params.add(CatalogTableType.MANAGED()); + params.add( + CatalogStorageFormat.apply( + Option.apply(new URI("/some-location")), + Option.empty(), + Option.empty(), + Option.empty(), + false, + null)); + params.add( + new StructType( + new StructField[] { + new StructField("name", StringType$.MODULE$, false, Metadata.empty()) + })); + params.add(Option.empty()); + params.add(Seq$.MODULE$.newBuilder().$plus$eq("name").result()); + params.add(Option.empty()); + params.add(""); + params.add(Instant.now().getEpochSecond()); + params.add(Instant.now().getEpochSecond()); + params.add("v1"); + params.add(new HashMap<>()); + params.add(Option.empty()); + params.add(Option.empty()); + params.add(Option.empty()); + params.add(Seq$.MODULE$.empty()); + params.add(false); + params.add(false); + params.add(new HashMap<>()); + if (applyMethod.getParameterCount() > 19) { + params.add(Option.empty()); + } + return (CatalogTable) applyMethod.invoke(null, params.toArray()); + } +} diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/LibraryTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/LibraryTest.java index 094977d93d768..3743a5ca9ed27 100644 --- a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/LibraryTest.java +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/LibraryTest.java @@ -23,15 +23,11 @@ import java.util.Map; import java.util.Optional; import java.util.UUID; -import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.FilterFunction; -import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SparkSession; -import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.ArgumentCaptor; @@ -44,52 +40,53 @@ public class LibraryTest { private final TypeReference> mapTypeReference = new TypeReference>() {}; - @RepeatedTest(30) - public void testSparkSql(SparkSession spark) throws IOException, TimeoutException { - when(SparkAgentTestExtension.OPEN_LINEAGE_SPARK_CONTEXT.getJobNamespace()) - .thenReturn("ns_name"); - when(SparkAgentTestExtension.OPEN_LINEAGE_SPARK_CONTEXT.getParentJobName()) - .thenReturn("job_name"); - when(SparkAgentTestExtension.OPEN_LINEAGE_SPARK_CONTEXT.getParentRunId()) - .thenReturn(Optional.of(UUID.fromString("ea445b5c-22eb-457a-8007-01c7c52b6e54"))); - - URL url = Resources.getResource("test_data/data.txt"); - final Dataset data = spark.read().textFile(url.getPath()); - - final long numAs = data.filter((FilterFunction) s -> s.contains("a")).count(); - final long numBs = data.filter((FilterFunction) s -> s.contains("b")).count(); - - System.out.println("Lines with a: " + numAs + ", lines with b: " + numBs); - spark.sparkContext().listenerBus().waitUntilEmpty(1000); - spark.stop(); - - ArgumentCaptor lineageEvent = - ArgumentCaptor.forClass(OpenLineage.RunEvent.class); - Mockito.verify(SparkAgentTestExtension.OPEN_LINEAGE_SPARK_CONTEXT, times(4)) - .emit(lineageEvent.capture()); - List events = lineageEvent.getAllValues(); - - assertEquals(4, events.size()); - - ObjectMapper objectMapper = OpenLineageClient.getObjectMapper(); - - for (int i = 0; i < events.size(); i++) { - OpenLineage.RunEvent event = events.get(i); - Map snapshot = - objectMapper.readValue( - Paths.get(String.format("integrations/%s/%d.json", "sparksql", i + 1)).toFile(), - mapTypeReference); - Map actual = - objectMapper.readValue(objectMapper.writeValueAsString(event), mapTypeReference); - assertThat(actual) - .satisfies( - new MatchesMapRecursively( - snapshot, - new HashSet<>( - Arrays.asList("runId", "nonInheritableMetadataKeys", "validConstraints")))); - } - verifySerialization(events); - } + // @RepeatedTest(30) + // public void testSparkSql(SparkSession spark) throws IOException, TimeoutException { + // when(SparkAgentTestExtension.OPEN_LINEAGE_SPARK_CONTEXT.getJobNamespace()) + // .thenReturn("ns_name"); + // when(SparkAgentTestExtension.OPEN_LINEAGE_SPARK_CONTEXT.getParentJobName()) + // .thenReturn("job_name"); + // when(SparkAgentTestExtension.OPEN_LINEAGE_SPARK_CONTEXT.getParentRunId()) + // .thenReturn(Optional.of(UUID.fromString("ea445b5c-22eb-457a-8007-01c7c52b6e54"))); + // + // URL url = Resources.getResource("test_data/data.txt"); + // final Dataset data = spark.read().textFile(url.getPath()); + // + // final long numAs = data.filter((FilterFunction) s -> s.contains("a")).count(); + // final long numBs = data.filter((FilterFunction) s -> s.contains("b")).count(); + // + // System.out.println("Lines with a: " + numAs + ", lines with b: " + numBs); + // spark.sparkContext().listenerBus().waitUntilEmpty(1000); + // spark.stop(); + // + // ArgumentCaptor lineageEvent = + // ArgumentCaptor.forClass(OpenLineage.RunEvent.class); + // Mockito.verify(SparkAgentTestExtension.OPEN_LINEAGE_SPARK_CONTEXT, times(4)) + // .emit(lineageEvent.capture()); + // List events = lineageEvent.getAllValues(); + // + // assertEquals(4, events.size()); + // + // ObjectMapper objectMapper = OpenLineageClient.getObjectMapper(); + // + // for (int i = 0; i < events.size(); i++) { + // OpenLineage.RunEvent event = events.get(i); + // Map snapshot = + // objectMapper.readValue( + // Paths.get(String.format("integrations/%s/%d.json", "sparksql", i + 1)).toFile(), + // mapTypeReference); + // Map actual = + // objectMapper.readValue(objectMapper.writeValueAsString(event), mapTypeReference); + // assertThat(actual) + // .satisfies( + // new MatchesMapRecursively( + // snapshot, + // new HashSet<>( + // Arrays.asList("runId", "nonInheritableMetadataKeys", + // "validConstraints")))); + // } + // verifySerialization(events); + // } @Test public void testRdd(SparkSession spark) throws IOException { diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/LogicalPlanSerializerTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/LogicalPlanSerializerTest.java index e7256dc4039c8..50787d486150c 100644 --- a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/LogicalPlanSerializerTest.java +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/LogicalPlanSerializerTest.java @@ -16,13 +16,9 @@ import io.openlineage.spark.agent.client.OpenLineageClient; import java.io.IOException; import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; import java.nio.file.Path; import java.nio.file.Paths; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; +import java.util.Collections; import java.util.Map; import java.util.Optional; import javax.annotation.Nullable; @@ -32,9 +28,6 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.TableIdentifier; -import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat; -import org.apache.spark.sql.catalyst.catalog.CatalogTable; -import org.apache.spark.sql.catalyst.catalog.CatalogTableType; import org.apache.spark.sql.catalyst.expressions.Attribute; import org.apache.spark.sql.catalyst.expressions.AttributeReference; import org.apache.spark.sql.catalyst.expressions.ExprId; @@ -73,6 +66,12 @@ public void testSerializeLogicalPlan() throws IOException { SparkSession session = SparkSession.builder().master("local").getOrCreate(); String jdbcUrl = "jdbc:postgresql://postgreshost:5432/sparkdata"; String sparkTableName = "my_spark_table"; + scala.collection.immutable.Map map = + (scala.collection.immutable.Map) + Map$.MODULE$ + .newBuilder() + .$plus$eq(Tuple2.apply("driver", Driver.class.getName())) + .result(); JDBCRelation relation = new JDBCRelation( new StructType( @@ -80,13 +79,7 @@ public void testSerializeLogicalPlan() throws IOException { new StructField("name", StringType$.MODULE$, false, Metadata.empty()) }), new Partition[] {}, - new JDBCOptions( - jdbcUrl, - sparkTableName, - Map$.MODULE$ - .newBuilder() - .$plus$eq(Tuple2.apply("driver", Driver.class.getName())) - .result()), + new JDBCOptions(jdbcUrl, sparkTableName, map), session); LogicalRelation logicalRelation = new LogicalRelation( @@ -137,7 +130,11 @@ public void testSerializeInsertIntoHadoopPlan() HadoopFsRelation hadoopFsRelation = new HadoopFsRelation( - new CatalogFileIndex(session, getCatalogTable(), 100L), + new CatalogFileIndex( + session, + CatalogTableTestUtils.getCatalogTable( + new TableIdentifier("test", Option.apply("db"))), + 100L), new StructType( new StructField[] { new StructField("name", StringType$.MODULE$, false, Metadata.empty()) @@ -206,44 +203,11 @@ public void testSerializeInsertIntoHadoopPlan() Map expectedHadoopFSNode = objectMapper.readValue(expectedHadoopFSNodePath.toFile(), mapTypeReference); - assertThat(commandActualNode).satisfies(new MatchesMapRecursively(expectedCommandNode)); - assertThat(hadoopFSActualNode).satisfies(new MatchesMapRecursively(expectedHadoopFSNode)); - } - - private CatalogTable getCatalogTable() throws InvocationTargetException, IllegalAccessException { - Method applyMethod = - Arrays.stream(CatalogTable.class.getDeclaredMethods()) - .filter(m -> m.getName().equals("apply")) - .findFirst() - .get(); - List params = new ArrayList<>(); - params.add(new TableIdentifier("test", Option.apply("db"))); - params.add(CatalogTableType.MANAGED()); - params.add(CatalogStorageFormat.empty()); - params.add( - new StructType( - new StructField[] { - new StructField("name", StringType$.MODULE$, false, Metadata.empty()) - })); - params.add(Option.empty()); - params.add(Seq$.MODULE$.newBuilder().$plus$eq("name").result()); - params.add(Option.empty()); - params.add(""); - params.add(Instant.now().getEpochSecond()); - params.add(Instant.now().getEpochSecond()); - params.add("v1"); - params.add(new HashMap<>()); - params.add(Option.empty()); - params.add(Option.empty()); - params.add(Option.empty()); - params.add(Seq$.MODULE$.empty()); - params.add(false); - params.add(false); - params.add(new HashMap<>()); - if (applyMethod.getParameterCount() > 19) { - params.add(Option.empty()); - } - return (CatalogTable) applyMethod.invoke(null, params.toArray()); + assertThat(commandActualNode) + .satisfies(new MatchesMapRecursively(expectedCommandNode, Collections.singleton("exprId"))); + assertThat(hadoopFSActualNode) + .satisfies( + new MatchesMapRecursively(expectedHadoopFSNode, Collections.singleton("exprId"))); } @Test @@ -297,7 +261,7 @@ public void testSerializeBigQueryPlan() throws IOException { Map commandActualNode = objectMapper.readValue(logicalPlanSerializer.serialize(command), mapTypeReference); - Map hadoopFSActualNode = + Map bigqueryActualNode = objectMapper.readValue(logicalPlanSerializer.serialize(logicalRelation), mapTypeReference); Path expectedCommandNodePath = @@ -310,9 +274,12 @@ public void testSerializeBigQueryPlan() throws IOException { Map expectedBigQueryRelationNode = objectMapper.readValue(expectedBigQueryRelationNodePath.toFile(), mapTypeReference); - assertThat(commandActualNode).satisfies(new MatchesMapRecursively(expectedCommandNode)); - assertThat(hadoopFSActualNode) - .satisfies(new MatchesMapRecursively(expectedBigQueryRelationNode)); + assertThat(commandActualNode) + .satisfies(new MatchesMapRecursively(expectedCommandNode, Collections.singleton("exprId"))); + assertThat(bigqueryActualNode) + .satisfies( + new MatchesMapRecursively( + expectedBigQueryRelationNode, Collections.singleton("exprId"))); } @SuppressWarnings("rawtypes") diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/MatchesMapRecursively.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/MatchesMapRecursively.java index f8ed8cacac795..fa1743b44ce7f 100644 --- a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/MatchesMapRecursively.java +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/MatchesMapRecursively.java @@ -2,6 +2,7 @@ import io.openlineage.spark.agent.client.OpenLineageClient; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Predicate; @@ -9,10 +10,10 @@ import org.assertj.core.api.Condition; /** - * Custom Condition writen for recursive comparison of Map with ability to ignore specified Map keys - * AssertJ Built-in recursive comparison is not working with Map type, it can ignore only object - * properties Example usage: assertThat(actualMap).satisfies(new MatchesMapRecursively(expectedMap, - * new HashSet<>(Arrays.asList("runId")))); + * Custom Condition writen for recursive comparison of Map and List with ability to ignore specified + * Map keys AssertJ Built-in recursive comparison is not working with Map type, it can ignore only + * object properties Example usage: assertThat(actualMap).satisfies(new + * MatchesMapRecursively(expectedMap, new HashSet<>(Arrays.asList("runId")))); * * @see AbstractObjectAssert#usingRecursiveComparison() */ @@ -32,24 +33,58 @@ public MatchesMapRecursively(Map target) { target); } + public static Predicate> predicate(List target, Set omittedKeys) { + return (list) -> { + if (target.size() != list.size()) { + return false; + } + for (int i = 0; i < target.size(); i++) { + boolean eq; + if (target.get(i) instanceof Map) { + eq = + MatchesMapRecursively.predicate((Map) target.get(i), omittedKeys) + .test((Map) target.get(i)); + } else if (target.get(i) instanceof List) { + eq = + MatchesMapRecursively.predicate((List) target.get(i), omittedKeys) + .test((List) list.get(i)); + } else if (list.get(i) == null) { + eq = true; + } else { + eq = target.get(i).equals(list.get(i)); + } + if (!eq) { + return false; + } + } + return true; + }; + } + public static Predicate> predicate( - Map target, Set ommittedKeys) { + Map target, Set omittedKeys) { return (map) -> { if (!map.keySet().containsAll(target.keySet())) { return false; } for (String k : target.keySet()) { - if (!ommittedKeys.contains(k)) { + if (omittedKeys.contains(k)) { continue; } Object val = map.get(k); boolean eq; if (val instanceof Map) { eq = - MatchesMapRecursively.predicate((Map) target.get(k), ommittedKeys) + MatchesMapRecursively.predicate((Map) target.get(k), omittedKeys) .test((Map) val); + } else if (val instanceof List) { + eq = + MatchesMapRecursively.predicate((List) target.get(k), omittedKeys) + .test((List) val); } else if (k.equals("_producer") || k.equals("producer")) { eq = OpenLineageClient.OPEN_LINEAGE_CLIENT_URI.toString().equals(val); + } else if (val == null) { + eq = true; } else { eq = val.equals(target.get(k)); } diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/SparkReadWriteIntegTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/SparkReadWriteIntegTest.java index d0b461b59b304..a7926a2115765 100644 --- a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/SparkReadWriteIntegTest.java +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/SparkReadWriteIntegTest.java @@ -63,6 +63,7 @@ import org.codehaus.jackson.map.ObjectMapper; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; @@ -74,6 +75,7 @@ import scala.collection.immutable.HashMap; @ExtendWith(SparkAgentTestExtension.class) +@Tag("integration-test") public class SparkReadWriteIntegTest { private final KafkaContainer kafkaContainer = @@ -271,6 +273,7 @@ public void testInsertIntoDataSourceDirVisitor(@TempDir Path tempDir, SparkSessi throws IOException, InterruptedException, TimeoutException { Path testFile = writeTestDataToFile(tempDir); Path parquetDir = tempDir.resolve("parquet").toAbsolutePath(); + // Two events from CreateViewCommand spark.read().json("file://" + testFile.toAbsolutePath()).createOrReplaceTempView("testdata"); spark.sql( "INSERT OVERWRITE DIRECTORY '" @@ -284,7 +287,7 @@ public void testInsertIntoDataSourceDirVisitor(@TempDir Path tempDir, SparkSessi ArgumentCaptor lineageEvent = ArgumentCaptor.forClass(OpenLineage.RunEvent.class); - Mockito.verify(SparkAgentTestExtension.OPEN_LINEAGE_SPARK_CONTEXT, times(4)) + Mockito.verify(SparkAgentTestExtension.OPEN_LINEAGE_SPARK_CONTEXT, times(6)) .emit(lineageEvent.capture()); List events = lineageEvent.getAllValues(); Optional completionEvent = diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/StaticExecutionContextFactory.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/StaticExecutionContextFactory.java index 203f873cd9ee7..f3ead65eb283a 100644 --- a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/StaticExecutionContextFactory.java +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/StaticExecutionContextFactory.java @@ -90,7 +90,7 @@ public SparkSQLExecutionContext createSparkSQLExecutionContext(long executionId) SparkSQLExecutionContext sparksql = new SparkSQLExecutionContext( - executionId, sparkContext, outputDatasets, inputDatasets) { + executionId, sparkContext, qe, outputDatasets, inputDatasets) { @Override public ZonedDateTime toZonedTime(long time) { return getZonedTime(); @@ -121,7 +121,11 @@ public void end(SparkListenerSQLExecutionEnd endEvent) { .orElseGet( () -> new SparkSQLExecutionContext( - executionId, sparkContext, Collections.emptyList(), Collections.emptyList())); + executionId, + sparkContext, + null, + Collections.emptyList(), + Collections.emptyList())); } private static List>> diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/AlterTableAddColumnsCommandVisitorTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/AlterTableAddColumnsCommandVisitorTest.java new file mode 100644 index 0000000000000..0acd7911c9b44 --- /dev/null +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/AlterTableAddColumnsCommandVisitorTest.java @@ -0,0 +1,108 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.SparkAgentTestExtension; +import java.util.Arrays; +import java.util.List; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.execution.command.AlterTableAddColumnsCommand; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +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 scala.Option; +import scala.collection.JavaConversions; +import scala.collection.Map$; +import scala.collection.immutable.HashMap; + +@ExtendWith(SparkAgentTestExtension.class) +public class AlterTableAddColumnsCommandVisitorTest { + + SparkSession session; + AlterTableAddColumnsCommandVisitor visitor; + String database; + + @AfterEach + public void afterEach() { + dropTables(); + } + + private void dropTables() { + session + .sessionState() + .catalog() + .dropTable(new TableIdentifier("table1", Option.apply(database)), true, true); + } + + @BeforeEach + public void setup() { + session = + SparkSession.builder() + .config("spark.sql.warehouse.dir", "/tmp/warehouse") + .master("local") + .getOrCreate(); + + dropTables(); + + StructType schema = + new StructType( + new StructField[] { + new StructField("col1", StringType$.MODULE$, false, new Metadata(new HashMap<>())) + }); + + session.catalog().createTable("table1", "csv", schema, Map$.MODULE$.empty()); + database = session.catalog().currentDatabase(); + visitor = new AlterTableAddColumnsCommandVisitor(session); + } + + @Test + public void testAlterTableAddColumns() { + AlterTableAddColumnsCommand command = + new AlterTableAddColumnsCommand( + new TableIdentifier("table1", Option.apply(database)), + JavaConversions.asScalaIterator( + Arrays.asList( + new StructField( + "col2", StringType$.MODULE$, false, new Metadata(new HashMap<>())), + new StructField( + "col3", StringType$.MODULE$, false, new Metadata(new HashMap<>()))) + .iterator()) + .toSeq()); + + command.run(session); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertEquals(3, datasets.get(0).getFacets().getSchema().getFields().size()); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", "/tmp/warehouse/table1") + .hasFieldOrPropertyWithValue("namespace", "file"); + } + + @Test + public void testAlterUpdateColumnsBeforeCommandRun() { + AlterTableAddColumnsCommand command = + new AlterTableAddColumnsCommand( + new TableIdentifier("table1", Option.apply(database)), + JavaConversions.asScalaIterator( + Arrays.asList( + new StructField( + "col2", StringType$.MODULE$, false, new Metadata(new HashMap<>()))) + .iterator()) + .toSeq()); + + // command is not run + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets).isEmpty(); + } +} diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/AlterTableRenameCommandVisitorTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/AlterTableRenameCommandVisitorTest.java new file mode 100644 index 0000000000000..211ea0d79d528 --- /dev/null +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/AlterTableRenameCommandVisitorTest.java @@ -0,0 +1,105 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import static org.assertj.core.api.Assertions.assertThat; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.SparkAgentTestExtension; +import io.openlineage.spark.agent.facets.PreviousTableNameFacet; +import java.util.List; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.execution.command.AlterTableRenameCommand; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +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 scala.Option; +import scala.collection.Map$; +import scala.collection.immutable.HashMap; + +@ExtendWith(SparkAgentTestExtension.class) +public class AlterTableRenameCommandVisitorTest { + + SparkSession session; + AlterTableRenameCommandVisitor visitor; + String database; + + @AfterEach + public void afterEach() { + dropTables(); + } + + private void dropTables() { + session + .sessionState() + .catalog() + .dropTable(new TableIdentifier("old_table", Option.apply(database)), true, true); + session + .sessionState() + .catalog() + .dropTable(new TableIdentifier("new_table", Option.apply(database)), true, true); + } + + @BeforeEach + public void setup() { + session = + SparkSession.builder() + .config("spark.sql.warehouse.dir", "/tmp/warehouse") + .master("local") + .getOrCreate(); + + database = session.catalog().currentDatabase(); + dropTables(); + + StructType schema = + new StructType( + new StructField[] { + new StructField("a", StringType$.MODULE$, false, new Metadata(new HashMap<>())) + }); + + session.catalog().createTable("old_table", "csv", schema, Map$.MODULE$.empty()); + visitor = new AlterTableRenameCommandVisitor(session); + } + + @Test + void testAlterRenameCommandCommand() { + AlterTableRenameCommand command = + new AlterTableRenameCommand( + new TableIdentifier("old_table", Option.apply(database)), + new TableIdentifier("new_table", Option.apply(database)), + false); + command.run(session); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", "/tmp/warehouse/new_table") + .hasFieldOrPropertyWithValue("namespace", "file"); + + PreviousTableNameFacet previousTableNameFacet = + ((PreviousTableNameFacet) + datasets.get(0).getFacets().getAdditionalProperties().get("previousTableName")); + + assertThat(previousTableNameFacet.getPreviousTableUri()).isEqualTo("/tmp/warehouse/old_table"); + assertThat(previousTableNameFacet.getCurrentTableUri()).isEqualTo("/tmp/warehouse/new_table"); + } + + @Test + void testAlterRenameCommandCommandVisitorBeforeCommandRun() { + AlterTableRenameCommand command = + new AlterTableRenameCommand( + new TableIdentifier("old_table", Option.apply(database)), + new TableIdentifier("new_table", Option.apply(database)), + false); + + // command is not run + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets).isEmpty(); + } +} diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateTableCommandVisitorTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateTableCommandVisitorTest.java new file mode 100644 index 0000000000000..c418eea5660b8 --- /dev/null +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/CreateTableCommandVisitorTest.java @@ -0,0 +1,42 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.SparkAgentTestExtension; +import io.openlineage.spark.agent.lifecycle.CatalogTableTestUtils; +import java.util.List; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.execution.command.CreateTableCommand; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import scala.Option; + +@ExtendWith(SparkAgentTestExtension.class) +public class CreateTableCommandVisitorTest { + + CreateTableCommandVisitor visitor; + String database = "default"; + CreateTableCommand command; + TableIdentifier table = new TableIdentifier("create_table", Option.apply(database)); + + @BeforeEach + public void setup() { + command = new CreateTableCommand(CatalogTableTestUtils.getCatalogTable(table), true); + visitor = new CreateTableCommandVisitor(); + } + + @Test + void testCreateTableCommand() { + List datasets = visitor.apply(command); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + assertEquals(1, datasets.get(0).getFacets().getSchema().getFields().size()); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", "/some-location") + .hasFieldOrPropertyWithValue("namespace", "file"); + } +} diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/DropTableCommandVisitorTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/DropTableCommandVisitorTest.java new file mode 100644 index 0000000000000..60221cfda8017 --- /dev/null +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/DropTableCommandVisitorTest.java @@ -0,0 +1,88 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.SparkAgentTestExtension; +import io.openlineage.spark.agent.facets.TableStateChangeFacet; +import java.util.List; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.execution.command.DropTableCommand; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +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 scala.collection.Map$; +import scala.collection.immutable.HashMap; + +@ExtendWith(SparkAgentTestExtension.class) +public class DropTableCommandVisitorTest { + + SparkSession session; + DropTableCommandVisitor visitor; + DropTableCommand command; + String database; + TableIdentifier table = new TableIdentifier("drop_table"); + + @BeforeEach + public void setup() { + session = + SparkSession.builder() + .config("spark.sql.warehouse.dir", "/tmp/warehouse") + .master("local") + .getOrCreate(); + + database = session.catalog().currentDatabase(); + command = new DropTableCommand(table, true, false, true); + visitor = new DropTableCommandVisitor(session); + } + + @AfterEach + public void afterEach() { + session.sessionState().catalog().dropTable(table, true, true); + } + + @Test + public void testDropTableCommandWhenTableDoesNotExist() { + // make sure table does not exist + session.sessionState().catalog().dropTable(table, true, true); + command.run(session); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets).isEmpty(); + } + + @Test + public void testDropCommand() { + // create some other table first + StructType schema = + new StructType( + new StructField[] { + new StructField("field1", StringType$.MODULE$, false, new Metadata(new HashMap<>())) + }); + session.catalog().createTable("drop_table", "csv", schema, Map$.MODULE$.empty()); + + // apply the visitor before running the command + List datasets = visitor.apply(command); + + assertEquals(null, datasets.get(0).getFacets().getSchema()); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", "/tmp/warehouse/drop_table") + .hasFieldOrPropertyWithValue("namespace", "file"); + + TableStateChangeFacet tableStateChangeFacet = + ((TableStateChangeFacet) + datasets.get(0).getFacets().getAdditionalProperties().get("tableStateChange")); + + assertThat( + tableStateChangeFacet.getStateChange().equals(TableStateChangeFacet.StateChange.DROP)); + } +} diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/LoadDataCommandVisitorTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/LoadDataCommandVisitorTest.java new file mode 100644 index 0000000000000..dfee6ce605eb2 --- /dev/null +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/LoadDataCommandVisitorTest.java @@ -0,0 +1,56 @@ +package io.openlineage.spark.agent.lifecycle.plan; + +import static org.assertj.core.api.Assertions.assertThat; + +import io.openlineage.client.OpenLineage; +import java.util.List; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier$; +import org.apache.spark.sql.execution.command.LoadDataCommand; +import org.apache.spark.sql.types.IntegerType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.Test; +import scala.Option; +import scala.collection.Map$; +import scala.collection.immutable.HashMap; + +class LoadDataCommandVisitorTest { + @Test + void testLoadDataCommand() { + SparkSession session = + SparkSession.builder() + .config("spark.sql.warehouse.dir", "/tmp/warehouse") + .master("local") + .getOrCreate(); + String database = session.catalog().currentDatabase(); + + StructType schema = + new StructType( + new StructField[] { + new StructField("key", IntegerType$.MODULE$, false, new Metadata(new HashMap<>())), + new StructField("value", StringType$.MODULE$, false, new Metadata(new HashMap<>())) + }); + + session.catalog().createTable("table", "csv", schema, Map$.MODULE$.empty()); + + LoadDataCommandVisitor visitor = new LoadDataCommandVisitor(session); + + LoadDataCommand command = + new LoadDataCommand( + TableIdentifier$.MODULE$.apply("table", Option.apply(database)), + "/path/to/data", + true, + false, + Option.empty()); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", "/tmp/warehouse/table") + .hasFieldOrPropertyWithValue("namespace", "file"); + } +} diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/LogicalRDDVisitorTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/LogicalRDDVisitorTest.java index d45ffa7451655..13928e9e3c032 100644 --- a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/LogicalRDDVisitorTest.java +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/LogicalRDDVisitorTest.java @@ -31,8 +31,8 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; +import scala.collection.Seq$; import scala.collection.immutable.HashMap; -import scala.collection.immutable.Seq$; @ExtendWith(SparkAgentTestExtension.class) class LogicalRDDVisitorTest { @@ -55,7 +55,7 @@ public void testApply(@TempDir Path tmpDir) { new StructField("aString", StringType$.MODULE$, false, new Metadata(new HashMap<>())) }); jobConf = new JobConf(); - FileInputFormat.addInputPath(jobConf, new org.apache.hadoop.fs.Path("file:///path/to/data/")); + FileInputFormat.addInputPath(jobConf, new org.apache.hadoop.fs.Path("file://" + tmpDir)); RDD hadoopRdd = new HadoopRDD<>( session.sparkContext(), @@ -82,7 +82,7 @@ public void testApply(@TempDir Path tmpDir) { List datasets = visitor.apply(logicalRDD); assertThat(datasets) .singleElement() - .hasFieldOrPropertyWithValue("name", "/path/to/data") + .hasFieldOrPropertyWithValue("name", tmpDir.toString()) .hasFieldOrPropertyWithValue("namespace", "file"); } } diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/LogicalRelationVisitorTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/LogicalRelationVisitorTest.java index 49ad8e9855fe5..9e80bfdf3784a 100644 --- a/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/LogicalRelationVisitorTest.java +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/lifecycle/plan/LogicalRelationVisitorTest.java @@ -57,7 +57,7 @@ void testApply(String connectionUri) { jdbcUrl, sparkTableName, Map$.MODULE$ - .newBuilder() + .newBuilder() .$plus$eq(Tuple2.apply("driver", Driver.class.getName())) .result()), session); diff --git a/integration/spark/src/test/common/java/io/openlineage/spark/agent/util/PathUtilsTest.java b/integration/spark/src/test/common/java/io/openlineage/spark/agent/util/PathUtilsTest.java new file mode 100644 index 0000000000000..5200d40d1202c --- /dev/null +++ b/integration/spark/src/test/common/java/io/openlineage/spark/agent/util/PathUtilsTest.java @@ -0,0 +1,176 @@ +package io.openlineage.spark.agent.util; + +import static io.openlineage.spark.agent.util.PathUtils.enrichHiveMetastoreURIWithTableName; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.fs.Path; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.catalog.SessionCatalog; +import org.apache.spark.sql.internal.SessionState; +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; +import scala.Option; + +@Slf4j +public class PathUtilsTest { + + @Test + void testPathSeparation() { + Path path = new Path("scheme:/asdf/fdsa"); + assertThat(path.toUri().getScheme()).isEqualTo("scheme"); + assertThat(path.toUri().getAuthority()).isEqualTo(null); + assertThat(path.toUri().getPath()).isEqualTo("/asdf/fdsa"); + + path = new Path("scheme://asdf/fdsa"); + assertThat(path.toUri().getScheme()).isEqualTo("scheme"); + assertThat(path.toUri().getAuthority()).isEqualTo("asdf"); + assertThat(path.toUri().getPath()).isEqualTo("/fdsa"); + + path = new Path("scheme:///asdf/fdsa"); + assertThat(path.toUri().getScheme()).isEqualTo("scheme"); + assertThat(path.toUri().getAuthority()).isEqualTo(null); + assertThat(path.toUri().getPath()).isEqualTo("/asdf/fdsa"); + + path = new Path("scheme:////asdf/fdsa"); + assertThat(path.toUri().getScheme()).isEqualTo("scheme"); + assertThat(path.toUri().getAuthority()).isEqualTo(null); + assertThat(path.toUri().getPath()).isEqualTo("/asdf/fdsa"); + } + + @Test + void testFromPathWithoutSchema() { + DatasetIdentifier di = PathUtils.fromPath(new Path("/home/test")); + assertThat(di.getName()).isEqualTo("/home/test"); + assertThat(di.getNamespace()).isEqualTo("file"); + + di = PathUtils.fromPath(new Path("/home/test"), "hive"); + assertThat(di.getName()).isEqualTo("/home/test"); + assertThat(di.getNamespace()).isEqualTo("hive"); + + di = PathUtils.fromPath(new Path("home/test")); + assertThat(di.getName()).isEqualTo("home/test"); + assertThat(di.getNamespace()).isEqualTo("file"); + } + + @Test + void testFromPathWithSchema() { + DatasetIdentifier di = PathUtils.fromPath(new Path("file:/home/test")); + assertThat(di.getName()).isEqualTo("/home/test"); + assertThat(di.getNamespace()).isEqualTo("file"); + + di = PathUtils.fromPath(new Path("hdfs://namenode:8020/home/test")); + assertThat(di.getName()).isEqualTo("/home/test"); + assertThat(di.getNamespace()).isEqualTo("hdfs://namenode:8020"); + } + + @Test + void testFromURI() throws URISyntaxException { + DatasetIdentifier di = PathUtils.fromURI(new URI("file:///home/test"), null); + assertThat(di.getName()).isEqualTo("/home/test"); + assertThat(di.getNamespace()).isEqualTo("file"); + + di = PathUtils.fromURI(new URI(null, null, "/home/test", null), "file"); + assertThat(di.getName()).isEqualTo("/home/test"); + assertThat(di.getNamespace()).isEqualTo("file"); + + di = + PathUtils.fromURI( + new URI("hdfs", null, "localhost", 8020, "/home/test", null, null), "file"); + assertThat(di.getName()).isEqualTo("/home/test"); + assertThat(di.getNamespace()).isEqualTo("hdfs://localhost:8020"); + + di = PathUtils.fromURI(new URI("s3://data-bucket/path"), "file"); + assertThat(di.getName()).isEqualTo("path"); + assertThat(di.getNamespace()).isEqualTo("s3://data-bucket"); + } + + @Test + void testFromCatalogTableWithHiveTables() throws URISyntaxException { + SparkSession sparkSession = mock(SparkSession.class); + SparkContext sparkContext = mock(SparkContext.class); + SparkConf sparkConf = new SparkConf(); + sparkConf.set("spark.sql.catalogImplementation", "hive"); + sparkConf.set("spark.sql.hive.metastore.uris", "thrift://10.1.0.1:9083"); + when(sparkContext.getConf()).thenReturn(sparkConf); + when(sparkSession.sparkContext()).thenReturn(sparkContext); + + CatalogTable catalogTable = mock(CatalogTable.class); + when(catalogTable.qualifiedName()).thenReturn("table"); + + DatasetIdentifier di = PathUtils.fromCatalogTable(catalogTable, Optional.of(sparkConf)); + assertThat(di.getName()).isEqualTo("table"); + assertThat(di.getNamespace()).isEqualTo("hive://10.1.0.1:9083"); + + sparkConf.set( + "spark.sql.hive.metastore.uris", "anotherprotocol://127.0.0.1:1010,yetanother://something"); + di = PathUtils.fromCatalogTable(catalogTable, Optional.of(sparkConf)); + assertThat(di.getName()).isEqualTo("table"); + assertThat(di.getNamespace()).isEqualTo("hive://127.0.0.1:1010"); + + sparkConf.remove("spark.sql.hive.metastore.uris"); + sparkConf.set("spark.hadoop.hive.metastore.uris", "thrift://10.1.0.1:9083"); + di = PathUtils.fromCatalogTable(catalogTable, Optional.of(sparkConf)); + assertThat(di.getName()).isEqualTo("table"); + assertThat(di.getNamespace()).isEqualTo("hive://10.1.0.1:9083"); + } + + @Test + void testFromCatalogWithNoHiveMetastoreAndHdfsLocation() throws URISyntaxException { + CatalogTable catalogTable = mock(CatalogTable.class); + CatalogStorageFormat catalogStorageFormat = mock(CatalogStorageFormat.class); + SparkConf sparkConf = new SparkConf(); + + sparkConf.remove("spark.hadoop.hive.metastore.uris"); + when(catalogTable.storage()).thenReturn(catalogStorageFormat); + when(catalogStorageFormat.locationUri()) + .thenReturn(Option.apply(new URI("hdfs://namenode:8020/warehouse/table"))); + DatasetIdentifier di = PathUtils.fromCatalogTable(catalogTable, Optional.of(sparkConf)); + assertThat(di.getName()).isEqualTo("/warehouse/table"); + assertThat(di.getNamespace()).isEqualTo("hdfs://namenode:8020"); + } + + @Test + void testFromCatalogExceptionIsThrownWhenUnableToExtractDatasetIdentifier() { + CatalogTable catalogTable = mock(CatalogTable.class); + assertThrows(IllegalArgumentException.class, () -> PathUtils.fromCatalogTable(catalogTable)); + } + + @Test + void testEnrichMetastoreUriWithTableName() throws URISyntaxException { + assertThat(enrichHiveMetastoreURIWithTableName(new URI("thrift://10.1.0.1:9083"), "/db/table")) + .isEqualTo(new URI("hive://10.1.0.1:9083/db/table")); + } + + @Test + void testFromCatalogFromDefaultTablePath() throws URISyntaxException { + SparkSession sparkSession = mock(SparkSession.class); + SessionState sessionState = mock(SessionState.class); + SessionCatalog sessionCatalog = mock(SessionCatalog.class); + + when(sparkSession.sessionState()).thenReturn(sessionState); + when(sessionState.catalog()).thenReturn(sessionCatalog); + when(sessionCatalog.defaultTablePath(any())).thenReturn(new URI("/warehouse/table")); + + try (MockedStatic mocked = mockStatic(SparkSession.class)) { + mocked.when(SparkSession::active).thenReturn(sparkSession); + DatasetIdentifier di = + PathUtils.fromCatalogTable(mock(CatalogTable.class), Optional.of(new SparkConf())); + + assertThat(di.getName()).isEqualTo("/warehouse/table"); + assertThat(di.getNamespace()).isEqualTo("file"); + } + } +} diff --git a/integration/spark/src/test/resources/log4j.properties b/integration/spark/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..54ff19e3d7f93 --- /dev/null +++ b/integration/spark/src/test/resources/log4j.properties @@ -0,0 +1,9 @@ +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + +# set the log level for the openlineage spark library +log4j.logger.io.openlineage=DEBUG diff --git a/integration/spark/src/test/resources/spark_scripts/spark_alter_table.py b/integration/spark/src/test/resources/spark_scripts/spark_alter_table.py new file mode 100644 index 0000000000000..e4bb7ff43f575 --- /dev/null +++ b/integration/spark/src/test/resources/spark_scripts/spark_alter_table.py @@ -0,0 +1,21 @@ +import os +import time + +os.makedirs("/tmp/ctas_load", exist_ok=True) + +from pyspark.sql import SparkSession + +spark = SparkSession.builder \ + .master("local") \ + .appName("Open Lineage Integration Alter Table") \ + .config("spark.sql.warehouse.dir", "file:/tmp/alter_test/") \ + .enableHiveSupport() \ + .getOrCreate() + +spark.sparkContext.setLogLevel('info') + +spark.sql("CREATE TABLE alter_table_test (a string, b string)") +time.sleep(1) +spark.sql("ALTER TABLE alter_table_test ADD COLUMNS (c string, d string)") +time.sleep(1) +spark.sql("ALTER TABLE alter_table_test RENAME TO alter_table_test_new") diff --git a/integration/spark/src/test/resources/spark_scripts/spark_create_table.py b/integration/spark/src/test/resources/spark_scripts/spark_create_table.py new file mode 100644 index 0000000000000..a65a8bf4bc1a7 --- /dev/null +++ b/integration/spark/src/test/resources/spark_scripts/spark_create_table.py @@ -0,0 +1,16 @@ +import os + +os.makedirs("/tmp/create_test", exist_ok=True) + +from pyspark.sql import SparkSession + +spark = SparkSession.builder \ + .master("local") \ + .appName("Open Lineage Integration Create Table") \ + .config("spark.sql.warehouse.dir", "file:/tmp/create_test/") \ + .enableHiveSupport() \ + .getOrCreate() + +spark.sparkContext.setLogLevel('info') + +spark.sql("CREATE TABLE create_table_test (a string, b string)") \ No newline at end of file diff --git a/integration/spark/src/test/resources/spark_scripts/spark_ctas_load.py b/integration/spark/src/test/resources/spark_scripts/spark_ctas_load.py new file mode 100644 index 0000000000000..30c7017a0020f --- /dev/null +++ b/integration/spark/src/test/resources/spark_scripts/spark_ctas_load.py @@ -0,0 +1,25 @@ +import os +import time + +os.makedirs("/tmp/ctas_load", exist_ok=True) + +from pyspark.sql import SparkSession + +spark = SparkSession.builder \ + .master("local") \ + .appName("Open Lineage Integration CTAS Load") \ + .config("spark.sql.warehouse.dir", "/tmp/ctas_load") \ + .enableHiveSupport() \ + .getOrCreate() +spark.sparkContext.setLogLevel('info') + +df = spark.createDataFrame([ + {'a': 1, 'b': 2}, + {'a': 3, 'b': 4} +]) + +df.createOrReplaceTempView('temp') + +spark.sql("CREATE TABLE tbl1 USING hive LOCATION '/tmp/ctas_load/tbl1' AS SELECT a, b FROM temp") + +spark.sql(f"LOAD DATA LOCAL INPATH '/test_data/test_data.csv' INTO TABLE tbl1") diff --git a/integration/spark/src/test/resources/spark_scripts/spark_delta.py b/integration/spark/src/test/resources/spark_scripts/spark_delta.py new file mode 100644 index 0000000000000..c12619c63317b --- /dev/null +++ b/integration/spark/src/test/resources/spark_scripts/spark_delta.py @@ -0,0 +1,21 @@ +import os +from pyspark.sql import SparkSession + +os.makedirs("/tmp/delta", exist_ok=True) + +spark = SparkSession.builder \ + .master("local") \ + .appName("Open Lineage Integration Delta") \ + .config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") \ + .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") \ + .config("packages", "io.delta:delta-core_2.12:1.0.0") \ + .getOrCreate() +spark.sparkContext.setLogLevel('info') + +df = spark.createDataFrame([ + {'a': 1, 'b': 2}, + {'a': 3, 'b': 4} +]) +df.createOrReplaceTempView('temp') + +spark.sql("CREATE TABLE tbl USING delta LOCATION '/tmp/delta' AS SELECT * FROM temp") diff --git a/integration/spark/src/test/resources/spark_scripts/spark_drop_table.py b/integration/spark/src/test/resources/spark_scripts/spark_drop_table.py new file mode 100644 index 0000000000000..8b2b8c37ed967 --- /dev/null +++ b/integration/spark/src/test/resources/spark_scripts/spark_drop_table.py @@ -0,0 +1,18 @@ +import os + +os.makedirs("/tmp/ctas_load", exist_ok=True) + +from pyspark.sql import SparkSession + +spark = SparkSession.builder \ + .master("local") \ + .appName("Open Lineage Integration Drop Table") \ + .config("spark.sql.warehouse.dir", "file:/tmp/drop_test/") \ + .enableHiveSupport() \ + .getOrCreate() + +spark.sparkContext.setLogLevel('info') + +spark.sql("CREATE TABLE drop_table_test (a string, b string)") + +spark.sql("DROP TABLE drop_table_test") \ No newline at end of file diff --git a/integration/spark/src/test/resources/spark_scripts/spark_octas_load.py b/integration/spark/src/test/resources/spark_scripts/spark_octas_load.py new file mode 100644 index 0000000000000..0afe36271bd10 --- /dev/null +++ b/integration/spark/src/test/resources/spark_scripts/spark_octas_load.py @@ -0,0 +1,22 @@ +import os + +os.makedirs("/tmp/ctas_load", exist_ok=True) + +from pyspark.sql import SparkSession + +spark = SparkSession.builder \ + .master("local") \ + .appName("Open Lineage Integration OCTAS Load") \ + .config("spark.sql.warehouse.dir", "/tmp/ctas_load") \ + .enableHiveSupport() \ + .getOrCreate() +spark.sparkContext.setLogLevel('info') + +df = spark.createDataFrame([ + {'a': 1, 'b': 2}, + {'a': 3, 'b': 4} +]) + +df.createOrReplaceTempView('temp') + +spark.sql("CREATE TABLE tbl2 STORED AS PARQUET LOCATION '/tmp/ctas_load/tbl2' AS SELECT a, b FROM temp") diff --git a/integration/spark/src/test/resources/test_data/serde/aggregate-node.json b/integration/spark/src/test/resources/test_data/serde/aggregate-node.json index b5b23e8d7754f..70c4e173c4242 100644 --- a/integration/spark/src/test/resources/test_data/serde/aggregate-node.json +++ b/integration/spark/src/test/resources/test_data/serde/aggregate-node.json @@ -2,32 +2,18 @@ "@class": "org.apache.spark.sql.catalyst.plans.logical.Aggregate", "groupingExpressions": [], "aggregateExpressions": [], - "origin": { - "line": null, - "startPosition": null - }, - "schema": [], "allAttributes": { "attrs": [ { "name": "name", - "dataType": { - "ordering": {} - }, "nullable": false, - "metadata": null, "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, "deterministic": true, "resolved": true } ] }, "resolved": true, - "statsCache": null, "traceEnabled": false, "streaming": false, "canonicalizedPlan": false diff --git a/integration/spark/src/test/resources/test_data/serde/bigqueryrelation-node.json b/integration/spark/src/test/resources/test_data/serde/bigqueryrelation-node.json index f177caf46b2bb..46cf77c7a869a 100644 --- a/integration/spark/src/test/resources/test_data/serde/bigqueryrelation-node.json +++ b/integration/spark/src/test/resources/test_data/serde/bigqueryrelation-node.json @@ -1,5 +1,7 @@ { + "@class": "org.apache.spark.sql.execution.datasources.LogicalRelation", "relation": { + "@class": "com.google.cloud.spark.bigquery.BigQueryRelation", "options": { "tableId": { "project": null, @@ -14,122 +16,80 @@ "credentialsKey": { "present": false }, - "credentialsFile": { - "present": false - }, - "accessToken": { - "present": false - }, - "filter": { - "present": false - }, - "schema": { - "present": false - }, + "credentialsFile": 6, + "accessToken": 6, + "filter": 6, "maxParallelism": { - "asInt": 2, - "present": true + "isPresent": true, + "asInt": 2 }, "defaultParallelism": 10, - "temporaryGcsBucket": { - "present": false - }, - "persistentGcsBucket": { - "present": false - }, - "persistentGcsPath": { - "present": false - }, + "temporaryGcsBucket": 6, + "persistentGcsBucket": 6, + "persistentGcsPath": 6, "intermediateFormat": "PARQUET", "readDataFormat": "ARROW", "combinePushedDownFilters": true, "viewsEnabled": false, - "materializationProject": { - "present": false - }, - "materializationDataset": { - "present": false - }, - "partitionField": { - "present": false - }, + "materializationProject": 6, + "materializationDataset": 6, + "partitionField": 6, "partitionExpirationMs": { - "asLong": 2, - "present": true - }, - "partitionRequireFilter": { - "present": false - }, - "partitionType": { - "present": false - }, - "clusteredFields": { - "present": false - }, - "createDisposition": { - "present": false + "isPresent": true, + "asLong": 2 }, + "partitionRequireFilter": 6, + "partitionType": 6, + "clusteredFields": 6, + "createDisposition": 6, "optimizedEmptyProjection": true, "useAvroLogicalTypes": false, "loadSchemaUpdateOptions": [], "materializationExpirationTimeInMinutes": 1440, "maxReadRowsRetries": 3, + "pushAllFilters": true, + "tableIdWithoutThePartition": 4, "partitionTypeOrDefault": "DAY", "bigQueryClientConnectTimeout": 60000, "bigQueryClientReadTimeout": 60000, "bigQueryClientRetrySettings": { "totalTimeout": { "seconds": 600, - "zero": false, "negative": false, - "units": [ - "SECONDS", - "NANOS" - ], + "zero": false, + "units": ["SECONDS", "NANOS"], "nano": 0 }, "initialRetryDelay": { "seconds": 1, - "zero": false, "negative": false, - "units": [ - "SECONDS", - "NANOS" - ], + "zero": false, + "units": ["SECONDS", "NANOS"], "nano": 250000000 }, "retryDelayMultiplier": 1.6, "maxRetryDelay": { "seconds": 5, - "zero": false, "negative": false, - "units": [ - "SECONDS", - "NANOS" - ], + "zero": false, + "units": ["SECONDS", "NANOS"], "nano": 0 }, "maxAttempts": 0, "jittered": true, "initialRpcTimeout": { "seconds": 60, - "zero": false, "negative": false, - "units": [ - "SECONDS", - "NANOS" - ], + "zero": false, + "units": ["SECONDS", "NANOS"], "nano": 0 }, "rpcTimeoutMultiplier": 1.6, "maxRpcTimeout": { "seconds": 300, - "zero": false, "negative": false, - "units": [ - "SECONDS", - "NANOS" - ], + "zero": false, + "units": ["SECONDS", "NANOS"], "nano": 0 } } @@ -152,119 +112,28 @@ "numBytes": null, "numLongTermBytes": null, "numRows": null, - "definition": { - "type": {}, - "schema": { - "fields": [ - { - "name": "name", - "type": { - "standardType": "STRING" - }, - "subFields": null, - "mode": null, - "description": null, - "policyTags": null - } - ] - } - }, - "encryptionConfiguration": null, - "labels": {}, - "requirePartitionFilter": null - }, - "tableId": { - "project": null, - "dataset": "dataset", - "table": "test", - "iamresourceName": "projects/null/datasets/dataset/tables/test" - }, - "tableName": "dataset.test", - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": true, - "metadata": { - "map": {} - }, - "comment": null + "definition": {} } - ], + }, "traceEnabled": false }, - "output": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "exprId": { - "id": 1, - "jvmId": "a822420c-ca2f-483b-a3c1-4e561ccae8d6" - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - ], + "output": [{ + "name": "name", + "nullable": false, + "metadata": { + "map": {} + }, + "qualifier": [], + "deterministic": true, + "resolved": true + }], "catalogTable": null, "isStreaming": false, - "origin": { - "line": null, - "startPosition": null - }, - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], "allAttributes": { "attrs": [] }, "resolved": true, "statsCache": null, - "attributeMap": { - "name#1": { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "exprId": { - "id": 1, - "jvmId": "a822420c-ca2f-483b-a3c1-4e561ccae8d6" - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - }, "traceEnabled": false, "canonicalizedPlan": false } \ No newline at end of file diff --git a/integration/spark/src/test/resources/test_data/serde/hadoopfsrelation-node.json b/integration/spark/src/test/resources/test_data/serde/hadoopfsrelation-node.json index 1a95afbe54c74..a207fdfc914a8 100644 --- a/integration/spark/src/test/resources/test_data/serde/hadoopfsrelation-node.json +++ b/integration/spark/src/test/resources/test_data/serde/hadoopfsrelation-node.json @@ -1,11 +1,8 @@ { - "@class":"org.apache.spark.sql.execution.datasources.LogicalRelation", + "@class": "org.apache.spark.sql.execution.datasources.LogicalRelation", "relation": { - "@class":"org.apache.spark.sql.execution.datasources.HadoopFsRelation", + "@class": "org.apache.spark.sql.execution.datasources.HadoopFsRelation", "location": { - "sparkSession": { - "traceEnabled": false - }, "table": { "identifier": { "table": "test", @@ -16,34 +13,17 @@ "name": "MANAGED" }, "storage": { - "locationUri": null, + "locationUri": "/some-location", "inputFormat": null, "outputFormat": null, "serde": null, "compressed": false, "properties": {} }, - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], "provider": null, - "partitionColumnNames": [ - "name" - ], + "partitionColumnNames": ["name"], "bucketSpec": null, "owner": "", - "createTime": 1631621948, - "lastAccessTime": 1631621948, "createVersion": "v1", "properties": {}, "stats": null, @@ -56,141 +36,27 @@ }, "sizeInBytes": 100, "hadoopConf": { - "finalParameters": [ - "mapreduce.job.end-notification.max.retry.interval", - "mapreduce.job.end-notification.max.attempts" - ] + "finalParameters": ["mapreduce.job.end-notification.max.retry.interval", "mapreduce.job.end-notification.max.attempts"] } }, - "partitionSchema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], - "dataSchema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], "bucketSpec": null, "fileFormat": {}, - "options": {}, - "sparkSession": { - "traceEnabled": false - }, - "overlappedPartCols": { - "name": { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - }, - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ] + "options": {} }, - "output": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "exprId": { - "id": 1, - "jvmId": "bd5688b3-28fa-4539-9fdc-9f52e777a577" - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - ], + "output": [{ + "name": "name", + "nullable": false, + "qualifier": [], + "deterministic": true, + "resolved": true + }], "catalogTable": null, "isStreaming": false, - "origin": { - "line": null, - "startPosition": null - }, - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], "allAttributes": { "attrs": [] }, "resolved": true, "statsCache": null, - "attributeMap": { - "name#1": { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "exprId": { - "id": 1, - "jvmId": "bd5688b3-28fa-4539-9fdc-9f52e777a577" - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - }, "traceEnabled": false, "canonicalizedPlan": false } \ No newline at end of file diff --git a/integration/spark/src/test/resources/test_data/serde/insertintods-node.json b/integration/spark/src/test/resources/test_data/serde/insertintods-node.json index 18872af479c21..e909c4715589d 100644 --- a/integration/spark/src/test/resources/test_data/serde/insertintods-node.json +++ b/integration/spark/src/test/resources/test_data/serde/insertintods-node.json @@ -1,390 +1,145 @@ { + "@class": "org.apache.spark.sql.execution.datasources.InsertIntoDataSourceCommand", "logicalRelation": { - "@class":"org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand", + "@class": "org.apache.spark.sql.execution.datasources.LogicalRelation", "relation": { - "location": { - "sparkSession": { - "traceEnabled": false - }, - "table": { - "identifier": { - "table": "test", - "database": "db", - "identifier": "test" - }, - "tableType": { - "name": "MANAGED" - }, - "storage": { - "locationUri": null, - "inputFormat": null, - "outputFormat": null, - "serde": null, - "compressed": false, - "properties": {} - }, - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], - "provider": null, - "partitionColumnNames": [ - "name" - ], - "bucketSpec": null, - "owner": "", - "createTime": 1631621948, - "lastAccessTime": 1631621948, - "createVersion": "v1", - "properties": {}, - "stats": null, - "viewText": null, - "comment": null, - "unsupportedFeatures": [], - "tracksPartitionsInCatalog": false, - "schemaPreservesCase": false, - "ignoredProperties": {} - }, - "sizeInBytes": 100, - "hadoopConf": { - "finalParameters": [ - "mapreduce.job.end-notification.max.retry.interval", - "mapreduce.job.end-notification.max.attempts" - ] + "@class": "com.google.cloud.spark.bigquery.BigQueryRelation", + "options": { + "tableId": { + "dataset": "test-dataset", + "table": "QUERY", + "iamresourceName": "projects/null/datasets/test-dataset/tables/QUERY" + }, + "query": { + "present": true + }, + "parentProjectId": "test_serialization", + "credentialsKey": { + "present": false + }, + "credentialsFile": 7, + "accessToken": 7, + "filter": 7, + "maxParallelism": { + "isPresent": true, + "asInt": 2 + }, + "defaultParallelism": 10, + "temporaryGcsBucket": 7, + "persistentGcsBucket": 7, + "persistentGcsPath": 7, + "intermediateFormat": "PARQUET", + "readDataFormat": "ARROW", + "combinePushedDownFilters": true, + "viewsEnabled": false, + "materializationProject": 7, + "materializationDataset": 7, + "partitionField": 7, + "partitionExpirationMs": { + "isPresent": true, + "asLong": 2 + }, + "partitionRequireFilter": 7, + "partitionType": 7, + "clusteredFields": 7, + "createDisposition": 7, + "optimizedEmptyProjection": true, + "useAvroLogicalTypes": false, + "loadSchemaUpdateOptions": [], + "materializationExpirationTimeInMinutes": 1440, + "maxReadRowsRetries": 3, + "pushAllFilters": true, + "tableIdWithoutThePartition": 5, + "partitionTypeOrDefault": "DAY", + "bigQueryClientConnectTimeout": 60000, + "bigQueryClientReadTimeout": 60000, + "bigQueryClientRetrySettings": { + "totalTimeout": { + "seconds": 600, + "negative": false, + "zero": false, + "units": ["SECONDS", "NANOS"], + "nano": 0 + }, + "initialRetryDelay": { + "seconds": 1, + "negative": false, + "zero": false, + "units": ["SECONDS", "NANOS"], + "nano": 250000000 + }, + "retryDelayMultiplier": 1.6, + "maxRetryDelay": { + "seconds": 5, + "negative": false, + "zero": false, + "units": ["SECONDS", "NANOS"], + "nano": 0 + }, + "maxAttempts": 0, + "jittered": true, + "initialRpcTimeout": { + "seconds": 60, + "negative": false, + "zero": false, + "units": ["SECONDS", "NANOS"], + "nano": 0 + }, + "rpcTimeoutMultiplier": 1.6, + "maxRpcTimeout": { + "seconds": 300, + "negative": false, + "zero": false, + "units": ["SECONDS", "NANOS"], + "nano": 0 + } } }, - "partitionSchema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], - "dataSchema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], - "bucketSpec": null, - "fileFormat": {}, - "options": {}, - "sparkSession": { - "traceEnabled": false - }, - "overlappedPartCols": { - "name": { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - }, - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ] - }, - "output": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - ], - "catalogTable": null, - "isStreaming": false, - "origin": { - "line": null, - "startPosition": null - }, - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], - "allAttributes": { - "attrs": [] - }, - "resolved": true, - "statsCache": null, - "attributeMap": { - "name#1": { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - }, - "traceEnabled": false, - "canonicalizedPlan": false - }, - "query": { - "relation": { - "location": { - "sparkSession": { - "traceEnabled": false - }, - "table": { - "identifier": { - "table": "test", - "database": "db", - "identifier": "test" - }, - "tableType": { - "name": "MANAGED" - }, - "storage": { - "locationUri": null, - "inputFormat": null, - "outputFormat": null, - "serde": null, - "compressed": false, - "properties": {} - }, - "schema": [ - { + "table": { + "tableId": { + "dataset": "dataset", + "table": "test", + "iamresourceName": "projects/null/datasets/dataset/tables/test" + }, + "definition": { + "schema": { + "fields": [{ "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], - "provider": null, - "partitionColumnNames": [ - "name" - ], - "bucketSpec": null, - "owner": "", - "createTime": 1631621948, - "lastAccessTime": 1631621948, - "createVersion": "v1", - "properties": {}, - "stats": null, - "viewText": null, - "comment": null, - "unsupportedFeatures": [], - "tracksPartitionsInCatalog": false, - "schemaPreservesCase": false, - "ignoredProperties": {} - }, - "sizeInBytes": 100, - "hadoopConf": { - "finalParameters": [ - "mapreduce.job.end-notification.max.retry.interval", - "mapreduce.job.end-notification.max.attempts" - ] - } - }, - "partitionSchema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], - "dataSchema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], - "bucketSpec": null, - "fileFormat": {}, - "options": {}, - "sparkSession": { - "traceEnabled": false - }, - "overlappedPartCols": { - "name": { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null + "type": { + "standardType": "STRING" + } + }] + } } }, - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ] + "traceEnabled": false }, - "output": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - ], - "catalogTable": null, + "output": [{ + "name": "name", + "nullable": false, + "metadata": { + "map": {} + }, + "qualifier": [], + "deterministic": true, + "resolved": true + }], "isStreaming": false, - "origin": { - "line": null, - "startPosition": null - }, - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], "allAttributes": { "attrs": [] }, "resolved": true, - "statsCache": null, - "attributeMap": { - "name#1": { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - }, "traceEnabled": false, "canonicalizedPlan": false }, + "query": 2, "overwrite": false, - "origin": { - "line": null, - "startPosition": null - }, - "schema": [], "allAttributes": { "attrs": [] }, "resolved": true, - "statsCache": null, "metrics": {}, - "traceEnabled": false, "streaming": false, + "traceEnabled": false, "canonicalizedPlan": false } \ No newline at end of file diff --git a/integration/spark/src/test/resources/test_data/serde/insertintofs-node.json b/integration/spark/src/test/resources/test_data/serde/insertintofs-node.json index 900c666450bc2..45dac98fd271d 100644 --- a/integration/spark/src/test/resources/test_data/serde/insertintofs-node.json +++ b/integration/spark/src/test/resources/test_data/serde/insertintofs-node.json @@ -1,5 +1,5 @@ { - "@class":"org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand", + "@class": "org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand", "outputPath": { "root": false, "uriPathAbsolute": true, @@ -17,38 +17,28 @@ }, "staticPartitions": {}, "ifPartitionNotExists": false, - "partitionColumns": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "exprId": { - "id": 1, - "jvmId": "1a41a881-0cda-4d15-9d33-048db59eb018" - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - ], + "partitionColumns": [{ + "name": "name", + "nullable": false, + "metadata": { + "map": {} + }, + "exprId": { + "id": 1, + "jvmId": "9a566e0c-8f15-4239-9222-8f8dfd4557a1" + }, + "qualifier": [], + "deterministic": true, + "resolved": true + }], "bucketSpec": null, "fileFormat": {}, "options": {}, "query": { + "@class": "org.apache.spark.sql.execution.datasources.LogicalRelation", "relation": { + "@class": "org.apache.spark.sql.execution.datasources.HadoopFsRelation", "location": { - "sparkSession": { - "traceEnabled": false - }, "table": { "identifier": { "table": "test", @@ -59,34 +49,17 @@ "name": "MANAGED" }, "storage": { - "locationUri": null, + "locationUri": "/some-location", "inputFormat": null, "outputFormat": null, "serde": null, "compressed": false, "properties": {} }, - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], "provider": null, - "partitionColumnNames": [ - "name" - ], + "partitionColumnNames": ["name"], "bucketSpec": null, "owner": "", - "createTime": 1631630712, - "lastAccessTime": 1631630712, "createVersion": "v1", "properties": {}, "stats": null, @@ -99,233 +72,74 @@ }, "sizeInBytes": 100, "hadoopConf": { - "finalParameters": [ - "mapreduce.job.end-notification.max.retry.interval", - "mapreduce.job.end-notification.max.attempts" - ] + "finalParameters": ["mapreduce.job.end-notification.max.retry.interval", "mapreduce.job.end-notification.max.attempts"] } }, - "partitionSchema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], - "dataSchema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], "bucketSpec": null, - "fileFormat": {}, - "options": {}, - "sparkSession": { - "traceEnabled": false + "fileFormat": { }, - "overlappedPartCols": { - "name": { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - }, - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ] + "options": {} }, - "output": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "exprId": { - "id": 1, - "jvmId": "1a41a881-0cda-4d15-9d33-048db59eb018" - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - ], + "output": [{ + "name": "name", + "nullable": false, + "qualifier": [], + "deterministic": true, + "resolved": true + }], "catalogTable": null, "isStreaming": false, - "origin": { - "line": null, - "startPosition": null - }, - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], "allAttributes": { "attrs": [] }, "resolved": true, "statsCache": null, - "attributeMap": { - "name#1": { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "exprId": { - "id": 1, - "jvmId": "1a41a881-0cda-4d15-9d33-048db59eb018" - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - }, "traceEnabled": false, "canonicalizedPlan": false }, "mode": "Overwrite", "catalogTable": null, "fileIndex": null, - "outputColumnNames": [ - "name" - ], - "origin": { - "line": null, - "startPosition": null - }, - "schema": [], - "allAttributes": { - "attrs": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "exprId": { - "id": 1, - "jvmId": "1a41a881-0cda-4d15-9d33-048db59eb018" - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - ] - }, "resolved": true, "statsCache": null, "metrics": { "numFiles": { - "metricType": "sum", "metadata": { - "id": 0, "name": "number of written files", "countFailedValues": false }, - "org$apache$spark$util$AccumulatorV2$$atDriverSide": true, "zero": true, "atDriverSide": true, "registered": true }, "numOutputBytes": { - "metricType": "sum", "metadata": { - "id": 1, - "name": "bytes of written output", "countFailedValues": false }, - "org$apache$spark$util$AccumulatorV2$$atDriverSide": true, "zero": true, "atDriverSide": true, "registered": true }, "numOutputRows": { - "metricType": "sum", "metadata": { - "id": 2, "name": "number of output rows", "countFailedValues": false }, - "org$apache$spark$util$AccumulatorV2$$atDriverSide": true, "zero": true, "atDriverSide": true, "registered": true }, "numParts": { - "metricType": "sum", "metadata": { - "id": 3, "name": "number of dynamic part", "countFailedValues": false }, - "org$apache$spark$util$AccumulatorV2$$atDriverSide": true, "zero": true, "atDriverSide": true, "registered": true } }, - "traceEnabled": false, "streaming": false, + "traceEnabled": false, "canonicalizedPlan": false } \ No newline at end of file diff --git a/integration/spark/src/test/resources/test_data/serde/logicalrelation-node.json b/integration/spark/src/test/resources/test_data/serde/logicalrelation-node.json index 4b9759813e3d3..12a74f781fb0b 100644 --- a/integration/spark/src/test/resources/test_data/serde/logicalrelation-node.json +++ b/integration/spark/src/test/resources/test_data/serde/logicalrelation-node.json @@ -1,27 +1,16 @@ { + "@class": "org.apache.spark.sql.execution.datasources.LogicalRelation", "relation": { - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], + "@class": "org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation", "parts": [], "jdbcOptions": { - "url": "jdbc:postgresql://postgreshost:5432/sparkdata", "asProperties": { - "url": "jdbc:postgresql://postgreshost:5432/sparkdata", + "driver": "org.postgresql.Driver", "dbtable": "my_spark_table", - "driver": "org.postgresql.Driver" + "url": "jdbc:postgresql://postgreshost:5432/sparkdata" }, "asConnectionProperties": {}, + "url": "jdbc:postgresql://postgreshost:5432/sparkdata", "tableOrQuery": "my_spark_table", "driverClass": "org.postgresql.Driver", "numPartitions": null, @@ -42,72 +31,20 @@ }, "needConversion": false }, - "output": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "exprId": { - "id": 1, - "jvmId": "f7dff447-fa8b-4738-b173-7855f66f8614" - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - ], + "output": [{ + "name": "name", + "nullable": false, + "qualifier": [], + "deterministic": true, + "resolved": true + }], "catalogTable": null, "isStreaming": false, - "origin": { - "line": null, - "startPosition": null - }, - "schema": [ - { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "comment": null - } - ], "allAttributes": { "attrs": [] }, "resolved": true, "statsCache": null, - "attributeMap": { - "name#1": { - "name": "name", - "dataType": { - "ordering": {} - }, - "nullable": false, - "metadata": { - "map": {} - }, - "qualifier": [], - "origin": { - "line": null, - "startPosition": null - }, - "deterministic": true, - "resolved": true - } - }, "traceEnabled": false, "canonicalizedPlan": false } \ No newline at end of file diff --git a/integration/spark/src/test/resources/test_data/serde/openlineage-event.json b/integration/spark/src/test/resources/test_data/serde/openlineage-event.json index bba0845fca73e..41765ce58e14e 100644 --- a/integration/spark/src/test/resources/test_data/serde/openlineage-event.json +++ b/integration/spark/src/test/resources/test_data/serde/openlineage-event.json @@ -4,11 +4,11 @@ "run": { "runId": "5f24c93c-2ce9-49dc-82e7-95ab4915242f", "facets": { - "nominalTime": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/0.2.3-SNAPSHOT/integration/spark", - "nominalStartTime": "2021-01-01T00:00:01Z", - "nominalEndTime": "2021-01-01T00:00:01Z", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/NominalTimeRunFacet" + "parent": { + "job": { + "namespace": "namespace", + "name": "jobName" + } } } }, @@ -17,25 +17,19 @@ "name": "jobName", "facets": { "documentation": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/0.2.3-SNAPSHOT/integration/spark", - "description": "test documentation", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/DocumentationJobFacet" + "description": "test documentation" }, "sourceCodeLocation": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/0.2.3-SNAPSHOT/integration/spark", "type": "git", "url": "https://github.com/apache/spark", "repoUrl": "https://github.com/apache/spark", "path": "/path/to/file", "version": "v1", "tag": "v1.0.0", - "branch": "branch", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/SourceCodeLocationJobFacet" + "branch": "branch" }, "sql": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/0.2.3-SNAPSHOT/integration/spark", - "query": "SELECT * FROM test", - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/SQLJobFacet" + "query": "SELECT * FROM test" } } }, @@ -45,7 +39,6 @@ "name": "input", "inputFacets": { "dataQualityMetrics": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/0.2.3-SNAPSHOT/integration/spark", "rowCount": 10, "bytes": 20, "columnMetrics": { @@ -60,8 +53,7 @@ "25": 52.0 } } - }, - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/DataQualityMetricsInputDatasetFacet" + } } } } @@ -72,14 +64,10 @@ "name": "output", "outputFacets": { "outputStatistics": { - "_producer": "https://github.com/OpenLineage/OpenLineage/tree/0.2.3-SNAPSHOT/integration/spark", "rowCount": 10, - "size": 20, - "_schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/OutputStatisticsOutputDatasetFacet" + "size": 20 } } } - ], - "producer": "https://github.com/OpenLineage/OpenLineage/tree/0.2.3-SNAPSHOT/integration/spark", - "schemaURL": "https://openlineage.io/spec/1-0-1/OpenLineage.json#/definitions/RunEvent" + ] } \ No newline at end of file diff --git a/integration/spark/src/test/resources/test_data/test_data.csv b/integration/spark/src/test/resources/test_data/test_data.csv new file mode 100644 index 0000000000000..10c73689a5b03 --- /dev/null +++ b/integration/spark/src/test/resources/test_data/test_data.csv @@ -0,0 +1,2 @@ +1,2 +3,4 \ No newline at end of file diff --git a/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateDataSourceTableAsSelectCommandVisitorTest.java b/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateDataSourceTableAsSelectCommandVisitorTest.java new file mode 100644 index 0000000000000..6b2af8ed074d2 --- /dev/null +++ b/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateDataSourceTableAsSelectCommandVisitorTest.java @@ -0,0 +1,67 @@ +package io.openlineage.spark2.agent.lifecycle.plan; + +import static org.assertj.core.api.Assertions.assertThat; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.SparkAgentTestExtension; +import io.openlineage.spark.agent.lifecycle.plan.CreateDataSourceTableAsSelectCommandVisitor; +import java.net.URI; +import java.util.List; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier$; +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat$; +import org.apache.spark.sql.catalyst.catalog.CatalogTableType; +import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand; +import org.apache.spark.sql.types.IntegerType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import scala.Option; +import scala.collection.Map$; +import scala.collection.Seq$; +import scala.collection.immutable.HashMap; + +@ExtendWith(SparkAgentTestExtension.class) +class CreateDataSourceTableAsSelectCommandVisitorTest { + + @Test + void testCTASCommand() { + SparkSession session = SparkSession.builder().master("local").getOrCreate(); + + CreateDataSourceTableAsSelectCommandVisitor visitor = + new CreateDataSourceTableAsSelectCommandVisitor(); + + CreateDataSourceTableAsSelectCommand command = + new CreateDataSourceTableAsSelectCommand( + SparkUtils.catalogTable( + TableIdentifier$.MODULE$.apply("tablename", Option.apply("db")), + CatalogTableType.EXTERNAL(), + CatalogStorageFormat$.MODULE$.apply( + Option.apply(URI.create("s3://bucket/directory")), + null, + null, + null, + false, + Map$.MODULE$.empty()), + new StructType( + new StructField[] { + new StructField( + "key", IntegerType$.MODULE$, false, new Metadata(new HashMap<>())), + new StructField( + "value", StringType$.MODULE$, false, new Metadata(new HashMap<>())) + })), + null, + null, + Seq$.MODULE$.empty()); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", "directory") + .hasFieldOrPropertyWithValue("namespace", "s3://bucket"); + } +} diff --git a/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateDataSourceTableCommandVisitorTest.java b/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateDataSourceTableCommandVisitorTest.java new file mode 100644 index 0000000000000..dc8ed4ca92837 --- /dev/null +++ b/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateDataSourceTableCommandVisitorTest.java @@ -0,0 +1,62 @@ +package io.openlineage.spark2.agent.lifecycle.plan; + +import static org.assertj.core.api.Assertions.assertThat; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.SparkAgentTestExtension; +import io.openlineage.spark.agent.lifecycle.plan.CreateDataSourceTableCommandVisitor; +import java.net.URI; +import java.util.List; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier$; +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat$; +import org.apache.spark.sql.catalyst.catalog.CatalogTableType; +import org.apache.spark.sql.execution.command.CreateDataSourceTableCommand; +import org.apache.spark.sql.types.IntegerType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import scala.Option; +import scala.collection.Map$; +import scala.collection.immutable.HashMap; + +@ExtendWith(SparkAgentTestExtension.class) +class CreateDataSourceTableCommandVisitorTest { + + @Test + void testCreateDataSourceTableCommand() { + SparkSession session = SparkSession.builder().master("local").getOrCreate(); + CreateDataSourceTableCommandVisitor visitor = new CreateDataSourceTableCommandVisitor(); + + CreateDataSourceTableCommand command = + new CreateDataSourceTableCommand( + SparkUtils.catalogTable( + TableIdentifier$.MODULE$.apply("tablename", Option.apply("db")), + CatalogTableType.EXTERNAL(), + CatalogStorageFormat$.MODULE$.apply( + Option.apply(URI.create("s3://bucket/directory")), + null, + null, + null, + false, + Map$.MODULE$.empty()), + new StructType( + new StructField[] { + new StructField( + "key", IntegerType$.MODULE$, false, new Metadata(new HashMap<>())), + new StructField( + "value", StringType$.MODULE$, false, new Metadata(new HashMap<>())) + })), + false); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", "directory") + .hasFieldOrPropertyWithValue("namespace", "s3://bucket"); + } +} diff --git a/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateHiveTableAsSelectCommandVisitorTest.java b/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateHiveTableAsSelectCommandVisitorTest.java new file mode 100644 index 0000000000000..8ccf6f312702c --- /dev/null +++ b/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateHiveTableAsSelectCommandVisitorTest.java @@ -0,0 +1,112 @@ +package io.openlineage.spark2.agent.lifecycle.plan; + +import static org.assertj.core.api.Assertions.assertThat; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.SparkAgentTestExtension; +import io.openlineage.spark.agent.lifecycle.plan.CreateHiveTableAsSelectCommandVisitor; +import io.openlineage.spark.agent.util.ScalaConversionUtils; +import java.net.URI; +import java.util.Arrays; +import java.util.List; +import org.apache.spark.Partition; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier$; +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat$; +import org.apache.spark.sql.catalyst.catalog.CatalogTableType; +import org.apache.spark.sql.catalyst.expressions.AttributeReference; +import org.apache.spark.sql.catalyst.expressions.ExprId; +import org.apache.spark.sql.execution.datasources.LogicalRelation; +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions; +import org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation; +import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand; +import org.apache.spark.sql.types.IntegerType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.postgresql.Driver; +import scala.Option; +import scala.Tuple2; +import scala.collection.Map$; +import scala.collection.Seq$; +import scala.collection.immutable.HashMap; + +@ExtendWith(SparkAgentTestExtension.class) +class CreateHiveTableAsSelectCommandVisitorTest { + + @Test + void testCreateHiveTableAsSelectCommand() { + SparkSession session = SparkSession.builder().master("local").getOrCreate(); + CreateHiveTableAsSelectCommandVisitor visitor = new CreateHiveTableAsSelectCommandVisitor(); + + CreateHiveTableAsSelectCommand command = + new CreateHiveTableAsSelectCommand( + SparkUtils.catalogTable( + TableIdentifier$.MODULE$.apply("tablename", Option.apply("db")), + CatalogTableType.EXTERNAL(), + CatalogStorageFormat$.MODULE$.apply( + Option.apply(URI.create("s3://bucket/directory")), + null, + null, + null, + false, + Map$.MODULE$.empty()), + new StructType( + new StructField[] { + new StructField( + "key", IntegerType$.MODULE$, false, new Metadata(new HashMap<>())), + new StructField( + "value", StringType$.MODULE$, false, new Metadata(new HashMap<>())) + })), + new LogicalRelation( + new JDBCRelation( + new StructType( + new StructField[] { + new StructField("key", IntegerType$.MODULE$, false, null), + new StructField("value", StringType$.MODULE$, false, null) + }), + new Partition[] {}, + new JDBCOptions( + "", + "temp", + scala.collection.immutable.Map$.MODULE$ + .newBuilder() + .$plus$eq(Tuple2.apply("driver", Driver.class.getName())) + .result()), + session), + Seq$.MODULE$ + .newBuilder() + .$plus$eq( + new AttributeReference( + "key", + IntegerType$.MODULE$, + false, + null, + ExprId.apply(1L), + Seq$.MODULE$.empty())) + .$plus$eq( + new AttributeReference( + "value", + StringType$.MODULE$, + false, + null, + ExprId.apply(2L), + Seq$.MODULE$.empty())) + .result(), + Option.empty(), + false), + ScalaConversionUtils.fromList(Arrays.asList("key", "value")), + SaveMode.Overwrite); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", "directory") + .hasFieldOrPropertyWithValue("namespace", "s3://bucket"); + } +} diff --git a/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateTableLikeCommandVisitorTest.java b/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateTableLikeCommandVisitorTest.java new file mode 100644 index 0000000000000..f1eee57d1cb74 --- /dev/null +++ b/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/CreateTableLikeCommandVisitorTest.java @@ -0,0 +1,58 @@ +package io.openlineage.spark2.agent.lifecycle.plan; + +import static org.assertj.core.api.Assertions.assertThat; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.SparkAgentTestExtension; +import java.util.List; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier$; +import org.apache.spark.sql.execution.command.CreateTableLikeCommand; +import org.apache.spark.sql.types.IntegerType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import scala.Option; +import scala.collection.Map$; +import scala.collection.immutable.HashMap; + +@ExtendWith(SparkAgentTestExtension.class) +class CreateTableLikeCommandVisitorTest { + @Test + void testCreateTableLikeCommand() { + SparkSession session = + SparkSession.builder() + .config("spark.sql.warehouse.dir", "/tmp/warehouse") + .master("local") + .getOrCreate(); + String database = session.catalog().currentDatabase(); + + StructType schema = + new StructType( + new StructField[] { + new StructField("key", IntegerType$.MODULE$, false, new Metadata(new HashMap<>())), + new StructField("value", StringType$.MODULE$, false, new Metadata(new HashMap<>())) + }); + + session.catalog().createTable("table", "csv", schema, Map$.MODULE$.empty()); + + CreateTableLikeCommandVisitor visitor = new CreateTableLikeCommandVisitor(session); + + CreateTableLikeCommand command = + new CreateTableLikeCommand( + TableIdentifier$.MODULE$.apply("table", Option.apply(database)), + TableIdentifier$.MODULE$.apply("table", Option.apply(database)), + Option.apply("/path/to/data"), + false); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", "/path/to/data") + .hasFieldOrPropertyWithValue("namespace", "file"); + } +} diff --git a/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/OptimizedCreateHiveTableAsSelectCommandVisitorTest.java b/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/OptimizedCreateHiveTableAsSelectCommandVisitorTest.java new file mode 100644 index 0000000000000..15fea339d3304 --- /dev/null +++ b/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/OptimizedCreateHiveTableAsSelectCommandVisitorTest.java @@ -0,0 +1,113 @@ +package io.openlineage.spark2.agent.lifecycle.plan; + +import static org.assertj.core.api.Assertions.assertThat; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.SparkAgentTestExtension; +import io.openlineage.spark.agent.lifecycle.plan.OptimizedCreateHiveTableAsSelectCommandVisitor; +import io.openlineage.spark.agent.util.ScalaConversionUtils; +import java.net.URI; +import java.util.Arrays; +import java.util.List; +import org.apache.spark.Partition; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier$; +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat$; +import org.apache.spark.sql.catalyst.catalog.CatalogTableType; +import org.apache.spark.sql.catalyst.expressions.AttributeReference; +import org.apache.spark.sql.catalyst.expressions.ExprId; +import org.apache.spark.sql.execution.datasources.LogicalRelation; +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions; +import org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation; +import org.apache.spark.sql.hive.execution.OptimizedCreateHiveTableAsSelectCommand; +import org.apache.spark.sql.types.IntegerType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.postgresql.Driver; +import scala.Option; +import scala.Tuple2; +import scala.collection.Map$; +import scala.collection.Seq$; +import scala.collection.immutable.HashMap; + +@ExtendWith(SparkAgentTestExtension.class) +class OptimizedCreateHiveTableAsSelectCommandVisitorTest { + + @Test + void testOptimizedCreateHiveTableAsSelectCommand() { + SparkSession session = SparkSession.builder().master("local").getOrCreate(); + OptimizedCreateHiveTableAsSelectCommandVisitor visitor = + new OptimizedCreateHiveTableAsSelectCommandVisitor(); + + OptimizedCreateHiveTableAsSelectCommand command = + new OptimizedCreateHiveTableAsSelectCommand( + SparkUtils.catalogTable( + TableIdentifier$.MODULE$.apply("tablename", Option.apply("db")), + CatalogTableType.EXTERNAL(), + CatalogStorageFormat$.MODULE$.apply( + Option.apply(URI.create("s3://bucket/directory")), + null, + null, + null, + false, + Map$.MODULE$.empty()), + new StructType( + new StructField[] { + new StructField( + "key", IntegerType$.MODULE$, false, new Metadata(new HashMap<>())), + new StructField( + "value", StringType$.MODULE$, false, new Metadata(new HashMap<>())) + })), + new LogicalRelation( + new JDBCRelation( + new StructType( + new StructField[] { + new StructField("key", IntegerType$.MODULE$, false, null), + new StructField("value", StringType$.MODULE$, false, null) + }), + new Partition[] {}, + new JDBCOptions( + "", + "temp", + scala.collection.immutable.Map$.MODULE$ + .newBuilder() + .$plus$eq(Tuple2.apply("driver", Driver.class.getName())) + .result()), + session), + Seq$.MODULE$ + .newBuilder() + .$plus$eq( + new AttributeReference( + "key", + IntegerType$.MODULE$, + false, + null, + ExprId.apply(1L), + Seq$.MODULE$.empty())) + .$plus$eq( + new AttributeReference( + "value", + StringType$.MODULE$, + false, + null, + ExprId.apply(2L), + Seq$.MODULE$.empty())) + .result(), + Option.empty(), + false), + ScalaConversionUtils.fromList(Arrays.asList("key", "value")), + SaveMode.Overwrite); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", "directory") + .hasFieldOrPropertyWithValue("namespace", "s3://bucket"); + } +} diff --git a/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/SparkUtils.java b/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/SparkUtils.java new file mode 100644 index 0000000000000..f9ea61422dcc8 --- /dev/null +++ b/integration/spark/src/test/spark2/java/io/openlineage/spark2/agent/lifecycle/plan/SparkUtils.java @@ -0,0 +1,40 @@ +package io.openlineage.spark2.agent.lifecycle.plan; + +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.catalog.CatalogTable$; +import org.apache.spark.sql.catalyst.catalog.CatalogTableType; +import org.apache.spark.sql.types.StructType; +import scala.collection.Map$; +import scala.collection.Seq$; + +public class SparkUtils { + // Can't use Scala's default parameters from Java. + public static CatalogTable catalogTable( + TableIdentifier identifier, + CatalogTableType tableType, + CatalogStorageFormat storageFormat, + StructType schema) { + return CatalogTable$.MODULE$.apply( + identifier, + tableType, + storageFormat, + schema, + null, + Seq$.MODULE$.empty(), + null, + "", + System.currentTimeMillis(), + -1L, + "", + Map$.MODULE$.empty(), + null, + null, + null, + Seq$.MODULE$.empty(), + false, + true, + Map$.MODULE$.empty()); + } +} diff --git a/integration/spark/src/test/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/CreateTableAsSelectVisitorTest.java b/integration/spark/src/test/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/CreateTableAsSelectVisitorTest.java new file mode 100644 index 0000000000000..8d6f5f27bc104 --- /dev/null +++ b/integration/spark/src/test/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/CreateTableAsSelectVisitorTest.java @@ -0,0 +1,125 @@ +package io.openlineage.spark3.agent.lifecycle.plan; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.SparkAgentTestExtension; +import java.util.List; +import org.apache.commons.lang.reflect.FieldUtils; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.plans.logical.CreateTableAsSelect; +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation$; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions; +import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog; +import org.apache.spark.sql.types.IntegerType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; +import scala.collection.Map$; +import scala.collection.Seq$; +import scala.collection.immutable.HashMap; + +@ExtendWith(SparkAgentTestExtension.class) +class CreateTableAsSelectVisitorTest { + + private CreateTableAsSelect createCTAS(TableCatalog tableCatalog) { + return new CreateTableAsSelect( + tableCatalog, + new Identifier() { + @Override + public String[] namespace() { + return new String[] {"database", "schema"}; + } + + @Override + public String name() { + return "table"; + } + + @Override + public String toString() { + return "database.schema.table"; + } + }, + Seq$.MODULE$.empty(), + LocalRelation$.MODULE$.apply( + new StructField("key", IntegerType$.MODULE$, false, new Metadata(new HashMap<>())), + Seq$.MODULE$.empty()), + Map$.MODULE$.empty(), + Map$.MODULE$.empty(), + false); + } + + @Test + void testCreateTableAsSelectJdbcCommand() throws IllegalAccessException { + SparkSession session = SparkSession.builder().master("local").getOrCreate(); + CreateTableAsSelectVisitor visitor = new CreateTableAsSelectVisitor(session); + + JDBCTableCatalog tableCatalog = new JDBCTableCatalog(); + JDBCOptions options = mock(JDBCOptions.class); + when(options.url()).thenReturn("jdbc:postgresql://postgreshost:5432"); + FieldUtils.writeField(tableCatalog, "options", options, true); + + CreateTableAsSelect command = createCTAS(tableCatalog); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", "database.schema.table") + .hasFieldOrPropertyWithValue("namespace", "postgresql://postgreshost:5432"); + } + + @ParameterizedTest + @CsvSource({ + "hdfs://namenode:8020/warehouse,hdfs://namenode:8020,/warehouse/database.schema.table", + "/tmp/warehouse,file,/tmp/warehouse/database.schema.table" + }) + void testCreateTableAsSelectIcebergHadoopCommand( + String warehouseConf, String namespace, String name) throws IllegalAccessException { + SparkSession session = SparkSession.builder().master("local").getOrCreate(); + session.conf().set("spark.sql.catalog.test.type", "hadoop"); + session.conf().set("spark.sql.catalog.test.warehouse", warehouseConf); + + CreateTableAsSelectVisitor visitor = new CreateTableAsSelectVisitor(session); + SparkCatalog sparkCatalog = mock(SparkCatalog.class); + when(sparkCatalog.name()).thenReturn("test"); + + CreateTableAsSelect command = createCTAS(sparkCatalog); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", name) + .hasFieldOrPropertyWithValue("namespace", namespace); + } + + @Test + void testCreateTableAsSelectIcebergHiveCommand() throws IllegalAccessException { + SparkSession session = SparkSession.builder().master("local").getOrCreate(); + session.conf().set("spark.sql.catalog.test.type", "hive"); + session.conf().set("spark.sql.catalog.test.uri", "thrift://metastore-host:10001"); + + CreateTableAsSelectVisitor visitor = new CreateTableAsSelectVisitor(session); + SparkCatalog sparkCatalog = mock(SparkCatalog.class); + when(sparkCatalog.name()).thenReturn("test"); + + CreateTableAsSelect command = createCTAS(sparkCatalog); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", "database.schema.table") + .hasFieldOrPropertyWithValue("namespace", "hive://metastore-host:10001"); + } +} diff --git a/integration/spark/src/test/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/CreateTableLikeCommandVisitorTest.java b/integration/spark/src/test/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/CreateTableLikeCommandVisitorTest.java new file mode 100644 index 0000000000000..8bc7f88337852 --- /dev/null +++ b/integration/spark/src/test/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/CreateTableLikeCommandVisitorTest.java @@ -0,0 +1,62 @@ +package io.openlineage.spark3.agent.lifecycle.plan; + +import static org.assertj.core.api.Assertions.assertThat; + +import io.openlineage.client.OpenLineage; +import io.openlineage.spark.agent.SparkAgentTestExtension; +import java.util.List; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier$; +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat; +import org.apache.spark.sql.execution.command.CreateTableLikeCommand; +import org.apache.spark.sql.types.IntegerType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import scala.Option; +import scala.Option$; +import scala.collection.Map$; +import scala.collection.immutable.HashMap; + +@ExtendWith(SparkAgentTestExtension.class) +class CreateTableLikeCommandVisitorTest { + @Test + void testCreateTableLikeCommand() { + SparkSession session = + SparkSession.builder() + .config("spark.sql.warehouse.dir", "/tmp/warehouse") + .master("local") + .getOrCreate(); + String database = session.catalog().currentDatabase(); + + StructType schema = + new StructType( + new StructField[] { + new StructField("key", IntegerType$.MODULE$, false, new Metadata(new HashMap<>())), + new StructField("value", StringType$.MODULE$, false, new Metadata(new HashMap<>())) + }); + + session.catalog().createTable("table", "csv", schema, Map$.MODULE$.empty()); + + CreateTableLikeCommandVisitor visitor = new CreateTableLikeCommandVisitor(session); + + CreateTableLikeCommand command = + new CreateTableLikeCommand( + TableIdentifier$.MODULE$.apply("newtable", Option.apply(database)), + TableIdentifier$.MODULE$.apply("table", Option.apply(database)), + CatalogStorageFormat.empty(), + Option$.MODULE$.empty(), + Map$.MODULE$.empty(), + false); + + assertThat(visitor.isDefinedAt(command)).isTrue(); + List datasets = visitor.apply(command); + assertThat(datasets) + .singleElement() + .hasFieldOrPropertyWithValue("name", "/tmp/warehouse/newtable") + .hasFieldOrPropertyWithValue("namespace", "file"); + } +} diff --git a/integration/spark/src/test/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/DataSourceV2RelationVisitorTest.java b/integration/spark/src/test/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/DataSourceV2RelationVisitorTest.java index 4f13fbed72e53..a4aeb2e30db3a 100644 --- a/integration/spark/src/test/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/DataSourceV2RelationVisitorTest.java +++ b/integration/spark/src/test/spark3/java/io/openlineage/spark3/agent/lifecycle/plan/DataSourceV2RelationVisitorTest.java @@ -9,7 +9,8 @@ import org.apache.spark.sql.connector.catalog.Table; import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation; import org.apache.spark.sql.types.StructType; -import org.junit.Assert; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -20,13 +21,29 @@ public class DataSourceV2RelationVisitorTest { Table table = Mockito.mock(Table.class); Map tableProperties = new HashMap<>(); + @AfterEach + public void resetMock() { + Mockito.reset(dataSourceV2Relation); + Mockito.reset(table); + } + @Test public void testApplyExceptionIsThrownWhenNonSupportedProvider() { Exception exception = assertThrows( RuntimeException.class, () -> dataSourceV2RelationVisitor.apply(dataSourceV2Relation)); - Assert.assertTrue(exception.getMessage().startsWith("Couldn't find DatasetSource in plan")); + Assertions.assertTrue( + exception.getMessage().startsWith("Couldn't find provider for dataset in plan")); + } + + @Test + public void testIsDefinedAtFailsWhenProviderUnknown() { + tableProperties.put("provider", "unsupported/provider"); + Mockito.when((dataSourceV2Relation).table()).thenReturn(table); + Mockito.when(table.properties()).thenReturn(tableProperties); + + Assertions.assertFalse(dataSourceV2RelationVisitor.isDefinedAt(dataSourceV2Relation)); } @Test @@ -45,16 +62,17 @@ public void testApplyForIcebergOnGS() { TableProviderFacet tableProviderFacet = (TableProviderFacet) dataset.getFacets().getAdditionalProperties().get("table_provider"); - Assert.assertEquals("parquet", tableProviderFacet.getFormat()); - Assert.assertEquals("iceberg", tableProviderFacet.getProvider()); - Assert.assertEquals("gs://bucket/catalog/db/table", dataset.getNamespace()); - Assert.assertEquals("remote-gcs.db.table", dataset.getName()); + Assertions.assertEquals("parquet", tableProviderFacet.getFormat()); + Assertions.assertEquals("iceberg", tableProviderFacet.getProvider()); + Assertions.assertEquals("gs://bucket/catalog/db/table", dataset.getNamespace()); + Assertions.assertEquals("remote-gcs.db.table", dataset.getName()); } @Test public void testApplyForIcebergOnLocal() { tableProperties.put("provider", "iceberg"); tableProperties.put("location", "/tmp/catalog/db/table"); + tableProperties.put("format", "iceberg/parquet"); Mockito.when(table.properties()).thenReturn(tableProperties); Mockito.when((dataSourceV2Relation).table()).thenReturn(table); @@ -66,14 +84,14 @@ public void testApplyForIcebergOnLocal() { TableProviderFacet tableProviderFacet = (TableProviderFacet) dataset.getFacets().getAdditionalProperties().get("table_provider"); - Assert.assertEquals("file:///tmp/catalog/db/table", dataset.getNamespace()); - Assert.assertEquals("local.db.table", dataset.getName()); + Assertions.assertEquals("file:///tmp/catalog/db/table", dataset.getNamespace()); + Assertions.assertEquals("local.db.table", dataset.getName()); } @Test public void testIsDefinedAtForNonDefinedProvider() { Mockito.when(dataSourceV2Relation.table()).thenReturn(table); - Assert.assertFalse(dataSourceV2RelationVisitor.isDefinedAt(dataSourceV2Relation)); + Assertions.assertFalse(dataSourceV2RelationVisitor.isDefinedAt(dataSourceV2Relation)); } @Test @@ -81,6 +99,31 @@ public void testIsDefinedAtForIceberg() { tableProperties.put("provider", "iceberg"); Mockito.when((dataSourceV2Relation).table()).thenReturn(table); Mockito.when(table.properties()).thenReturn(tableProperties); - Assert.assertTrue(dataSourceV2RelationVisitor.isDefinedAt(dataSourceV2Relation)); + Assertions.assertTrue(dataSourceV2RelationVisitor.isDefinedAt(dataSourceV2Relation)); + } + + @Test + public void testIsDefinedForDelta() { + tableProperties.put("provider", "delta"); + Mockito.when((dataSourceV2Relation).table()).thenReturn(table); + Mockito.when(table.properties()).thenReturn(tableProperties); + Assertions.assertTrue(dataSourceV2RelationVisitor.isDefinedAt(dataSourceV2Relation)); + } + + @Test + public void testApplyDeltaLocal() { + tableProperties.put("provider", "delta"); + tableProperties.put("location", "file:/tmp/delta/spark-warehouse/tbl"); + tableProperties.put("format", "parquet"); + + Mockito.when(table.properties()).thenReturn(tableProperties); + Mockito.when((dataSourceV2Relation).table()).thenReturn(table); + Mockito.when(dataSourceV2Relation.schema()).thenReturn(new StructType()); + Mockito.when(table.name()).thenReturn("table"); + + OpenLineage.Dataset dataset = dataSourceV2RelationVisitor.apply(dataSourceV2Relation).get(0); + + Assertions.assertEquals("file:/tmp/delta/spark-warehouse/tbl", dataset.getNamespace()); + Assertions.assertEquals("table", dataset.getName()); } }