CARVIEW |
Select Language
HTTP/2 200
server: Apache
last-modified: Thu, 03 Jul 2025 18:14:56 GMT
etag: "50cd-6390a5ace2bf3-gzip"
content-encoding: gzip
access-control-allow-origin: *
content-security-policy: frame-src 'self' https://play.beam.apache.org/ https://www.youtube.com/ https://drive.google.com/ ;
content-type: text/html
via: 1.1 varnish, 1.1 varnish
accept-ranges: bytes
age: 0
date: Wed, 16 Jul 2025 09:42:36 GMT
x-served-by: cache-hel1410022-HEL, cache-bom4726-BOM
x-cache: MISS, MISS
x-cache-hits: 0, 0
x-timer: S1752658956.742354,VS0,VE278
vary: Accept-Encoding
strict-transport-security: max-age=31536000; includeSubDomains; preload
content-length: 3745
Managed (Apache Beam 2.66.0)
org.apache.beam.sdk.managed
Class Managed
- java.lang.Object
-
- org.apache.beam.sdk.managed.Managed
-
public class Managed extends java.lang.Object
Top-levelPTransform
s that build and instantiate turnkey transforms.Available transforms
This API currently supports two operations:
read(java.lang.String)
andwrite(java.lang.String)
. Please check the Managed IO configuration page to see available transforms and config options.Building a Managed turnkey transform
Turnkey transforms are represented as
SchemaTransform
s, which means each one has a defined configuration. A given transform can be built with aMap<String, Object>
that specifies arguments using like so:PCollection<Row> rows = pipeline.apply( Managed.read(ICEBERG) .withConfig(ImmutableMap.<String, Object>.builder() .put("foo", "abc") .put("bar", 123) .build())) .getOutput();
Instead of specifying configuration arguments directly in the code, one can provide the location to a YAML file that contains this information. Say we have the following
config.yaml
file:foo: "abc" bar: 123
The file's path can be passed in to the Managed API like so:
PCollection<Row> inputRows = pipeline.apply(Create.of(...)); inputRows.apply(Managed.write(ICEBERG).withConfigUrl("path/to/config.yaml"));
Runner specific features
Google Cloud Dataflow supports additional management features forManaged
including automatically upgrading transforms to the latest supported version. For more details and examples, please see Dataflow managed I/O.
-
-
Nested Class Summary
Nested Classes Modifier and Type Class and Description static class
Managed.ManagedTransform
-
Field Summary
Fields Modifier and Type Field and Description static java.lang.String
BIGQUERY
static java.lang.String
ICEBERG
static java.lang.String
ICEBERG_CDC
static java.lang.String
KAFKA
static java.util.Map<java.lang.String,java.lang.String>
READ_TRANSFORMS
static java.util.Map<java.lang.String,java.lang.String>
WRITE_TRANSFORMS
-
Constructor Summary
Constructors Constructor and Description Managed()
-
Method Summary
All Methods Static Methods Concrete Methods Modifier and Type Method and Description static Managed.ManagedTransform
read(java.lang.String source)
Instantiates aManaged.ManagedTransform
transform for the specified source.static Managed.ManagedTransform
write(java.lang.String sink)
Instantiates aManaged.ManagedTransform
transform for the specified sink.
-
-
-
Field Detail
-
ICEBERG
public static final java.lang.String ICEBERG
- See Also:
- Constant Field Values
-
ICEBERG_CDC
public static final java.lang.String ICEBERG_CDC
- See Also:
- Constant Field Values
-
KAFKA
public static final java.lang.String KAFKA
- See Also:
- Constant Field Values
-
BIGQUERY
public static final java.lang.String BIGQUERY
- See Also:
- Constant Field Values
-
READ_TRANSFORMS
public static final java.util.Map<java.lang.String,java.lang.String> READ_TRANSFORMS
-
WRITE_TRANSFORMS
public static final java.util.Map<java.lang.String,java.lang.String> WRITE_TRANSFORMS
-
-
Method Detail
-
read
public static Managed.ManagedTransform read(java.lang.String source)
Instantiates aManaged.ManagedTransform
transform for the specified source. The supported managed sources are:ICEBERG
: Read from Apache Iceberg tables using IcebergIOICEBERG_CDC
: CDC Read from Apache Iceberg tables using IcebergIOKAFKA
: Read from Apache Kafka topics using KafkaIOBIGQUERY
: Read from GCP BigQuery tables using BigQueryIO
-
write
public static Managed.ManagedTransform write(java.lang.String sink)
Instantiates aManaged.ManagedTransform
transform for the specified sink. The supported managed sinks are:
-
-