Apache Flink
Apache Flink is one of the most popular stream processing frameworks. Apache Flink jobs run on clusters,
which are composed of two types of nodes: TaskManagers
and JobManagers
. While clusters typically consists of
multiple TaskManagers
, only reason to run multiple JobManagers is high availability. The jobs are submitted
to JobManager
by JobClient
, that compiles user application into dataflow graph which is understandable by JobManager
.
JobManager
then coordinates job execution: it splits the parallel units of a job
to TaskManagers
, manages heartbeats, triggers checkpoints, reacts to failures and much more.
Apache Flink has multiple deployment modes - Session Mode, Application Mode and Per-Job mode. The most popular
are Session Mode and Application Mode. Session Mode consists of a JobManager
managing multiple jobs sharing single
Flink cluster. In this mode, JobClient
is executed on a machine that submits the job to the cluster.
Application Mode is used where cluster is utilized for a single job. In this mode, JobClient
, where the main method runs,
is executed on the JobManager
.
Flink jobs read data from Sources
and write data to Sinks
. In contrast to systems like Apache Spark, Flink jobs can write
data to multiple places - they can have multiple Sinks
.
Getting lineage from Flink
OpenLineage utilizes Flink's JobListener
interface. This interface is used by Flink to notify user of job submission,
successful finish of job, or job failure. Implementations of this interface are executed on JobClient
.
When OpenLineage listener receives information that job was submitted, it extracts Transformations
from job's
ExecutionEnvironment
. The Transformations
represent logical operations in the dataflow graph; they are composed
of both Flink's built-in operators, but also user-provided Sources
, Sinks
and functions. To get the lineage,
OpenLineage integration processes dataflow graph. Currently, OpenLineage is interested only in information contained
in Sources
and Sinks
, as they are the places where Flink interacts with external systems.
After job submission, OpenLineage integration starts actively listening to checkpoints - this gives insight into whether the job runs properly.
Limitations
Currently, OpenLineage's Flink integration is limited to getting information from jobs running in Application Mode.
OpenLineage integration extracts lineage only from following Sources
and Sinks
:
Sources | Sinks |
---|---|
KafkaSource | KafkaSink (1) |
FlinkKafkaConsumer | FlinkKafkaProducer |
IcebergFlinkSource | IcebergFlinkSink |
We expect this list to grow as we add support for more connectors.
(1) KafkaSink supports sinks that write to a single topic as well as multi topic sinks. The
limitation for multi topic sink is that: topics need to have the same schema and implementation
of KafkaRecordSerializationSchema
must extend KafkaTopicsDescriptor
.
Methods isFixedTopics
and getFixedTopics
from KafkaTopicsDescriptor
are used to extract multiple topics
from a sink.
Usage
In your job, you need to set up OpenLineageFlinkJobListener
.
For example:
JobListener listener = OpenLineageFlinkJobListener.builder()
.executionEnvironment(streamExecutionEnvironment)
.build();
streamExecutionEnvironment.registerJobListener(listener);
Also, OpenLineage needs certain parameters to be set in flink-conf.yaml
:
Configuration Key | Description | Expected Value | Default |
---|---|---|---|
execution.attached | This setting needs to be true if OpenLineage is to detect job start and failure | true | false |
OpenLineage jar needs to be present on JobManager
.
When the JobListener
is configured, you need to point the OpenLineage integration where the events should end up.
If you're using Marquez
, simplest way to do that is to set up OPENLINEAGE_URL
environment
variable to Marquez
URL. More advanced settings are in the client documentation..
Configuring Openlineage connector
Flink Openlineage connector utilizes standard Java client for Openlineage and allows all the configuration features present there to be used. The configuration can be passed with:
openlineage.yml
file with a environment propertyOPENLINEAGE_CONFIG
being set and pointing to configuration file. File structure and allowed options are described here.- Standard Flink configuration with the parameters defined below.
Flink Configuration parameters
The following parameters can be specified:
Parameter | Definition | Example |
---|---|---|
openlineage.transport.type | The transport type used for event emit, default type is console | http |
openlineage.facets.disabled | List of facets to disable, enclosed in [] (required from 0.21.x) and separated by ; , default is [spark_unknown;spark.logicalPlan;] (currently must contain ; ) | [some_facet1;some_facet1] |
openlineage.job.owners.<ownership-type> | Specifies ownership of the job. Multiple entries with different types are allowed. Config key name and value are used to create job ownership type and name (available since 1.13). | openlineage.job.owners.team="Some Team" |
Transports
Tip: See current list of all supported transports.
HTTP
Allows sending events to HTTP endpoint, using ApacheHTTPClient.
Configuration
type
- string, must be"http"
. Required.url
- string, base url for HTTP requests. Required.endpoint
- string specifying the endpoint to which events are sent, appended tourl
. Optional, default:/api/v1/lineage
.urlParams
- dictionary specifying query parameters send in HTTP requests. Optional.timeoutInMillis
- integer specifying timeout (in milliseconds) value used while connecting to server. Optional, default:5000
.auth
- dictionary specifying authentication options. Optional, by default no authorization is used. If set, requires thetype
property.type
- string specifying the "api_key" or the fully qualified class name of your TokenProvider. Required ifauth
is provided.apiKey
- string setting the Authentication HTTP header as the Bearer. Required iftype
isapi_key
.
headers
- dictionary specifying HTTP request headers. Optional.compression
- string, name of algorithm used by HTTP client to compress request body. Optional, default valuenull
, allowed values:gzip
. Added in v1.13.0.
Behavior
Events are serialized to JSON, and then are send as HTTP POST request with Content-Type: application/json
.
Examples
- Yaml Config
- Spark Config
- Flink Config
- Java Code
Anonymous connection:
transport:
type: http
url: http://localhost:5000
With authorization:
transport:
type: http
url: http://localhost:5000
auth:
type: api_key
api_key: f38d2189-c603-4b46-bdea-e573a3b5a7d5
Full example:
transport:
type: http
url: http://localhost:5000
endpoint: /api/v1/lineage
urlParams:
param0: value0
param1: value1
timeoutInMillis: 5000
auth:
type: api_key
api_key: f38d2189-c603-4b46-bdea-e573a3b5a7d5
headers:
X-Some-Extra-Header: abc
compression: gzip
Anonymous connection:
spark.openlineage.transport.type=http
spark.openlineage.transport.url=http://localhost:5000
With authorization:
spark.openlineage.transport.type=http
spark.openlineage.transport.url=http://localhost:5000
spark.openlineage.transport.auth.type=api_key
spark.openlineage.transport.auth.apiKey=f38d2189-c603-4b46-bdea-e573a3b5a7d5
Full example:
spark.openlineage.transport.type=http
spark.openlineage.transport.url=http://localhost:5000
spark.openlineage.transport.endpoint=/api/v1/lineage
spark.openlineage.transport.urlParams.param0=value0
spark.openlineage.transport.urlParams.param1=value1
spark.openlineage.transport.timeoutInMillis=5000
spark.openlineage.transport.auth.type=api_key
spark.openlineage.transport.auth.apiKey=f38d2189-c603-4b46-bdea-e573a3b5a7d5
spark.openlineage.transport.headers.X-Some-Extra-Header=abc
spark.openlineage.transport.compression=gzip
URL parsing within Spark integration
You can supply http parameters using values in url, the parsed spark.openlineage.*
properties are located in url as follows:
{transport.url}/{transport.endpoint}/namespaces/{namespace}/jobs/{parentJobName}/runs/{parentRunId}?app_name={appName}&api_key={transport.apiKey}&timeout={transport.timeout}&xxx={transport.urlParams.xxx}
example:
http://localhost:5000/api/v1/namespaces/ns_name/jobs/job_name/runs/xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx?app_name=app&api_key=abc&timeout=5000&xxx=xxx
Anonymous connection:
spark.openlineage.transport.type=http
spark.openlineage.transport.url=http://localhost:5000
With authorization:
openlineage.transport.type=http
openlineage.transport.url=http://localhost:5000
openlineage.transport.auth.type=api_key
openlineage.transport.auth.apiKey=f38d2189-c603-4b46-bdea-e573a3b5a7d5
Full example:
openlineage.transport.type=http
openlineage.transport.url=http://localhost:5000
openlineage.transport.endpoint=/api/v1/lineage
openlineage.transport.urlParams.param0=value0
openlineage.transport.urlParams.param1=value1
openlineage.transport.timeoutInMillis=5000
openlineage.transport.auth.type=api_key
openlineage.transport.auth.apiKey=f38d2189-c603-4b46-bdea-e573a3b5a7d5
openlineage.transport.headers.X-Some-Extra-Header=abc
openlineage.transport.compression=gzip
Anonymous connection:
import io.openlineage.client.OpenLineageClient;
import io.openlineage.client.transports.HttpConfig;
import io.openlineage.client.transports.HttpTransport;
HttpConfig httpConfig = new HttpConfig();
httpConfig.setUrl("http://localhost:5000");
OpenLineageClient client = OpenLineageClient.builder()
.transport(
new HttpTransport(httpConfig))
.build();
With authorization:
import io.openlineage.client.OpenLineageClient;
import io.openlineage.client.transports.ApiKeyTokenProvider;
import io.openlineage.client.transports.HttpConfig;
import io.openlineage.client.transports.HttpTransport;
ApiKeyTokenProvider apiKeyTokenProvider = new ApiKeyTokenProvider();
apiKeyTokenProvider.setApiKey("f38d2189-c603-4b46-bdea-e573a3b5a7d5");
HttpConfig httpConfig = new HttpConfig();
httpConfig.setUrl("http://localhost:5000");
httpConfig.setAuth(apiKeyTokenProvider);
OpenLineageClient client = OpenLineageClient.builder()
.transport(
new HttpTransport(httpConfig))
.build();
Full example:
import java.util.Map;
import io.openlineage.client.OpenLineageClient;
import io.openlineage.client.transports.ApiKeyTokenProvider;
import io.openlineage.client.transports.HttpConfig;
import io.openlineage.client.transports.HttpTransport;
Map<String, String> queryParams = Map.of(
"param0", "value0",
"param1", "value1"
);
Map<String, String> headers = Map.of(
"X-Some-Extra-Header", "abc"
);
ApiKeyTokenProvider apiKeyTokenProvider = new ApiKeyTokenProvider();
apiKeyTokenProvider.setApiKey("f38d2189-c603-4b46-bdea-e573a3b5a7d5");
HttpConfig httpConfig = new HttpConfig();
httpConfig.setUrl("http://localhost:5000");
httpConfig.setEndpoint("/api/v1/lineage");
httpConfig.setUrlParams(queryParams);
httpConfig.setAuth(apiKeyTokenProvider);
httpConfig.setTimeoutInMillis(headers);
httpConfig.setHeaders(5000);
httpConfig.setCompression(HttpConfig.Compression.GZIP);
OpenLineageClient client = OpenLineageClient.builder()
.transport(
new HttpTransport(httpConfig))
.build();
Kafka
If a transport type is set to kafka
, then the below parameters would be read and used when building KafkaProducer.
This transport requires the artifact org.apache.kafka:kafka-clients:3.1.0
(or compatible) on your classpath.
Configuration
-
type
- string, must be"kafka"
. Required. -
topicName
- string specifying the topic on what events will be sent. Required. -
properties
- a dictionary containing a Kafka producer config as in Kafka producer config. Required. -
localServerId
- deprecated, renamed tomessageKey
since v1.13.0. -
messageKey
- string, key for all Kafka messages produced by transport. Optional, default value described below. Added in v1.13.0.Default values for
messageKey
are:run:{parentJob.namespace}/{parentJob.name}
- for RunEvent with parent facetrun:{job.namespace}/{job.name}
- for RunEventjob:{job.namespace}/{job.name}
- for JobEventdataset:{dataset.namespace}/{dataset.name}
- for DatasetEvent
Behavior
Events are serialized to JSON, and then dispatched to the Kafka topic.
Notes
It is recommended to provide messageKey
if Job hierarchy is used. It can be any string, but it should be the same for all jobs in
hierarchy, like Airflow task -> Spark application -> Spark task runs
.
Examples
- Yaml Config
- Spark Config
- Flink Config
- Java Code
transport:
type: kafka
topicName: openlineage.events
properties:
bootstrap.servers: localhost:9092,another.host:9092
acks: all
retries: 3
key.serializer: org.apache.kafka.common.serialization.StringSerializer
value.serializer: org.apache.kafka.common.serialization.StringSerializer
messageKey: some-value
spark.openlineage.transport.type=kafka
spark.openlineage.transport.topicName=openlineage.events
spark.openlineage.transport.properties.bootstrap.servers=localhost:9092,another.host:9092
spark.openlineage.transport.properties.acks=all
spark.openlineage.transport.properties.retries=3
spark.openlineage.transport.properties.key.serializer=org.apache.kafka.common.serialization.StringSerializer
spark.openlineage.transport.properties.value.serializer=org.apache.kafka.common.serialization.StringSerializer
spark.openlineage.transport.messageKey=some-value
openlineage.transport.type=kafka
openlineage.transport.topicName=openlineage.events
openlineage.transport.properties.bootstrap.servers=localhost:9092,another.host:9092
openlineage.transport.properties.acks=all
openlineage.transport.properties.retries=3
openlineage.transport.properties.key.serializer=org.apache.kafka.common.serialization.StringSerializer
openlineage.transport.properties.value.serializer=org.apache.kafka.common.serialization.StringSerializer
openlineage.transport.messageKey=some-value
import java.util.Properties;
import io.openlineage.client.OpenLineageClient;
import io.openlineage.client.transports.KafkaConfig;
import io.openlineage.client.transports.KafkaTransport;
Properties kafkaProperties = new Properties();
kafkaProperties.setProperty("bootstrap.servers", "localhost:9092,another.host:9092");
kafkaProperties.setProperty("acks", "all");
kafkaProperties.setProperty("retries", "3");
kafkaProperties.setProperty("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
kafkaProperties.setProperty("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
KafkaConfig kafkaConfig = new KafkaConfig();
KafkaConfig.setTopicName("openlineage.events");
KafkaConfig.setProperties(kafkaProperties);
KafkaConfig.setLocalServerId("some-value");
OpenLineageClient client = OpenLineageClient.builder()
.transport(
new KafkaTransport(httpConfig))
.build();
Notes:
It is recommended to provide messageKey
if Job hierarchy is used. It can be any string, but it should be the same for all jobs in
hierarchy, like Airflow task -> Spark application
.
Default values are:
run:{parentJob.namespace}/{parentJob.name}/{parentRun.id}
- for RunEvent with parent facetrun:{job.namespace}/{job.name}/{run.id}
- for RunEventjob:{job.namespace}/{job.name}
- for JobEventdataset:{dataset.namespace}/{dataset.name}
- for DatasetEvent
Console
This straightforward transport emits OpenLineage events directly to the console through a logger. No additional configuration is required.
Behavior
Events are serialized to JSON. Then each event is logged with INFO
level to logger with name ConsoleTransport
.
Notes
Be cautious when using the DEBUG
log level, as it might result in double-logging due to the OpenLineageClient
also logging.
Configuration
type
- string, must be"console"
. Required.
Examples
- Yaml Config
- Spark Config
- Flink Config
- Java Code
transport:
type: console
spark.openlineage.transport.type=console
openlineage.transport.type=console
import java.util.Properties;
import io.openlineage.client.OpenLineageClient;
import io.openlineage.client.transports.ConsoleTransport;
OpenLineageClient client = OpenLineageClient.builder()
.transport(
new ConsoleTransport())
.build();
File
Designed mainly for integration testing, the FileTransport
emits OpenLineage events to a given file.
Configuration
type
- string, must be"file"
. Required.location
- string specifying the path of the file. Required.
Behavior
- If the target file is absent, it's created.
- Events are serialized to JSON, and then appended to a file, separated by newlines.
- Intrinsic newline characters within the event JSON are eliminated to ensure one-line events.
Notes for Yarn/Kubernetes
This transport type is pretty useless on Spark/Flink applications deployed to Yarn or Kubernetes cluster:
- Each executor will write file to a local filesystem of Yarn container/K8s pod. So resulting file will be removed when such container/pod is destroyed.
- Kubernetes persistent volumes are not destroyed after pod removal. But all the executors will write to the same network disk in parallel, producing a broken file.
Examples
- Yaml Config
- Spark Config
- Flink Config
- Java Code
transport:
type: file
location: /path/to/your/file
spark.openlineage.transport.type=file
spark.openlineage.transport.location=/path/to/your/filext
openlineage.transport.type=file
openlineage.transport.location=/path/to/your/file
import java.util.Properties;
import io.openlineage.client.OpenLineageClient;
import io.openlineage.client.transports.FileConfig;
import io.openlineage.client.transports.FileTransport;
FileConfig fileConfig = new FileConfig("/path/to/your/file");
OpenLineageClient client = OpenLineageClient.builder()
.transport(
new FileTransport(fileConfig))
.build();
Composite
The CompositeTransport
is designed to combine multiple transports, allowing event emission to several destinations. This is useful when events need to be sent to multiple targets, such as a logging system and an API endpoint. The events are delivered sequentially - one after another in a defined order.
Configuration
type
- string, must be "composite". Required.transports
- a list or a map of transport configurations. Required.continueOnFailure
- boolean flag, determines if the process should continue even when one of the transports fails. Default istrue
.withThreadPool
- boolean flag, determines if a thread pool for parallel event emission should be kept between event emissions. Default istrue
.
Behavior
- The configured transports will be initialized and used in sequence (sorted by transport name) to emit OpenLineage events.
- If
continueOnFailure
is set tofalse
, a failure in one transport will stop the event emission process, and an exception will be raised. - If
continueOnFailure
istrue
, the failure will be logged, but the remaining transports will still attempt to send the event.
Notes for Multiple Transports
The composite transport can be used with any OpenLineage transport (e.g. HttpTransport
, KafkaTransport
, etc).
Ideal for scenarios where OpenLineage events need to reach multiple destinations for redundancy or different types of processing.
The transports
configuration can be provided in two formats:
- A list of transport configurations, where each transport may optionally include a
name
field. - A map of transport configurations, where the key acts as the name for each transport. The map format is particularly useful for configurations set via environment variables or Java properties, providing a more convenient and flexible setup.
Why are transport names used?
Transport names are not required for basic functionality. Their primary purpose is to enable configuration of composite transports via environment variables, which is only supported when names are defined.
Examples
- Yaml Config (List)
- Yaml Config (Map)
- Spark Config
- Flink Config
- Java Code
transport:
type: composite
continueOnFailure: true
transports:
- type: http
url: http://example.com/api
name: my_http
- type: kafka
topicName: openlineage.events
properties:
bootstrap.servers: localhost:9092,another.host:9092
acks: all
retries: 3
key.serializer: org.apache.kafka.common.serialization.StringSerializer
value.serializer: org.apache.kafka.common.serialization.StringSerializer
messageKey: some-value
continueOnFailure: true
transport:
type: composite
continueOnFailure: true
transports:
my_http:
type: http
url: http://example.com/api
name: my_http
my_kafka:
type: kafka
topicName: openlineage.events
properties:
bootstrap.servers: localhost:9092,another.host:9092
acks: all
retries: 3
key.serializer: org.apache.kafka.common.serialization.StringSerializer
value.serializer: org.apache.kafka.common.serialization.StringSerializer
messageKey: some-value
continueOnFailure: true
spark.openlineage.transport.type=composite
spark.openlineage.transport.continueOnFailure=true
spark.openlineage.transport.transports.my_http.type=http
spark.openlineage.transport.transports.my_http.url=http://example.com/api
spark.openlineage.transport.transports.my_kafka.type=kafka
spark.openlineage.transport.transports.my_kafka.topicName=openlineage.events
spark.openlineage.transport.transports.my_kafka.properties.bootstrap.servers=localhost:9092,another.host:9092
spark.openlineage.transport.transports.my_kafka.properties.acks=all
spark.openlineage.transport.transports.my_kafka.properties.retries=3
spark.openlineage.transport.transports.my_kafka.properties.key.serializer=org.apache.kafka.common.serialization.StringSerializer
spark.openlineage.transport.transports.my_kafka.properties.value.serializer=org.apache.kafka.common.serialization.StringSerializer
openlineage.transport.type=composite
openlineage.transport.continueOnFailure=true
openlineage.transport.transports.my_http.type=http
openlineage.transport.transports.my_http.url=http://example.com/api
openlineage.transport.transports.my_kafka.type=kafka
openlineage.transport.transports.my_kafka.topicName=openlineage.events
openlineage.transport.transports.my_kafka.properties.bootstrap.servers=localhost:9092,another.host:9092
openlineage.transport.transports.my_kafka.properties.acks=all
openlineage.transport.transports.my_kafka.properties.retries=3
openlineage.transport.transports.my_kafka.properties.key.serializer=org.apache.kafka.common.serialization.StringSerializer
openlineage.transport.transports.my_kafka.properties.value.serializer=org.apache.kafka.common.serialization.StringSerializer
import java.util.Arrays;
import io.openlineage.client.OpenLineageClient;
import io.openlineage.client.transports.CompositeConfig;
import io.openlineage.client.transports.HttpConfig;
import io.openlineage.client.transports.HttpTransport;
import io.openlineage.client.transports.KafkaConfig;
import io.openlineage.client.transports.KafkaTransport;
HttpConfig httpConfig = new HttpConfig();
httpConfig.setUrl("http://example.com/api");
KafkaConfig kafkaConfig = new KafkaConfig();
KafkaConfig.setTopicName("openlineage.events");
KafkaConfig.setLocalServerId("some-value");
CompositeConfig compositeConfig = new CompositeConfig(Arrays.asList(
new HttpTransport(httpConfig),
new KafkaTransport(kafkaConfig)
), true);
OpenLineageClient client = OpenLineageClient.builder()
.transport(
new CompositeTransport(compositeConfig))
.build();
Transform
The TransformTransport
is designed to enable event manipulation before emitting the event. Together with CompositeTransport
, it can be used to
send different events into multiple backends.
Configuration
type
- string, must be "transform". Required.transformerClass
- class name of the event transformer. Class has to implementio.openlineage.client.transports.transform.EventTransformer
interface and provide public no-arg constructor. Class needs to be available on the classpath. Required.transformerProperties
- Extra properties that can be passed intotransformerClass
based on the configuration. Optional.transport
- Transport configuration to emit modified events. Required.
Behavior
- The configured
transformerClass
will be used to alter events before the emission. - Modified events will be passed into the configured
transport
for further processing.
EventTransformer
interface
public class CustomEventTransformer implements EventTransformer {
@Override
public void initialize(Map<String, String> properties) { ... }
@Override
public RunEvent transform(RunEvent event) { ... }
@Override
public DatasetEvent transform(DatasetEvent event) { .. }
@Override
public JobEvent transform(JobEvent event) { ... }
}
Examples
- Yaml Config
- Spark Config
- Flink Config
- Java Code
transport:
type: transform
transformerClass: io.openlineage.CustomEventTransformer
transformerProperties:
key1: value1
key2: value2
transport:
type: http
url: http://example.com/api
name: my_http
spark.openlineage.transport.type=transform
spark.openlineage.transport.transformerClass=io.openlineage.CustomEventTransformer
spark.openlineage.transport.transformerProperties.key1=value1
spark.openlineage.transport.transformerProperties.key2=value2
spark.openlineage.transport.transport.type=http
spark.openlineage.transport.transport.url=http://example.com/api
openlineage.transport.type=transform
openlineage.transport.transformerClass=io.openlineage.CustomEventTransformer
openlineage.transport.transformerProperties.key1=value1
openlineage.transport.transformerProperties.key2=value2
openlineage.transport.transport.type=http
openlineage.transport.transport.url=http://example.com/api
import java.util.Arrays;
import io.openlineage.client.OpenLineageClient;
import io.openlineage.client.transports.TransformConfig;
import io.openlineage.client.transports.HttpConfig;
import io.openlineage.client.transports.HttpTransport;
HttpConfig httpConfig = new HttpConfig();
httpConfig.setUrl(URI.create("http://example.com/api"));
TransformConfig transformConfig = new TransformConfig();
transformConfig.setTransformerClass(CustomEventTransformer.class.getName());
transformConfig.setTransport(httpConfig);
OpenLineageClient client = OpenLineageClient
.builder()
.transport(new TransformTransport(transformConfig))
.build();
GcpLineage
To use this transport in your project, you need to include io.openlineage:transports-gcplineage
artifact in
your build configuration. This is particularly important for environments like Spark
, where this transport must be on
the classpath for lineage events to be emitted correctly.
Configuration
type
- string, must be"gcplineage"
. Required.endpoint
- string, specifies the endpoint to which events are sent, default value isdatalineage.googleapis.com:443
. Optional.projectId
- string, the project quota identifier. If not provided, it is determined based on user credentials. Optional.location
- string, Dataplex location. Optional, default:"us"
.credentialsFile
- string, path to the Service Account credentials JSON file. Optional, if not provided Application Default Credentials are usedmode
- enum that specifies the type of client used for publishing OpenLineage events to GCP Lineage service. Possible values:sync
(synchronous) orasync
(asynchronous). Optional, default:async
.
Behavior
- Events are serialized to JSON, included as part of a
gRPC
request, and then dispatched to theGCP Lineage service
endpoint. - Depending on the
mode
chosen, requests are sent using either a synchronous or asynchronous client.
Examples
- Yaml Config
- Spark Config
- Flink Config
- Java Code
transport:
type: gcplineage
projectId: your_gcp_project_id
location: us
mode: sync
credentialsFile: path/to/credentials.json
spark.openlineage.transport.type=gcplineage
spark.openlineage.transport.projectId=your_gcp_project_id
spark.openlineage.transport.location=us
spark.openlineage.transport.mode=sync
spark.openlineage.transport.credentialsFile=path/to/credentials.json
openlineage.transport.type=gcplineage
openlineage.transport.projectId=your_gcp_project_id
openlineage.transport.location=us
openlineage.transport.mode=sync
openlineage.transport.credentialsFile=path/to/credentials.json
import io.openlineage.client.OpenLineageClient;
import io.openlineage.client.transports.gcplineage.GcpLineageTransportConfig;
import io.openlineage.client.transports.dataplex.GcpLineageTransport;
GcpLineageTransportConfig gcplineageConfig = new GcpLineageTransportConfig();
gcplineageConfig.setProjectId("your_gcp_project_id");
gcplineageConfig.setLocation("your_gcp_location");
gcplineageConfig.setMode("sync");
gcplineageConfig.setCredentialsFile("path/to/credentials.json");
OpenLineageClient client = OpenLineageClient.builder()
.transport(
new GcpLineageTransport(gcplineageConfig))
.build();
Google Cloud Storage
To use this transport in your project, you need to include io.openlineage:transports-gcs
artifact in
your build configuration. This is particularly important for environments like Spark
, where this transport must be on
the classpath for lineage events to be emitted correctly.
Configuration
type
- string, must be"gcs"
. Required.projectId
- string, the project quota identifier. Required.credentialsFile
- string, path to the Service Account credentials JSON file. Optional, if not provided Application Default Credentials are usedbucketName
- string, the GCS bucket name. RequiredfileNamePrefix
- string, prefix for the event file names. Optional.
Behavior
- Events are serialized to JSON and stored in the specified GCS bucket.
- Each event file is named based on its
eventTime
, converted to epoch milliseconds, with an optional prefix if configured. - Two constructors are available: one accepting both
Storage
andGcsTransportConfig
and another solely acceptingGcsTransportConfig
.
Examples
- Yaml Config
- Spark Config
- Flink Config
- Java Code
transport:
type: gcs
bucketName: my-gcs-bucket
fileNamePrefix: /file/name/prefix/
credentialsFile: path/to/credentials.json
spark.openlineage.transport.type=gcs
spark.openlineage.transport.bucketName=my-gcs-bucket
spark.openlineage.transport.credentialsFile=path/to/credentials.json
spark.openlineage.transport.credentialsFile=file/name/prefix/
openlineage.transport.type=gcs
openlineage.transport.bucketName=my-gcs-bucket
openlineage.transport.credentialsFile=path/to/credentials.json
openlineage.transport.credentialsFile=file/name/prefix/
import io.openlineage.client.OpenLineageClient;
import io.openlineage.client.transports.gcs.GcsTransportConfig;
import io.openlineage.client.transports.dataplex.GcsTransport;
DataplexConfig gcsConfig = new GcsTransportConfig();
gcsConfig.setBucketName("my-bucket-name");
gcsConfig.setFileNamePrefix("/file/name/prefix/");
gcsConfig.setCredentialsFile("path/to/credentials.json");
OpenLineageClient client = OpenLineageClient.builder()
.transport(
new GcsTransport(dataplexConfig))
.build();
S3
To use this transport in your project, you need to include the following dependency in your build configuration. This is
particularly important for environments like Spark
, where this transport must be on the classpath for lineage events
to be emitted correctly.
Maven
<dependency>
<groupId>io.openlineage</groupId>
<artifactId>transports-s3</artifactId>
<version>1.26.0</version>
</dependency>
Configuration
type
- string, must be"s3"
. Required.endpoint
- string, the endpoint for S3 compliant service like MinIO, Ceph, etc. OptionalbucketName
- string, the S3 bucket name. RequiredfileNamePrefix
- string, prefix for the event file names. It is separated from the timestamp with underscore. It can include path and file name prefix. Optional.
Credentials
To authenticate, the transport uses the default credentials provider chain. The possible authentication methods include:
- Java system properties
- Environment variables
- Shared credentials config file (by default
~/.aws/config
) - EC2 instance credentials (convenient in EMR and Glue)
- and other
Refer to the documentation for details.
Behavior
- Events are serialized to JSON and stored in the specified S3 bucket.
- Each event file is named based on its
eventTime
, converted to epoch milliseconds, with an optional prefix if configured.
Examples
- Yaml Config
- Spark Config
- Flink Config
- Java Code
transport:
type: s3
endpoint: https://my-minio.example.com
bucketName: events
fileNamePrefix: my/service/events/event
spark.openlineage.transport.type=s3
spark.openlineage.transport.endpoint=https://my-minio.example.com
spark.openlineage.transport.bucketName=events
spark.openlineage.transport.fileNamePrefix=my/service/events/event
openlineage.transport.type=s3
openlineage.transport.endpoint=https://my-minio.example.com
openlineage.transport.bucketName=events
openlineage.transport.fileNamePrefix=my/service/events/event
import io.openlineage.client.OpenLineageClient;
import io.openlineage.client.transports.s3.S3TransportConfig;
import io.openlineage.client.transports.s3.S3Transport;
S3TransportConfig s3Config = new S3TransportConfig();
s3Config.setEndpoint("https://my-minio.example.com");
s3Config.setBucketName("events");
s3Config.setFileNamePrefix("my/service/events/event");
OpenLineageClient client = OpenLineageClient.builder()
.transport(new S3Transport(s3Config))
.build();
Circuit Breakers
This feature is available in OpenLineage versions >= 1.9.0.
To prevent from over-instrumentation OpenLineage integration provides a circuit breaker mechanism that stops OpenLineage from creating, serializing and sending OpenLineage events.
Simple Memory Circuit Breaker
Simple circuit breaker which is working based only on free memory within JVM. Configuration should
contain free memory threshold limit (percentage). Default value is 20%
. The circuit breaker
will close within first call if free memory is low. circuitCheckIntervalInMillis
parameter is used
to configure a frequency circuit breaker is called. Default value is 1000ms
, when no entry in config.
timeoutInSeconds
is optional. If set, OpenLineage code execution is terminated when a timeout
is reached (added in version 1.13).
- Yaml Config
- Spark Config
- Flink Config
circuitBreaker:
type: simpleMemory
memoryThreshold: 20
circuitCheckIntervalInMillis: 1000
timeoutInSeconds: 90
Parameter | Definition | Example |
---|---|---|
spark.openlineage.circuitBreaker.type | Circuit breaker type selected | simpleMemory |
spark.openlineage.circuitBreaker.memoryThreshold | Memory threshold | 20 |
spark.openlineage.circuitBreaker.circuitCheckIntervalInMillis | Frequency of checking circuit breaker | 1000 |
spark.openlineage.circuitBreaker.timeoutInSeconds | Optional timeout for OpenLineage execution (Since version 1.13) | 90 |
Parameter | Definition | Example |
---|---|---|
openlineage.circuitBreaker.type | Circuit breaker type selected | simpleMemory |
openlineage.circuitBreaker.memoryThreshold | Memory threshold | 20 |
openlineage.circuitBreaker.circuitCheckIntervalInMillis | Frequency of checking circuit breaker | 1000 |
spark.openlineage.circuitBreaker.timeoutInSeconds | Optional timeout for OpenLineage execution (Since version 1.13) | 90 |
Java Runtime Circuit Breaker
More complex version of circuit breaker. The amount of free memory can be low as long as
amount of time spent on Garbage Collection is acceptable. JavaRuntimeCircuitBreaker
closes
when free memory drops below threshold and amount of time spent on garbage collection exceeds
given threshold (10%
by default). The circuit breaker is always open when checked for the first time
as GC threshold is computed since the previous circuit breaker call.
circuitCheckIntervalInMillis
parameter is used
to configure a frequency circuit breaker is called.
Default value is 1000ms
, when no entry in config.
timeoutInSeconds
is optional. If set, OpenLineage code execution is terminated when a timeout
is reached (added in version 1.13).
- Yaml Config
- Spark Config
- Flink Config
circuitBreaker:
type: javaRuntime
memoryThreshold: 20
gcCpuThreshold: 10
circuitCheckIntervalInMillis: 1000
timeoutInSeconds: 90
Parameter | Definition | Example |
---|---|---|
spark.openlineage.circuitBreaker.type | Circuit breaker type selected | javaRuntime |
spark.openlineage.circuitBreaker.memoryThreshold | Memory threshold | 20 |
spark.openlineage.circuitBreaker.gcCpuThreshold | Garbage Collection CPU threshold | 10 |
spark.openlineage.circuitBreaker.circuitCheckIntervalInMillis | Frequency of checking circuit breaker | 1000 |
spark.openlineage.circuitBreaker.timeoutInSeconds | Optional timeout for OpenLineage execution (Since version 1.13) | 90 |
Parameter | Definition | Example |
---|---|---|
openlineage.circuitBreaker.type | Circuit breaker type selected | javaRuntime |
openlineage.circuitBreaker.memoryThreshold | Memory threshold | 20 |
openlineage.circuitBreaker.gcCpuThreshold | Garbage Collection CPU threshold | 10 |
openlineage.circuitBreaker.circuitCheckIntervalInMillis | Frequency of checking circuit breaker | 1000 |
spark.openlineage.circuitBreaker.timeoutInSeconds | Optional timeout for OpenLineage execution (Since version 1.13) | 90 |
Custom Circuit Breaker
List of available circuit breakers can be extended with custom one loaded via ServiceLoader
with own implementation of io.openlineage.client.circuitBreaker.CircuitBreakerBuilder
.