|
| 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.sdk.io.iceberg; |
| 19 | + |
| 20 | +import static org.hamcrest.MatcherAssert.assertThat; |
| 21 | + |
| 22 | +import java.io.Serializable; |
| 23 | +import java.util.List; |
| 24 | +import java.util.Map; |
| 25 | +import java.util.UUID; |
| 26 | +import org.apache.beam.sdk.Pipeline; |
| 27 | +import org.apache.beam.sdk.options.PipelineOptionsFactory; |
| 28 | +import org.apache.beam.sdk.transforms.Create; |
| 29 | +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; |
| 30 | +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; |
| 31 | +import org.apache.iceberg.CatalogUtil; |
| 32 | +import org.apache.iceberg.DeleteFiles; |
| 33 | +import org.apache.iceberg.Table; |
| 34 | +import org.apache.iceberg.catalog.TableIdentifier; |
| 35 | +import org.apache.iceberg.data.IcebergGenerics; |
| 36 | +import org.apache.iceberg.data.Record; |
| 37 | +import org.hamcrest.Matchers; |
| 38 | +import org.junit.ClassRule; |
| 39 | +import org.junit.Rule; |
| 40 | +import org.junit.Test; |
| 41 | +import org.junit.rules.TemporaryFolder; |
| 42 | +import org.junit.runner.RunWith; |
| 43 | +import org.junit.runners.JUnit4; |
| 44 | + |
| 45 | +@RunWith(JUnit4.class) |
| 46 | +public class AppendFilesToTablesTest implements Serializable { |
| 47 | + |
| 48 | + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); |
| 49 | + |
| 50 | + @Rule |
| 51 | + public transient TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); |
| 52 | + |
| 53 | + @Test |
| 54 | + public void testAppendAfterDelete() throws Exception { |
| 55 | + TableIdentifier tableId = |
| 56 | + TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); |
| 57 | + |
| 58 | + Map<String, String> catalogProps = |
| 59 | + ImmutableMap.<String, String>builder() |
| 60 | + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) |
| 61 | + .put("warehouse", warehouse.location) |
| 62 | + .build(); |
| 63 | + |
| 64 | + IcebergCatalogConfig catalog = |
| 65 | + IcebergCatalogConfig.builder() |
| 66 | + .setCatalogName("name") |
| 67 | + .setCatalogProperties(catalogProps) |
| 68 | + .build(); |
| 69 | + |
| 70 | + // 1. Create table and write some data using first pipeline |
| 71 | + Pipeline p1 = Pipeline.create(PipelineOptionsFactory.create()); |
| 72 | + p1.apply("Records To Add", Create.of(TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT1))) |
| 73 | + .setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA)) |
| 74 | + .apply("Append To Table", IcebergIO.writeRows(catalog).to(tableId)); |
| 75 | + |
| 76 | + p1.run().waitUntilFinish(); |
| 77 | + |
| 78 | + // 2. Delete the data |
| 79 | + Table table = warehouse.loadTable(tableId); |
| 80 | + DeleteFiles delete = table.newDelete(); |
| 81 | + // Delete all data files in the current snapshot |
| 82 | + table.currentSnapshot().addedDataFiles(table.io()).forEach(delete::deleteFile); |
| 83 | + delete.commit(); |
| 84 | + |
| 85 | + // 3. Write more data using a fresh second pipeline |
| 86 | + Pipeline p2 = Pipeline.create(PipelineOptionsFactory.create()); |
| 87 | + p2.apply("More Records To Add", Create.of(TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT2))) |
| 88 | + .setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA)) |
| 89 | + .apply("Append More To Table", IcebergIO.writeRows(catalog).to(tableId)); |
| 90 | + |
| 91 | + p2.run().waitUntilFinish(); |
| 92 | + |
| 93 | + // Verify data - after delete and append, only FILE1SNAPSHOT2 should be present |
| 94 | + table.refresh(); |
| 95 | + List<Record> writtenRecords = ImmutableList.copyOf(IcebergGenerics.read(table).build()); |
| 96 | + assertThat(writtenRecords, Matchers.containsInAnyOrder(TestFixtures.FILE1SNAPSHOT2.toArray())); |
| 97 | + } |
| 98 | +} |
0 commit comments