1717package com .example .dataflow ;
1818
1919// [START dataflow_bigquery_read_projection_and_filtering]
20- import com .google .api . services . bigquery . model . TableRow ;
21- import java .util .Arrays ;
20+ import com .google .common . collect . ImmutableMap ;
21+ import java .util .List ;
2222import org .apache .beam .sdk .Pipeline ;
23- import org .apache .beam .sdk .io .gcp .bigquery .BigQueryIO ;
24- import org .apache .beam .sdk .io .gcp .bigquery .BigQueryIO .TypedRead ;
23+ import org .apache .beam .sdk .managed .Managed ;
2524import org .apache .beam .sdk .options .PipelineOptionsFactory ;
2625import org .apache .beam .sdk .transforms .MapElements ;
27- import org .apache .beam .sdk .values .TypeDescriptor ;
26+ import org .apache .beam .sdk .values .Row ;
27+ import org .apache .beam .sdk .values .TypeDescriptors ;
2828
2929public class BigQueryReadWithProjectionAndFiltering {
3030 public static void main (String [] args ) {
@@ -36,28 +36,30 @@ public static void main(String[] args) {
3636 .withValidation ()
3737 .as (ExamplePipelineOptions .class );
3838
39+ String tableSpec = String .format ("%s:%s.%s" ,
40+ options .getProjectId (),
41+ options .getDatasetName (),
42+ options .getTableName ());
43+
44+ ImmutableMap <String , Object > config = ImmutableMap .<String , Object >builder ()
45+ .put ("table" , tableSpec )
46+ .put ("row_restriction" , "age > 18" )
47+ .put ("fields" , List .of ("user_name" , "age" ))
48+ .build ();
49+
3950 // Create a pipeline and apply transforms.
4051 Pipeline pipeline = Pipeline .create (options );
4152 pipeline
42- .apply (BigQueryIO .readTableRows ()
43- // Read rows from a specified table.
44- .from (String .format ("%s:%s.%s" ,
45- options .getProjectId (),
46- options .getDatasetName (),
47- options .getTableName ()))
48- .withMethod (TypedRead .Method .DIRECT_READ )
49- .withSelectedFields (Arrays .asList ("user_name" , "age" ))
50- .withRowRestriction ("age > 18" )
51- )
52- // The output from the previous step is a PCollection<TableRow>.
53+ .apply (Managed .read (Managed .BIGQUERY ).withConfig (config )).getSinglePCollection ()
5354 .apply (MapElements
54- .into (TypeDescriptor .of (TableRow .class ))
55- // Use TableRow to access individual fields in the row.
56- .via ((TableRow row ) -> {
57- var name = (String ) row .get ("user_name" );
58- var age = row .get ("age" );
59- System .out .printf ("Name: %s, Age: %s%n" , name , age );
60- return row ;
55+ .into (TypeDescriptors .strings ())
56+ // Access individual fields in the row.
57+ .via ((Row row ) -> {
58+ String output = String .format ("Name: %s, Age: %s%n" ,
59+ row .getString ("user_name" ),
60+ row .getInt64 ("age" ));
61+ System .out .println (output );
62+ return output ;
6163 }));
6264 pipeline .run ().waitUntilFinish ();
6365 }
0 commit comments