Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion .github/trigger_files/IO_Iceberg_Integration_Tests.json
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run.",
"modification": 2
"modification": 4
}
1 change: 1 addition & 0 deletions sdks/java/io/iceberg/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,7 @@ dependencies {
testImplementation project(":sdks:java:io:google-cloud-platform")
testImplementation library.java.google_api_services_bigquery

testImplementation library.java.google_auth_library_oauth2_http
testRuntimeOnly library.java.slf4j_jdk14
testImplementation project(path: ":runners:direct-java", configuration: "shadow")
testRuntimeOnly project(path: ":runners:google-cloud-dataflow-java")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@
*/
package org.apache.beam.sdk.io.iceberg.catalog;

import static org.apache.beam.sdk.io.iceberg.IcebergUtils.beamSchemaToIcebergSchema;
import static org.apache.beam.sdk.io.iceberg.IcebergUtils.icebergSchemaToBeamSchema;
import static org.apache.beam.sdk.managed.Managed.ICEBERG;
import static org.apache.beam.sdk.managed.Managed.ICEBERG_CDC;
import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
Expand Down Expand Up @@ -313,7 +315,7 @@ public Row apply(Long num) {
};

protected static final org.apache.iceberg.Schema ICEBERG_SCHEMA =
IcebergUtils.beamSchemaToIcebergSchema(BEAM_SCHEMA);
beamSchemaToIcebergSchema(BEAM_SCHEMA);
protected static final SimpleFunction<Row, Record> RECORD_FUNC =
new SimpleFunction<Row, Record>() {
@Override
Expand Down Expand Up @@ -346,7 +348,7 @@ private List<Row> populateTable(Table table, @Nullable String charOverride) thro
}
DataWriter<Record> writer =
Parquet.writeData(file)
.schema(ICEBERG_SCHEMA)
.schema(table.schema())
.createWriterFunc(GenericParquetWriter::create)
.overwrite()
.withSpec(table.spec())
Expand Down Expand Up @@ -652,7 +654,7 @@ public void testWrite() throws IOException {
pipeline.run().waitUntilFinish();

Table table = catalog.loadTable(TableIdentifier.parse(tableId()));
assertTrue(table.schema().sameSchema(ICEBERG_SCHEMA));
assertEquals(BEAM_SCHEMA, icebergSchemaToBeamSchema(table.schema()));

// Read back and check records are correct
List<Record> returnedRecords = readRecords(table);
Expand All @@ -664,16 +666,23 @@ public void testWrite() throws IOException {
public void testWriteToPartitionedTable() throws IOException {
Map<String, Object> config = new HashMap<>(managedIcebergConfig(tableId()));
int truncLength = "value_x".length();
config.put(
"partition_fields",
Arrays.asList("bool_field", "hour(datetime)", "truncate(str, " + truncLength + ")"));
List<String> partitionFields =
Arrays.asList("bool_field", "hour(datetime)", "truncate(str, " + truncLength + ")");
config.put("partition_fields", partitionFields);
PCollection<Row> input = pipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA);
input.apply(Managed.write(ICEBERG).withConfig(config));
pipeline.run().waitUntilFinish();

// Read back and check records are correct
Table table = catalog.loadTable(TableIdentifier.parse(tableId()));
List<Record> returnedRecords = readRecords(table);
PartitionSpec expectedSpec =
PartitionSpec.builderFor(table.schema())
.identity("bool_field")
.hour("datetime")
.truncate("str", truncLength)
.build();
assertEquals(expectedSpec, table.spec());
assertThat(
returnedRecords, containsInAnyOrder(inputRows.stream().map(RECORD_FUNC::apply).toArray()));
}
Expand Down Expand Up @@ -815,10 +824,8 @@ private void writeToDynamicDestinations(
Table table3 = catalog.loadTable(TableIdentifier.parse(tableId() + "_3_d"));
Table table4 = catalog.loadTable(TableIdentifier.parse(tableId() + "_4_e"));

org.apache.iceberg.Schema tableSchema =
IcebergUtils.beamSchemaToIcebergSchema(rowFilter.outputSchema());
for (Table t : Arrays.asList(table0, table1, table2, table3, table4)) {
assertTrue(t.schema().sameSchema(tableSchema));
assertEquals(rowFilter.outputSchema(), icebergSchemaToBeamSchema(t.schema()));
}

// Read back and check records are correct
Expand All @@ -830,6 +837,7 @@ private void writeToDynamicDestinations(
readRecords(table3),
readRecords(table4));

org.apache.iceberg.Schema tableSchema = beamSchemaToIcebergSchema(rowFilter.outputSchema());
SerializableFunction<Row, Record> recordFunc =
row -> IcebergUtils.beamRowToIcebergRecord(tableSchema, row);

Expand Down Expand Up @@ -936,7 +944,7 @@ public void testWriteToDynamicNamespaces() throws IOException {
table3false,
table4true,
table4false)) {
assertTrue(t.schema().sameSchema(ICEBERG_SCHEMA));
assertEquals(BEAM_SCHEMA, icebergSchemaToBeamSchema(t.schema()));
}

// Read back and check records are correct
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.io.iceberg.catalog;

import java.util.Map;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.rest.RESTCatalog;
import org.junit.After;
import org.junit.BeforeClass;

/** Tests for {@link org.apache.iceberg.rest.RESTCatalog} using BigLake Metastore. */
public class RESTCatalogBLMSIT extends IcebergCatalogBaseIT {
private static Map<String, String> catalogProps;

// Using a special bucket for this test class because
// BigLake does not support using subfolders as a warehouse (yet)
private static final String BIGLAKE_WAREHOUSE = "gs://managed-iceberg-biglake-its";

@BeforeClass
public static void setup() {
warehouse = BIGLAKE_WAREHOUSE;
catalogProps =
ImmutableMap.<String, String>builder()
.put("type", "rest")
.put("uri", "https://biglake.googleapis.com/iceberg/v1/restcatalog")
.put("warehouse", BIGLAKE_WAREHOUSE)
.put("header.x-goog-user-project", OPTIONS.getProject())
.put("rest-metrics-reporting-enabled", "false")
.put("io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO")
.put("rest.auth.type", "org.apache.iceberg.gcp.auth.GoogleAuthManager")
.build();
}

@After
public void after() {
// making sure the cleanup path is directed at the correct warehouse
warehouse = BIGLAKE_WAREHOUSE;
}

@Override
public String type() {
return "biglake";
}

@Override
public Catalog createCatalog() {
RESTCatalog restCatalog = new RESTCatalog();
restCatalog.initialize(catalogName, catalogProps);
return restCatalog;
}

@Override
public Map<String, Object> managedIcebergConfig(String tableId) {
return ImmutableMap.<String, Object>builder()
.put("table", tableId)
.put("catalog_properties", catalogProps)
.build();
}
}
Loading