Skip to main content
Version: Next

Dataset Metrics

Input and output facets in OpenLineage specification describe datasets in the context of a given run. For example, an amount of rows read is not a dataset facet as it does not describe the dataset. For the convenience, OpenLineage events contain this information under inputFacets and outputFacets fields of input and output datasets respectively.

Standard Input / Output dataset statistics

OpenLineage specification comes with:

which are collected by the Spark integration. Those facets basically contain:

  • amount rows read/written,
  • amount of bytes read/written,
  • amount of files read/written.

As a limitation to this, a row count for input datasets is collected only for DataSourceV2 api datasets.

Iceberg specific metrics reports

Even more extensive metrics are collected for Iceberg tables, as the library exposes MetricReport API. Two report types are currently supported:

  • ScanReport - carries metrics being collected during scan planning against a given table. Amongst some general information about the involved table, such as the snapshot id or the table name, it includes metrics like:
    • total scan planning duration
    • number of data/delete files included in the result
    • number of data/delete manifests scanned/skipped
    • number of data/delete files scanned/skipped
    • number of equality/positional delete files scanned
  • CommitReport - carries metrics being collected after committing changes to a table (aka producing a snapshot). Amongst some general information about the involved table, such as the snapshot id or the table name, it includes metrics like:
    • total duration
    • number of attempts required for the commit to succeed
    • number of added/removed data/delete files
    • number of added/removed equality/positional delete files
    • number of added/removed equality/positional deletes

At the bottom of the page, we list example facets generated by Spark integration.

This feature is delivered by implementing custom OpenLineageMetricsReporter class as Iceberg metrics reporter and injecting it automatically into Iceberg catalog. If any other custom reporter is present, OpenLineageMetricsReporter will overwrite it, but it will still report metrics to it.

In case of any issues, a spark config flag: spark.openlineage.vendors.iceberg.metricsReporterDisabled=true can be used to disable this feature.

"icebergScanReport": {
"_producer":"https://github.com/OpenLineage/OpenLineage/tree/1.26.0-SNAPSHOT/integration/spark",
"_schemaURL":"https://openlineage.io/spec/facets/1-0-0/IcebergScanReportInputDatasetFacet.json",
"snapshotId":4115428054613373118,
"filterDescription":"",
"projectedFieldNames":[
"a",
"b"
],
"scanMetrics":{
"totalPlanningDuration":21,
"resultDataFiles":1,
"resultDeleteFiles":0,
"totalDataManifests":1,
"totalDeleteManifests":0,
"scannedDataManifests":1,
"skippedDataManifests":0,
"totalFileSizeInBytes":676,
"totalDeleteFileSizeInBytes":0,
"skippedDataFiles":0,
"skippedDeleteFiles":0,
"scannedDeleteManifests":0,
"skippedDeleteManifests":0,
"indexedDeleteFiles":0,
"equalityDeleteFiles":0,
"positionalDeleteFiles":0
},
"metadata":{
"engine-version":"3.3.4",
"iceberg-version":"Apache Iceberg 1.6.0 (commit 229d8f6fcd109e6c8943ea7cbb41dab746c6d0ed)",
"app-id":"local-1733228790932",
"engine-name":"spark"
}
}
"icebergCommitReport": {
"snapshotId":3131594900391425696,
"sequenceNumber":2,
"operation":"append",
"commitMetrics":{
"totalDuration":87,
"attempts":1,
"addedDataFiles":1,
"totalDataFiles":2,
"totalDeleteFiles":0,
"addedRecords":1,
"totalRecords":4,
"addedFilesSizeInBytes":651,
"totalFilesSizeInBytes":1343,
"totalPositionalDeletes":0,
"totalEqualityDeletes":0
},
"metadata":{
"engine-version":"3.3.4",
"app-id":"local-1733228862465",
"engine-name":"spark",
"iceberg-version":"Apache Iceberg 1.6.0 (commit 229d8f6fcd109e6c8943ea7cbb41dab746c6d0ed)"
}
}