|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.beam.examples.iceberg.snippets; |
| 19 | + |
| 20 | +import java.util.Arrays; |
| 21 | +import java.util.Map; |
| 22 | +import org.apache.beam.sdk.Pipeline; |
| 23 | +import org.apache.beam.sdk.managed.Managed; |
| 24 | +import org.apache.beam.sdk.schemas.Schema; |
| 25 | +import org.apache.beam.sdk.schemas.transforms.AddFields; |
| 26 | +import org.apache.beam.sdk.schemas.transforms.Group; |
| 27 | +import org.apache.beam.sdk.transforms.Create; |
| 28 | +import org.apache.beam.sdk.transforms.MapElements; |
| 29 | +import org.apache.beam.sdk.transforms.Mean; |
| 30 | +import org.apache.beam.sdk.transforms.Sum; |
| 31 | +import org.apache.beam.sdk.values.PCollection; |
| 32 | +import org.apache.beam.sdk.values.Row; |
| 33 | +import org.apache.beam.sdk.values.TypeDescriptors; |
| 34 | +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; |
| 35 | + |
| 36 | +@SuppressWarnings("all") |
| 37 | +public class Quickstart { |
| 38 | + static final String PROJECT_ID = "apache-beam-testing"; |
| 39 | + static final String BUCKET_NAME = "my-bucket"; |
| 40 | + |
| 41 | + public static void main(String[] args) { |
| 42 | + // [START hadoop_catalog_props] |
| 43 | + Map<String, String> catalogProps = |
| 44 | + ImmutableMap.of( |
| 45 | + "type", "hadoop", |
| 46 | + "warehouse", "file:///tmp/beam-iceberg-local-quickstart"); |
| 47 | + // [END hadoop_catalog_props] |
| 48 | + } |
| 49 | + |
| 50 | + public static void publicDatasets() { |
| 51 | + // [START biglake_public_catalog_props] |
| 52 | + Map<String, String> catalogProps = |
| 53 | + ImmutableMap.of( |
| 54 | + "type", "rest", |
| 55 | + "uri", "https://biglake.googleapis.com/iceberg/v1/restcatalog", |
| 56 | + "warehouse", "gs://biglake-public-nyc-taxi-iceberg", |
| 57 | + "header.x-goog-user-project", PROJECT_ID, |
| 58 | + "rest.auth.type", "google", |
| 59 | + "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", |
| 60 | + "header.X-Iceberg-Access-Delegation", "vended-credentials"); |
| 61 | + // [END biglake_public_catalog_props] |
| 62 | + |
| 63 | + // [START biglake_public_query] |
| 64 | + Pipeline p = Pipeline.create(); |
| 65 | + |
| 66 | + // Set up query properties: |
| 67 | + Map<String, Object> config = |
| 68 | + ImmutableMap.of( |
| 69 | + "table", |
| 70 | + "public_data.nyc_taxicab", |
| 71 | + "catalog_properties", |
| 72 | + catalogProps, |
| 73 | + "filter", |
| 74 | + "data_file_year = 2021 AND tip_amount > 100", |
| 75 | + "keep", |
| 76 | + Arrays.asList("passenger_count", "total_amount", "trip_distance")); |
| 77 | + |
| 78 | + // Read Iceberg records |
| 79 | + PCollection<Row> icebergRows = |
| 80 | + p.apply(Managed.read("iceberg").withConfig(config)).getSinglePCollection(); |
| 81 | + |
| 82 | + // Perform further analysis on records |
| 83 | + PCollection<Row> result = |
| 84 | + icebergRows |
| 85 | + .apply(AddFields.<Row>create().field("num_trips", Schema.FieldType.INT32, 1)) |
| 86 | + .apply( |
| 87 | + Group.<Row>byFieldNames("passenger_count") |
| 88 | + .aggregateField("num_trips", Sum.ofIntegers(), "num_trips") |
| 89 | + .aggregateField("total_amount", Mean.of(), "avg_fare") |
| 90 | + .aggregateField("trip_distance", Mean.of(), "avg_distance")); |
| 91 | + |
| 92 | + // Print to console |
| 93 | + result.apply( |
| 94 | + MapElements.into(TypeDescriptors.voids()) |
| 95 | + .via( |
| 96 | + row -> { |
| 97 | + System.out.println(row); |
| 98 | + return null; |
| 99 | + })); |
| 100 | + |
| 101 | + // Execute |
| 102 | + p.run().waitUntilFinish(); |
| 103 | + // [END biglake_public_query] |
| 104 | + } |
| 105 | + |
| 106 | + public static void other() { |
| 107 | + // [START biglake_catalog_props] |
| 108 | + Map<String, String> catalogProps = |
| 109 | + ImmutableMap.of( |
| 110 | + "type", "rest", |
| 111 | + "uri", "https://biglake.googleapis.com/iceberg/v1/restcatalog", |
| 112 | + "warehouse", "gs://" + BUCKET_NAME, |
| 113 | + "header.x-goog-user-project", PROJECT_ID, |
| 114 | + "rest.auth.type", "google", |
| 115 | + "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", |
| 116 | + "header.X-Iceberg-Access-Delegation", "vended-credentials"); |
| 117 | + // [END biglake_catalog_props] |
| 118 | + |
| 119 | + // [START managed_iceberg_config] |
| 120 | + Map<String, Object> managedConfig = |
| 121 | + ImmutableMap.of("table", "my_db.my_table", "catalog_properties", catalogProps); |
| 122 | + |
| 123 | + // Note: The table will get created when inserting data (see below) |
| 124 | + // [END managed_iceberg_config] |
| 125 | + |
| 126 | + // [START managed_iceberg_insert] |
| 127 | + Schema inputSchema = |
| 128 | + Schema.builder().addInt64Field("id").addStringField("name").addInt32Field("age").build(); |
| 129 | + |
| 130 | + Pipeline p = Pipeline.create(); |
| 131 | + p.apply( |
| 132 | + Create.of( |
| 133 | + Row.withSchema(inputSchema).addValues(1, "Mark", 34).build(), |
| 134 | + Row.withSchema(inputSchema).addValues(2, "Omar", 24).build(), |
| 135 | + Row.withSchema(inputSchema).addValues(3, "Rachel", 27).build())) |
| 136 | + .apply(Managed.write("iceberg").withConfig(managedConfig)); |
| 137 | + |
| 138 | + p.run(); |
| 139 | + // [END managed_iceberg_insert] |
| 140 | + |
| 141 | + // [START managed_iceberg_read] |
| 142 | + Pipeline q = Pipeline.create(); |
| 143 | + PCollection<Row> rows = |
| 144 | + q.apply(Managed.read("iceberg").withConfig(managedConfig)).getSinglePCollection(); |
| 145 | + |
| 146 | + rows.apply( |
| 147 | + MapElements.into(TypeDescriptors.voids()) |
| 148 | + .via( |
| 149 | + row -> { |
| 150 | + System.out.println(row); |
| 151 | + return null; |
| 152 | + })); |
| 153 | + |
| 154 | + q.run(); |
| 155 | + // [END managed_iceberg_read] |
| 156 | + } |
| 157 | +} |
0 commit comments