From 0437a8dcfcc50407c20a458348c9e1a1fdd6bef8 Mon Sep 17 00:00:00 2001 From: Byron Ellis Date: Thu, 16 Nov 2023 18:11:37 -0800 Subject: [PATCH 1/3] Initial Iceberg connector --- .../beam/gradle/BeamModulePlugin.groovy | 2 +- sdks/java/io/catalog/build.gradle | 34 ++ .../apache/beam/sdk/io/catalog/Catalog.java | 21 + .../sdk/io/catalog/CatalogEnvironment.java | 29 ++ .../beam/sdk/io/catalog/CatalogResource.java | 37 ++ .../io/catalog/CatalogResourceIdentifier.java | 39 ++ .../sdk/io/catalog/CatalogSinkResource.java | 26 ++ .../sdk/io/catalog/CatalogSourceResource.java | 26 ++ .../sdk/io/catalog/CatalogTableResource.java | 20 + .../beam/sdk/io/catalog/package-info.java | 1 + sdks/java/io/iceberg/build.gradle | 98 +++++ .../beam/io/iceberg/CombinedScanReader.java | 200 +++++++++ .../beam/io/iceberg/DynamicDestinations.java | 163 +++++++ .../org/apache/beam/io/iceberg/Iceberg.java | 412 ++++++++++++++++++ .../beam/io/iceberg/IcebergBoundedSource.java | 143 ++++++ .../beam/io/iceberg/IcebergDestination.java | 46 ++ .../apache/beam/io/iceberg/IcebergSink.java | 229 ++++++++++ .../beam/io/iceberg/IcebergWriteResult.java | 95 ++++ .../beam/io/iceberg/MetadataUpdate.java | 172 ++++++++ .../beam/io/iceberg/MetadataUpdates.java | 56 +++ .../apache/beam/io/iceberg/PrepareWrite.java | 91 ++++ .../apache/beam/io/iceberg/RecordWriter.java | 114 +++++ .../beam/io/iceberg/RecordWriterFactory.java | 75 ++++ .../apache/beam/io/iceberg/TableFactory.java | 51 +++ .../beam/io/iceberg/WriteBundlesToFiles.java | 312 +++++++++++++ .../iceberg/WriteGroupedRecordsToFiles.java | 60 +++ .../apache/beam/io/iceberg/package-info.java | 1 + .../beam/io/iceberg/util/PropertyBuilder.java | 38 ++ .../beam/io/iceberg/util/RowHelper.java | 95 ++++ .../beam/io/iceberg/util/ScanHelper.java | 37 ++ .../beam/io/iceberg/util/SchemaHelper.java | 116 +++++ .../beam/io/iceberg/BoundedScanTests.java | 96 ++++ .../org/apache/beam/io/iceberg/SinkTests.java | 71 +++ .../beam/io/iceberg/TestDataWarehouse.java | 142 ++++++ .../apache/beam/io/iceberg/TestFixtures.java | 102 +++++ settings.gradle.kts | 4 + 36 files changed, 3253 insertions(+), 1 deletion(-) create mode 100644 sdks/java/io/catalog/build.gradle create mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/Catalog.java create mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogEnvironment.java create mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResource.java create mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResourceIdentifier.java create mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSinkResource.java create mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSourceResource.java create mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogTableResource.java create mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/package-info.java create mode 100644 sdks/java/io/iceberg/build.gradle create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/CombinedScanReader.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/DynamicDestinations.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/Iceberg.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergBoundedSource.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergDestination.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergSink.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteResult.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdate.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdates.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PrepareWrite.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriter.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriterFactory.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/TableFactory.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteBundlesToFiles.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRecordsToFiles.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/package-info.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/PropertyBuilder.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/RowHelper.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/ScanHelper.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/SchemaHelper.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/BoundedScanTests.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/SinkTests.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestDataWarehouse.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestFixtures.java diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 1a0164a0e53a..ab002456bcf8 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -1151,7 +1151,7 @@ class BeamModulePlugin implements Plugin { options.compilerArgs += ([ '-parameters', '-Xlint:all', - '-Werror' +// '-Werror' ] + (defaultLintSuppressions + configuration.disableLintWarnings).collect { "-Xlint:-${it}" }) } diff --git a/sdks/java/io/catalog/build.gradle b/sdks/java/io/catalog/build.gradle new file mode 100644 index 000000000000..b00417f973a8 --- /dev/null +++ b/sdks/java/io/catalog/build.gradle @@ -0,0 +1,34 @@ +/* + * 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. + */ + +plugins { id 'org.apache.beam.module' } +applyJavaNature( + automaticModuleName: 'org.apache.beam.sdk.io.catalog' +) + +description = "Apache Beam :: SDKs :: Java :: IO :: Catalog" +ext.summary = "Beam Catalog" + +dependencies { + implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation library.java.vendored_guava_32_1_2_jre + testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") + testImplementation library.java.junit + testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") +} \ No newline at end of file diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/Catalog.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/Catalog.java new file mode 100644 index 000000000000..0ab87a267f03 --- /dev/null +++ b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/Catalog.java @@ -0,0 +1,21 @@ +/* + * 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.catalog; + +/** Static Catalog class */ +public class Catalog {} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogEnvironment.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogEnvironment.java new file mode 100644 index 000000000000..14e156fcfd28 --- /dev/null +++ b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogEnvironment.java @@ -0,0 +1,29 @@ +/* + * 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.catalog; + +public interface CatalogEnvironment { + + String defaultNamespace(); + + CatalogResource find(CatalogResourceIdentifier id); + + default CatalogResource find(String... path) { + return find(new CatalogResourceIdentifier(path)); + } +} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResource.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResource.java new file mode 100644 index 000000000000..916a6bc514c7 --- /dev/null +++ b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResource.java @@ -0,0 +1,37 @@ +/* + * 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.catalog; + +/** Generic interface for catalog resources. */ +public interface CatalogResource { + + /** @return Whether or not you can use this resource as a source */ + default boolean isSource() { + return false; + } + + /** @return Whether or not you can use this resource as a sink */ + default boolean isSink() { + return false; + } + + /** @return Whether or not you can use this resource as a function/transform. */ + default boolean isTransform() { + return false; + } +} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResourceIdentifier.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResourceIdentifier.java new file mode 100644 index 000000000000..31b76b470fc1 --- /dev/null +++ b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResourceIdentifier.java @@ -0,0 +1,39 @@ +/* + * 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.catalog; + +import java.util.Arrays; + +public class CatalogResourceIdentifier { + private String[] namespace; + private String name; + + public CatalogResourceIdentifier(String... name) { + if (name.length == 1) { + this.name = name[0]; + this.namespace = new String[0]; + } else { + this.name = name[name.length - 1]; + this.namespace = Arrays.copyOf(name, name.length - 1); + } + } + + public static CatalogResourceIdentifier of(String... name) { + return new CatalogResourceIdentifier(name); + } +} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSinkResource.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSinkResource.java new file mode 100644 index 000000000000..d3a7ccaa71da --- /dev/null +++ b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSinkResource.java @@ -0,0 +1,26 @@ +/* + * 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.catalog; + +public interface CatalogSinkResource extends CatalogResource { + + @Override + default boolean isSink() { + return true; + } +} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSourceResource.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSourceResource.java new file mode 100644 index 000000000000..b892a307a7ee --- /dev/null +++ b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSourceResource.java @@ -0,0 +1,26 @@ +/* + * 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.catalog; + +public interface CatalogSourceResource extends CatalogResource { + + @Override + default boolean isSource() { + return true; + } +} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogTableResource.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogTableResource.java new file mode 100644 index 000000000000..c75084635816 --- /dev/null +++ b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogTableResource.java @@ -0,0 +1,20 @@ +/* + * 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.catalog; + +public interface CatalogTableResource extends CatalogSinkResource, CatalogSourceResource {} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/package-info.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/package-info.java new file mode 100644 index 000000000000..08d120cdd5c4 --- /dev/null +++ b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/package-info.java @@ -0,0 +1 @@ +package org.apache.beam.sdk.io.catalog; diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle new file mode 100644 index 000000000000..6e39575c1679 --- /dev/null +++ b/sdks/java/io/iceberg/build.gradle @@ -0,0 +1,98 @@ +import java.util.stream.Collectors + +/* + * 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. + */ + +plugins { id 'org.apache.beam.module' } +applyJavaNature( + automaticModuleName: 'org.apache.beam.sdk.io.iceberg', +) + +description = "Apache Beam :: SDKs :: Java :: IO :: Iceberg" +ext.summary = "Integration with Iceberg data warehouses." + +def hadoopVersions = [ + "285": "2.8.5", + "292": "2.9.2", + "2102": "2.10.2", + "324": "3.2.4", +] + +hadoopVersions.each {kv -> configurations.create("hadoopVersion$kv.key")} + +def iceberg_version = "1.4.2" +def parquet_version = "1.12.0" +def orc_version = "1.9.2" +def hive_version = "3.1.3" + +dependencies { + implementation library.java.vendored_guava_32_1_2_jre + implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:io:hadoop-common") + implementation library.java.slf4j_api + implementation "org.apache.parquet:parquet-column:$parquet_version" + implementation "org.apache.parquet:parquet-common:$parquet_version" + implementation "org.apache.orc:orc-core:$orc_version" + implementation "org.apache.iceberg:iceberg-core:$iceberg_version" + implementation "org.apache.iceberg:iceberg-api:$iceberg_version" + implementation "org.apache.iceberg:iceberg-parquet:$iceberg_version" + implementation "org.apache.iceberg:iceberg-orc:$iceberg_version" + implementation "org.apache.iceberg:iceberg-arrow:$iceberg_version" + implementation "org.apache.iceberg:iceberg-data:$iceberg_version" + + + + provided library.java.avro + provided library.java.hadoop_client + permitUnusedDeclared library.java.hadoop_client + provided library.java.hadoop_common + testImplementation library.java.hadoop_client + + testImplementation "org.apache.iceberg:iceberg-gcp:$iceberg_version" + testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") + testImplementation library.java.junit + testRuntimeOnly library.java.slf4j_jdk14 + testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + hadoopVersions.each {kv -> + "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-client:$kv.value" + } +} + +hadoopVersions.each {kv -> + configurations."hadoopVersion$kv.key" { + resolutionStrategy { + force "org.apache.hadoop:hadoop-client:$kv.value" + } + } +} + +task hadoopVersionsTest(group: "Verification") { + description = "Runs Iceberg tests with different Hadoop versions" + def taskNames = hadoopVersions.keySet().stream() + .map{num -> "hadoopVersion${num}Test"} + .collect(Collectors.toList()) + dependsOn taskNames +} + +hadoopVersions.each { kv -> + task "hadoopVersion${kv.key}Test"(type: Test, group: "Verification") { + description = "Runs Iceberg tests with Hadoop version $kv.value" + classpath = configurations."hadoopVersion$kv.key" + sourceSets.test.runtimeClasspath + include '**/*Test.class' + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/CombinedScanReader.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/CombinedScanReader.java new file mode 100644 index 000000000000..74ec62af66f8 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/CombinedScanReader.java @@ -0,0 +1,200 @@ +/* + * 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.io.iceberg; + +import java.io.IOException; +import java.util.LinkedList; +import java.util.NoSuchElementException; +import javax.annotation.Nullable; +import org.apache.beam.io.iceberg.util.SchemaHelper; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings("all") +public class CombinedScanReader extends BoundedSource.BoundedReader { + private static final Logger LOG = LoggerFactory.getLogger(CombinedScanReader.class); + + IcebergBoundedSource source; + + @Nullable CombinedScanTask task; + + @Nullable Schema schema; + + transient @Nullable org.apache.iceberg.Schema project; + + transient @Nullable FileIO io; + transient @Nullable EncryptionManager encryptionManager; + + transient @Nullable InputFilesDecryptor decryptor; + + transient LinkedList files = new LinkedList<>(); + + transient CloseableIterator baseIter = null; + + transient Record current; + + public CombinedScanReader( + IcebergBoundedSource source, @Nullable CombinedScanTask task, @Nullable Schema schema) { + this.source = source; + this.task = task; + this.schema = schema; + if (this.schema != null) { + project = SchemaHelper.convert(schema); + } + } + + @Override + public boolean start() throws IOException { + if (task == null) { + return false; + } + + Table table = source.table(); + + io = table.io(); + encryptionManager = table.encryption(); + decryptor = new InputFilesDecryptor(task, io, encryptionManager); + + files.addAll(task.files()); + + return advance(); + } + + @Override + public boolean advance() throws IOException { + do { + // If our current iterator is working... do that. + if (baseIter != null && baseIter.hasNext()) { + current = baseIter.next(); + return true; + } + + // Close out the current iterator and try to open a new one + if (baseIter != null) { + baseIter.close(); + baseIter = null; + } + + LOG.info("Trying to open new file."); + FileScanTask fileTask = null; + while (files.size() > 0 && fileTask == null) { + fileTask = files.removeFirst(); + if (fileTask.isDataTask()) { + LOG.error("{} is a DataTask. Skipping.", fileTask.toString()); + fileTask = null; + } + } + + // We have a new file to start reading + if (fileTask != null) { + DataFile file = fileTask.file(); + InputFile input = decryptor.getInputFile(fileTask); + + CloseableIterable iterable = null; + switch (file.format()) { + case ORC: + LOG.info("Preparing ORC input"); + iterable = + ORC.read(input) + .project(project) + .createReaderFunc( + fileSchema -> GenericOrcReader.buildReader(project, fileSchema)) + .filter(fileTask.residual()) + .build(); + break; + case PARQUET: + LOG.info("Preparing Parquet input."); + iterable = + Parquet.read(input) + .project(project) + .createReaderFunc( + fileSchema -> GenericParquetReaders.buildReader(project, fileSchema)) + .filter(fileTask.residual()) + .build(); + break; + case AVRO: + LOG.info("Preparing Avro input."); + iterable = + Avro.read(input).project(project).createReaderFunc(DataReader::create).build(); + break; + default: + throw new UnsupportedOperationException("Cannot read format: " + file.format()); + } + + if (iterable != null) { + baseIter = iterable.iterator(); + } + } else { + LOG.info("We have exhausted all available files in this CombinedScanTask"); + } + + } while (baseIter != null); + return false; + } + + private Row convert(Record record) { + Row.Builder b = Row.withSchema(schema); + for (int i = 0; i < schema.getFieldCount(); i++) { + // TODO: A lot obviously + b.addValue(record.getField(schema.getField(i).getName())); + } + return b.build(); + } + + @Override + public Row getCurrent() throws NoSuchElementException { + if (current == null) { + throw new NoSuchElementException(); + } + return convert(current); + } + + @Override + public void close() throws IOException { + if (baseIter != null) { + baseIter.close(); + } + files.clear(); + io.close(); + } + + @Override + public BoundedSource getCurrentSource() { + return source; + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/DynamicDestinations.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/DynamicDestinations.java new file mode 100644 index 000000000000..8032e5c1fbbb --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/DynamicDestinations.java @@ -0,0 +1,163 @@ +/* + * 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.io.iceberg; + +import static org.apache.beam.sdk.values.TypeDescriptors.extractFromTypeParameters; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; + +import java.io.Serializable; +import java.util.List; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; + +@SuppressWarnings("all") // TODO: Remove this once development is stable. +public abstract class DynamicDestinations implements Serializable { + + interface SideInputAccessor { + SideInputT sideInput(PCollectionView view); + } + + private transient SideInputAccessor sideInputAccessor; + private transient PipelineOptions options; + + static class ProcessContextSideInputAccessor implements SideInputAccessor { + private DoFn.ProcessContext processContext; + + public ProcessContextSideInputAccessor(DoFn.ProcessContext processContext) { + this.processContext = processContext; + } + + @Override + public SideInputT sideInput(PCollectionView view) { + return processContext.sideInput(view); + } + } + + public PipelineOptions getOptions() { + return options; + } + + public List> getSideInputs() { + return Lists.newArrayList(); + } + + protected final SideInputT sideInput(PCollectionView view) { + checkState( + getSideInputs().contains(view), + "View %s not declared in getSideInputs() (%s)", + view, + getSideInputs()); + if (sideInputAccessor == null) { + throw new IllegalStateException("sideInputAccessor (transient field) is null"); + } + return sideInputAccessor.sideInput(view); + } + + void setSideInputProcessContext(DoFn.ProcessContext context) { + this.sideInputAccessor = new ProcessContextSideInputAccessor(context); + this.options = context.getPipelineOptions(); + } + + public abstract DestinationT getDestination(ValueInSingleWindow element); + + public Coder getDestinationCoder() { + return null; + } + + public abstract Table getTable(DestinationT destination); + + public abstract Schema getSchema(DestinationT destination); + + public abstract PartitionSpec getPartitionSpec(DestinationT destination); + + public abstract FileFormat getFileFormat(DestinationT destination); + + Coder getDestinationCoderWithDefault(CoderRegistry registry) + throws CannotProvideCoderException { + Coder destinationCoder = getDestinationCoder(); + if (destinationCoder != null) { + return destinationCoder; + } + TypeDescriptor descriptor = + extractFromTypeParameters( + this, + DynamicDestinations.class, + new TypeDescriptors.TypeVariableExtractor< + DynamicDestinations, DestinationT>() {}); + try { + return registry.getCoder(descriptor); + } catch (CannotProvideCoderException e) { + throw new CannotProvideCoderException( + "Failed to infer coder for DestinationT from type " + + descriptor + + ", please provide it explicitly by overriding getDestinationCoder()", + e); + } + } + + public static class StaticTableDestination + extends DynamicDestinations { + + final Iceberg.Table table; + + public StaticTableDestination(Iceberg.Table table) { + this.table = table; + } + + @Override + public String getDestination(ValueInSingleWindow element) { + return table.table().name(); + } + + @Override + public Table getTable(String destination) { + return table.table(); + } + + @Override + public Schema getSchema(String destination) { + return getTable(destination).schema(); + } + + @Override + public PartitionSpec getPartitionSpec(String destination) { + return getTable(destination).spec(); + } + + @Override + public FileFormat getFileFormat(String destination) { + return FileFormat.PARQUET; + } + } + + public static StaticTableDestination constant(Iceberg.Table table) { + return new StaticTableDestination<>(table); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/Iceberg.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/Iceberg.java new file mode 100644 index 000000000000..58dc5a6287e9 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/Iceberg.java @@ -0,0 +1,412 @@ +/* + * 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.io.iceberg; + +import com.google.auto.value.AutoValue; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.beam.io.iceberg.util.PropertyBuilder; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableBiFunction; +import org.apache.beam.sdk.transforms.SerializableFunctions; +import org.apache.beam.sdk.values.PCollection; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expression; + +@SuppressWarnings("all") // TODO: Remove this once development is stable. +public class Iceberg { + + public static String DEFAULT_CATALOG_NAME = "default"; + + public enum ScanType { + TABLE, + BATCH + } + + public enum WriteFormat { + AVRO, + PARQUET, + ORC + } + + public static Catalog catalog(String name) { + return Catalog.builder().name(name).build(); + } + + public static Catalog catalog() { + return catalog(DEFAULT_CATALOG_NAME); + } + + @AutoValue + public abstract static class Scan implements Serializable { + + public abstract ScanType getType(); + + public abstract Catalog getCatalog(); + + public abstract ImmutableList getTable(); + + public abstract Schema getSchema(); + + public abstract @Nullable Expression getFilter(); + + public abstract @Nullable Boolean getCaseSensitive(); + + public abstract ImmutableMap getOptions(); + + public abstract @Nullable Long getSnapshot(); + + public abstract @Nullable Long getTimestamp(); + + public abstract @Nullable Long getFromSnapshotInclusive(); + + public abstract @Nullable String getFromSnapshotRefInclusive(); + + public abstract @Nullable Long getFromSnapshotExclusive(); + + public abstract @Nullable String getFromSnapshotRefExclusive(); + + public abstract @Nullable Long getToSnapshot(); + + public abstract @Nullable String getToSnapshotRef(); + + public abstract @Nullable String getTag(); + + public abstract @Nullable String getBranch(); + + public static Scan.Builder builder() { + return new AutoValue_Iceberg_Scan.Builder() + .type(ScanType.TABLE) + .filter(null) + .caseSensitive(null) + .options(ImmutableMap.of()) + .snapshot(null) + .timestamp(null) + .fromSnapshotInclusive(null) + .fromSnapshotRefInclusive(null) + .fromSnapshotExclusive(null) + .fromSnapshotRefExclusive(null) + .toSnapshot(null) + .toSnapshotRef(null) + .tag(null) + .branch(null); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder type(ScanType type); + + public abstract Builder catalog(Catalog catalog); + + public abstract Builder table(ImmutableList table); + + public Builder table(String... table) { + return table(ImmutableList.copyOf(table)); + } + + public abstract Builder schema(Schema schema); + + public abstract Builder filter(@Nullable Expression filter); + + public abstract Builder caseSensitive(@Nullable Boolean caseSensitive); + + public abstract Builder options(ImmutableMap options); + + public abstract Builder snapshot(@Nullable Long snapshot); + + public abstract Builder timestamp(@Nullable Long timestamp); + + public abstract Builder fromSnapshotInclusive(@Nullable Long fromInclusive); + + public abstract Builder fromSnapshotRefInclusive(@Nullable String ref); + + public abstract Builder fromSnapshotExclusive(@Nullable Long fromExclusive); + + public abstract Builder fromSnapshotRefExclusive(@Nullable String ref); + + public abstract Builder toSnapshot(@Nullable Long snapshot); + + public abstract Builder toSnapshotRef(@Nullable String ref); + + public abstract Builder tag(@Nullable String tag); + + public abstract Builder branch(@Nullable String branch); + + public abstract Scan build(); + } + } + + @AutoValue + public abstract static class Catalog implements Serializable { + + public abstract String getName(); + + /* Core Properties */ + public abstract @Nullable String getIcebergCatalogType(); + + public abstract @Nullable String getCatalogImplementation(); + + public abstract @Nullable String getFileIOImplementation(); + + public abstract @Nullable String getWarehouseLocation(); + + public abstract @Nullable String getMetricsReporterImplementation(); + + /* Caching */ + public abstract boolean getCacheEnabled(); + + public abstract boolean getCacheCaseSensitive(); + + public abstract long getCacheExpirationIntervalMillis(); + + public abstract boolean getIOManifestCacheEnabled(); + + public abstract long getIOManifestCacheExpirationIntervalMillis(); + + public abstract long getIOManifestCacheMaxTotalBytes(); + + public abstract long getIOManifestCacheMaxContentLength(); + + public abstract @Nullable String getUri(); + + public abstract int getClientPoolSize(); + + public abstract long getClientPoolEvictionIntervalMs(); + + public abstract @Nullable String getClientPoolCacheKeys(); + + public abstract @Nullable String getLockImplementation(); + + public abstract long getLockHeartbeatIntervalMillis(); + + public abstract long getLockHeartbeatTimeoutMillis(); + + public abstract int getLockHeartbeatThreads(); + + public abstract long getLockAcquireIntervalMillis(); + + public abstract long getLockAcquireTimeoutMillis(); + + public abstract @Nullable String getAppIdentifier(); + + public abstract @Nullable String getUser(); + + public abstract long getAuthSessionTimeoutMillis(); + + public abstract @Nullable Configuration getConfiguration(); + + public static Catalog.Builder builder() { + return new AutoValue_Iceberg_Catalog.Builder() + .icebergCatalogType(null) + .catalogImplementation(null) + .fileIOImplementation(null) + .warehouseLocation(null) + .metricsReporterImplementation(null) // TODO: Set this to our implementation + .cacheEnabled(CatalogProperties.CACHE_ENABLED_DEFAULT) + .cacheCaseSensitive(CatalogProperties.CACHE_CASE_SENSITIVE_DEFAULT) + .cacheExpirationIntervalMillis(CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_DEFAULT) + .iOManifestCacheEnabled(CatalogProperties.IO_MANIFEST_CACHE_ENABLED_DEFAULT) + .iOManifestCacheExpirationIntervalMillis( + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_DEFAULT) + .iOManifestCacheMaxTotalBytes(CatalogProperties.IO_MANIFEST_CACHE_MAX_TOTAL_BYTES_DEFAULT) + .iOManifestCacheMaxContentLength( + CatalogProperties.IO_MANIFEST_CACHE_MAX_CONTENT_LENGTH_DEFAULT) + .uri(null) + .clientPoolSize(CatalogProperties.CLIENT_POOL_SIZE_DEFAULT) + .clientPoolEvictionIntervalMs( + CatalogProperties.CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS_DEFAULT) + .clientPoolCacheKeys(null) + .lockImplementation(null) + .lockHeartbeatIntervalMillis(CatalogProperties.LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT) + .lockHeartbeatTimeoutMillis(CatalogProperties.LOCK_HEARTBEAT_TIMEOUT_MS_DEFAULT) + .lockHeartbeatThreads(CatalogProperties.LOCK_HEARTBEAT_THREADS_DEFAULT) + .lockAcquireIntervalMillis(CatalogProperties.LOCK_ACQUIRE_INTERVAL_MS_DEFAULT) + .lockAcquireTimeoutMillis(CatalogProperties.LOCK_HEARTBEAT_TIMEOUT_MS_DEFAULT) + .appIdentifier(null) + .user(null) + .authSessionTimeoutMillis(CatalogProperties.AUTH_SESSION_TIMEOUT_MS_DEFAULT) + .configuration(null); + } + + public ImmutableMap properties() { + return new PropertyBuilder() + .put(CatalogUtil.ICEBERG_CATALOG_TYPE, getIcebergCatalogType()) + .put(CatalogProperties.CATALOG_IMPL, getCatalogImplementation()) + .put(CatalogProperties.FILE_IO_IMPL, getFileIOImplementation()) + .put(CatalogProperties.WAREHOUSE_LOCATION, getWarehouseLocation()) + .put(CatalogProperties.METRICS_REPORTER_IMPL, getMetricsReporterImplementation()) + .put(CatalogProperties.CACHE_ENABLED, getCacheEnabled()) + .put(CatalogProperties.CACHE_CASE_SENSITIVE, getCacheCaseSensitive()) + .put(CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS, getCacheExpirationIntervalMillis()) + .build(); + } + + public org.apache.iceberg.catalog.Catalog catalog() { + Configuration conf = getConfiguration(); + if (conf == null) { + conf = new Configuration(); + } + return CatalogUtil.buildIcebergCatalog(getName(), properties(), conf); + } + + public Table.Builder table() { + return new AutoValue_Iceberg_Table.Builder().catalog(this); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder name(String name); + + /* Core Properties */ + public abstract Builder icebergCatalogType(@Nullable String icebergType); + + public abstract Builder catalogImplementation(@Nullable String catalogImpl); + + public abstract Builder fileIOImplementation(@Nullable String fileIOImpl); + + public abstract Builder warehouseLocation(@Nullable String warehouse); + + public abstract Builder metricsReporterImplementation(@Nullable String metricsImpl); + + /* Caching */ + public abstract Builder cacheEnabled(boolean cacheEnabled); + + public abstract Builder cacheCaseSensitive(boolean cacheCaseSensitive); + + public abstract Builder cacheExpirationIntervalMillis(long expiration); + + public abstract Builder iOManifestCacheEnabled(boolean enabled); + + public abstract Builder iOManifestCacheExpirationIntervalMillis(long expiration); + + public abstract Builder iOManifestCacheMaxTotalBytes(long bytes); + + public abstract Builder iOManifestCacheMaxContentLength(long length); + + public abstract Builder uri(@Nullable String uri); + + public abstract Builder clientPoolSize(int size); + + public abstract Builder clientPoolEvictionIntervalMs(long interval); + + public abstract Builder clientPoolCacheKeys(@Nullable String keys); + + public abstract Builder lockImplementation(@Nullable String lockImplementation); + + public abstract Builder lockHeartbeatIntervalMillis(long interval); + + public abstract Builder lockHeartbeatTimeoutMillis(long timeout); + + public abstract Builder lockHeartbeatThreads(int threads); + + public abstract Builder lockAcquireIntervalMillis(long interval); + + public abstract Builder lockAcquireTimeoutMillis(long timeout); + + public abstract Builder appIdentifier(@Nullable String id); + + public abstract Builder user(@Nullable String user); + + public abstract Builder authSessionTimeoutMillis(long timeout); + + public abstract Builder configuration(@Nullable Configuration conf); + + public abstract Catalog build(); + + public Builder withProperties(Map properties) { + return this; + } + } + } + + @AutoValue + public abstract static class Table implements Serializable { + + public abstract @Nullable Catalog catalog(); + + public abstract @Nullable List tablePath(); + + public TableIdentifier identifier() { + return TableIdentifier.of(tablePath().toArray(new String[0])); + } + + public org.apache.iceberg.Table table() { + return catalog().catalog().loadTable(identifier()); + } + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder catalog(Catalog catalog); + + public abstract Builder tablePath(List tablePath); + + public abstract Table build(); + } + } + + public static class Write + extends PTransform, IcebergWriteResult> { + + private final DynamicDestinations dynamicDestinations; + private final Catalog catalog; + + private final SerializableBiFunction toRecord; + + public Write( + Catalog catalog, + DynamicDestinations dynamicDestinations, + SerializableBiFunction toRecord) { + this.catalog = catalog; + this.dynamicDestinations = dynamicDestinations; + this.toRecord = toRecord; + } + + @Override + public IcebergWriteResult expand(PCollection input) { + try { + return input + .apply( + "Set Output Location", + new PrepareWrite( + dynamicDestinations, SerializableFunctions.identity(), input.getCoder())) + .apply( + "Dynamic Write", + new IcebergSink( + dynamicDestinations, + dynamicDestinations.getDestinationCoderWithDefault( + input.getPipeline().getCoderRegistry()), + RecordWriterFactory.tableRecords(toRecord, dynamicDestinations), + TableFactory.forCatalog(catalog))); + } catch (Exception e) { + RuntimeException e1 = new RuntimeException("Unable to expand transforms"); + e1.addSuppressed(e); + throw e1; + } + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergBoundedSource.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergBoundedSource.java new file mode 100644 index 000000000000..0c6028c85487 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergBoundedSource.java @@ -0,0 +1,143 @@ +/* + * 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.io.iceberg; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.io.iceberg.util.SchemaHelper; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings("all") // TODO: Remove this once development is stable. +public class IcebergBoundedSource extends BoundedSource { + private static final Logger LOG = LoggerFactory.getLogger(IcebergBoundedSource.class); + + private @Nullable CombinedScanTask task; + private Iceberg.Scan scan; + + public IcebergBoundedSource(Iceberg.Scan scan, @Nullable CombinedScanTask task) { + this.task = task; + this.scan = scan; + } + + public IcebergBoundedSource(Iceberg.Scan scan) { + this(scan, null); + } + + public @Nullable Catalog catalog() { + return scan.getCatalog().catalog(); + } + + public @Nullable Table table() { + Catalog catalog = catalog(); + if (catalog != null) { + return catalog.loadTable( + TableIdentifier.of(scan.getTable().toArray(new String[scan.getTable().size()]))); + } else { + return null; + } + } + + @Override + public List> split( + long desiredBundleSizeBytes, PipelineOptions options) throws Exception { + ArrayList tasks = new ArrayList<>(); + Table table = table(); + if (table != null) { + + switch (scan.getType()) { + case TABLE: + // Override the split size with our desired size + TableScan tableScan = table.newScan(); + + if (desiredBundleSizeBytes > 0) { + tableScan = tableScan.option(TableProperties.SPLIT_SIZE, "" + desiredBundleSizeBytes); + } + + // Always project to our destination schema + tableScan = tableScan.project(SchemaHelper.convert(scan.getSchema())); + + if (scan.getFilter() != null) { + tableScan = tableScan.filter(scan.getFilter()); + } + if (scan.getCaseSensitive() != null) { + tableScan = tableScan.caseSensitive(scan.getCaseSensitive()); + } + if (scan.getSnapshot() != null) { + tableScan = tableScan.useSnapshot(scan.getSnapshot()); + } + if (scan.getBranch() != null) { + tableScan = tableScan.useRef(scan.getBranch()); + } else if (scan.getTag() != null) { + tableScan = tableScan.useRef(scan.getTag()); + } + try (CloseableIterable t = tableScan.planTasks()) { + for (CombinedScanTask c : t) { + tasks.add(new IcebergBoundedSource(scan, c)); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + break; + case BATCH: + // TODO: Add batch scan + break; + } + } + return tasks; + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + if (task == null) { + return 0; + } else { + return task.sizeBytes(); + } + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + } + + @Override + public Coder getOutputCoder() { + return RowCoder.of(scan.getSchema()); + } + + @Override + public BoundedReader createReader(PipelineOptions options) throws IOException { + return new CombinedScanReader(this, task, scan.getSchema()); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergDestination.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergDestination.java new file mode 100644 index 000000000000..9f9790aed4ef --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergDestination.java @@ -0,0 +1,46 @@ +/* + * 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.io.iceberg; + +import java.io.Serializable; +import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.iceberg.Schema; + +public class IcebergDestination implements Serializable { + + ResourceId resourceId; + String table; + Schema schema; + Iceberg.WriteFormat writeFormat; + + public IcebergDestination( + ResourceId resourceId, String table, Schema schema, Iceberg.WriteFormat writeFormat) { + this.resourceId = resourceId; + this.table = table; + this.schema = schema; + this.writeFormat = writeFormat; + } + + public Iceberg.WriteFormat getWriteFormat() { + return writeFormat; + } + + public Schema getSchema() { + return schema; + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergSink.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergSink.java new file mode 100644 index 000000000000..f8ef8721161e --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergSink.java @@ -0,0 +1,229 @@ +/* + * 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.io.iceberg; + +import com.google.common.collect.ImmutableList; +import java.util.UUID; +import org.apache.beam.io.iceberg.WriteBundlesToFiles.Result; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.ShardedKeyCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.ShardedKey; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.commons.lang.NotImplementedException; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.log4j.Logger; + +public class IcebergSink + extends PTransform>, IcebergWriteResult> { + + private static final Logger LOG = Logger.getLogger(IcebergSink.class); + + @VisibleForTesting static final int DEFAULT_MAX_WRITERS_PER_BUNDLE = 20; + @VisibleForTesting static final int DEFAULT_MAX_FILES_PER_PARTITION = 10_000; + @VisibleForTesting static final long DEFAULT_MAX_BYTES_PER_PARTITION = 10L * (1L << 40); // 10TB + static final long DEFAULT_MAX_BYTES_PER_FILE = (1L << 40); // 1TB + static final int DEFAULT_NUM_FILE_SHARDS = 0; + static final int FILE_TRIGGERING_RECORD_COUNT = 50_000; + + final DynamicDestinations dynamicDestinations; + final Coder destinationCoder; + + final RecordWriterFactory recordWriterFactory; + final TableFactory tableFactory; + + boolean triggered; + + public IcebergSink( + DynamicDestinations dynamicDestinations, + Coder destinationCoder, + RecordWriterFactory recordWriterFactory, + TableFactory tableFactory) { + this.dynamicDestinations = dynamicDestinations; + this.destinationCoder = destinationCoder; + this.triggered = false; + this.recordWriterFactory = recordWriterFactory; + this.tableFactory = tableFactory; + } + + private IcebergWriteResult expandTriggered(PCollection> input) { + + throw new NotImplementedException("Not yet implemented"); + } + + private IcebergWriteResult expandUntriggered(PCollection> input) { + + final PCollectionView fileView = createJobIdPrefixView(input.getPipeline()); + // We always do the equivalent of a dynamically sharded file creation + TupleTag> writtenFilesTag = + new TupleTag<>("writtenFiles"); + TupleTag, ElementT>> successfulWritesTag = + new TupleTag<>("successfulWrites"); + TupleTag, ElementT>> failedWritesTag = + new TupleTag<>("failedWrites"); + TupleTag> snapshotsTag = new TupleTag<>("snapshots"); + + final Coder elementCoder = + ((KvCoder) input.getCoder()).getValueCoder(); + + // Write everything to files + PCollectionTuple writeBundlesToFiles = + input.apply( + "Write Bundles To Files", + ParDo.of( + new WriteBundlesToFiles<>( + fileView, + successfulWritesTag, + failedWritesTag, + DEFAULT_MAX_WRITERS_PER_BUNDLE, + DEFAULT_MAX_BYTES_PER_FILE, + recordWriterFactory)) + .withSideInputs(fileView) + .withOutputTags( + writtenFilesTag, + TupleTagList.of(ImmutableList.of(successfulWritesTag, failedWritesTag)))); + + PCollection, ElementT>> successfulWrites = + writeBundlesToFiles + .get(successfulWritesTag) + .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), elementCoder)); + + PCollection, ElementT>> failedWrites = + writeBundlesToFiles + .get(failedWritesTag) + .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), elementCoder)); + + PCollection> writtenFilesGrouped = + failedWrites + .apply("Group By Destination", GroupByKey.create()) + .apply( + "Strip Shard ID", + MapElements.via( + new SimpleFunction< + KV, Iterable>, + KV>>() { + @Override + public KV> apply( + KV, Iterable> input) { + return KV.of(input.getKey().getKey(), input.getValue()); + } + })) + .setCoder(KvCoder.of(destinationCoder, IterableCoder.of(elementCoder))) + .apply( + "Write Grouped Records", + ParDo.of( + new WriteGroupedRecordsToFiles<>( + fileView, DEFAULT_MAX_BYTES_PER_FILE, recordWriterFactory))) + .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder)); + + PCollection> catalogUpdates = + PCollectionList.of( + writeBundlesToFiles + .get(writtenFilesTag) + .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder))) + .and(writtenFilesGrouped) + .apply("Flatten Files", Flatten.pCollections()) + .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder)); + + // Apply any sharded writes and flatten everything for catalog updates + PCollection> snapshots = + catalogUpdates + .apply( + "Extract Data File", + ParDo.of( + new DoFn, KV>() { + @ProcessElement + public void processElement( + ProcessContext c, @Element Result element) { + c.output( + KV.of( + element.tableId, + MetadataUpdate.of( + element.partitionSpec, element.update.getDataFiles().get(0)))); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), MetadataUpdate.coder())) + .apply(GroupByKey.create()) + .apply("Write Metadata Updates", ParDo.of(new MetadataUpdates<>(tableFactory))) + .setCoder(KvCoder.of(StringUtf8Coder.of(), SerializableCoder.of(Snapshot.class))); + + return new IcebergWriteResult( + input.getPipeline(), + successfulWrites, + catalogUpdates, + snapshots, + successfulWritesTag, + writtenFilesTag, + snapshotsTag); + } + + private PCollectionView createJobIdPrefixView(Pipeline p) { + + final String jobName = p.getOptions().getJobName(); + + return p.apply("JobIdCreationRoot_", Create.of((Void) null)) + .apply( + "CreateJobId", + ParDo.of( + new DoFn() { + @ProcessElement + public void process(ProcessContext c) { + c.output(jobName + "-" + UUID.randomUUID().toString()); + } + })) + .apply("JobIdSideInput", View.asSingleton()); + } + + public IcebergWriteResult expand(PCollection> input) { + + String jobName = input.getPipeline().getOptions().getJobName(); + + // We always window into global as far as I can tell? + PCollection> globalInput = + input.apply( + "rewindowIntoGlobal", + Window.>into(new GlobalWindows()) + .triggering(DefaultTrigger.of()) + .discardingFiredPanes()); + return triggered ? expandTriggered(input) : expandUntriggered(input); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteResult.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteResult.java new file mode 100644 index 000000000000..0040e385457c --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteResult.java @@ -0,0 +1,95 @@ +/* + * 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.io.iceberg; + +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Snapshot; + +@SuppressWarnings("all") +public final class IcebergWriteResult implements POutput { + + private final Pipeline pipeline; + @Nullable PCollection successfulInserts; + @Nullable TupleTag successfulInsertsTag; + + @Nullable PCollection>> catalogUpdates; + @Nullable TupleTag>> catalogUpdatesTag; + + @Nullable PCollection> snapshots; + + @Nullable TupleTag> snapshotsTag; + + public IcebergWriteResult( + Pipeline pipeline, + @Nullable PCollection successfulInserts, + @Nullable PCollection>> catalogUpdates, + @Nullable PCollection> snapshots, + @Nullable TupleTag successfulInsertsTag, + @Nullable TupleTag>> catalogUpdatesTag, + @Nullable TupleTag> snapshotsTag) { + this.pipeline = pipeline; + this.successfulInserts = successfulInserts; + this.catalogUpdates = catalogUpdates; + this.snapshots = snapshots; + + this.successfulInsertsTag = successfulInsertsTag; + this.catalogUpdatesTag = catalogUpdatesTag; + this.snapshotsTag = snapshotsTag; + } + + @Override + public Pipeline getPipeline() { + return pipeline; + } + + public PCollection getSuccessfulInserts() { + return successfulInserts; + } + + @Override + public Map, PValue> expand() { + ImmutableMap.Builder, PValue> output = ImmutableMap.builder(); + if (successfulInsertsTag != null) { + output.put(successfulInsertsTag, Preconditions.checkNotNull(successfulInserts)); + } + if (catalogUpdatesTag != null) { + output.put(catalogUpdatesTag, Preconditions.checkNotNull(catalogUpdates)); + } + if (snapshotsTag != null) { + output.put(snapshotsTag, Preconditions.checkNotNull(snapshots)); + } + + return output.build(); + } + + @Override + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) {} +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdate.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdate.java new file mode 100644 index 000000000000..bfde156d5c8b --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdate.java @@ -0,0 +1,172 @@ +/* + * 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.io.iceberg; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; +import java.util.Map; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.specific.SpecificData.SchemaConstructable; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.avro.AvroEncoderUtil; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.types.Types.StructType; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; + +@SuppressWarnings("all") +public class MetadataUpdate implements IndexedRecord, SchemaConstructable { + + private List dataFiles; + private List deleteFiles; + + private final Schema avroSchema; + + public MetadataUpdate(Schema avroSchema) { + this.avroSchema = avroSchema; + } + + public MetadataUpdate( + StructType partitionType, List dataFiles, List deleteFiles) { + this.dataFiles = dataFiles; + this.deleteFiles = deleteFiles; + + StructType dataFileStruct = DataFile.getType(partitionType); + Map dataFileNames = + ImmutableMap.of( + dataFileStruct, "org.apache.iceberg.GenericDataFile", + partitionType, "org.apache.iceberg.PartitionData"); + Schema dataFileSchema = AvroSchemaUtil.convert(dataFileStruct, dataFileNames); + Map deleteFileNames = + ImmutableMap.of( + dataFileStruct, "org.apache.iceberg.GenericDeleteFile", + partitionType, "org.apache.iceberg.PartitionData"); + Schema deleteFileSchema = AvroSchemaUtil.convert(dataFileStruct, deleteFileNames); + + this.avroSchema = + SchemaBuilder.builder() + .record(getClass().getName()) + .fields() + .name("dataFiles") + .prop(AvroSchemaUtil.FIELD_ID_PROP, "-1") + .type() + .nullable() + .array() + .items(dataFileSchema) + .noDefault() + .name("deleteFiles") + .prop(AvroSchemaUtil.FIELD_ID_PROP, "-1") + .type() + .nullable() + .array() + .items(deleteFileSchema) + .noDefault() + .endRecord(); + } + + public static MetadataUpdate of(PartitionSpec partitionSpec, DataFile dataFile) { + return new MetadataUpdate(partitionSpec.partitionType(), ImmutableList.of(dataFile), null); + } + + public List getDataFiles() { + return this.dataFiles; + } + + public List getDeleteFiles() { + return this.deleteFiles; + } + + @Override + public void put(int i, Object v) { + switch (i) { + case 0: + this.dataFiles = (List) v; + return; + case 1: + this.deleteFiles = (List) v; + return; + default: + } + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return this.dataFiles; + case 1: + return this.deleteFiles; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } + + @Override + public Schema getSchema() { + return avroSchema; + } + + protected static class MetadataUpdateCoder extends Coder { + + private static final ByteArrayCoder bytesCoder = ByteArrayCoder.of(); + + @Override + public void encode( + MetadataUpdate value, @UnknownKeyFor @NonNull @Initialized OutputStream outStream) + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull + @Initialized IOException { + bytesCoder.encode(AvroEncoderUtil.encode(value, value.getSchema()), outStream); + } + + @Override + public MetadataUpdate decode(@UnknownKeyFor @NonNull @Initialized InputStream inStream) + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull + @Initialized IOException { + byte[] updateBytes = bytesCoder.decode(inStream); + return AvroEncoderUtil.decode(updateBytes); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized List< + ? extends + @UnknownKeyFor @NonNull @Initialized Coder<@UnknownKeyFor @NonNull @Initialized ?>> + getCoderArguments() { + return ImmutableList.of(); + } + + @Override + public void verifyDeterministic() + throws @UnknownKeyFor @NonNull @Initialized NonDeterministicException {} + } + + public static Coder coder() { + return new MetadataUpdateCoder(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdates.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdates.java new file mode 100644 index 000000000000..a19cb3b9bc4d --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdates.java @@ -0,0 +1,56 @@ +/* + * 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.io.iceberg; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; + +public class MetadataUpdates + extends DoFn>, KV> { + + final TableFactory tableFactory; + + public MetadataUpdates(TableFactory tableFactory) { + this.tableFactory = tableFactory; + } + + @ProcessElement + public void processElement( + ProcessContext c, + @Element KV> element, + BoundedWindow window) { + Table table = tableFactory.getTable(element.getKey()); + AppendFiles update = table.newAppend(); + Iterable metadataUpdates = element.getValue(); + if (metadataUpdates != null) { + for (MetadataUpdate metadata : metadataUpdates) { + for (DataFile file : metadata.getDataFiles()) { + update.appendFile(file); + } + } + update.commit(); + c.outputWithTimestamp( + KV.of(element.getKey(), table.currentSnapshot()), window.maxTimestamp()); + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PrepareWrite.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PrepareWrite.java new file mode 100644 index 000000000000..9b380c822e58 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PrepareWrite.java @@ -0,0 +1,91 @@ +/* + * 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.io.iceberg; + +import java.io.IOException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.joda.time.Instant; + +@SuppressWarnings("all") // TODO: Remove this once development is stable. +public class PrepareWrite + extends PTransform, PCollection>> { + + private DynamicDestinations dynamicDestinations; + private SerializableFunction formatFunction; + private Coder outputCoder; + + public PrepareWrite( + DynamicDestinations dynamicDestinations, + SerializableFunction formatFunction, + Coder outputCoder) { + this.dynamicDestinations = dynamicDestinations; + this.formatFunction = formatFunction; + this.outputCoder = outputCoder; + } + + @Override + public PCollection> expand(PCollection input) { + + final Coder destCoder; + try { + destCoder = + KvCoder.of( + dynamicDestinations.getDestinationCoderWithDefault( + input.getPipeline().getCoderRegistry()), + outputCoder); + } catch (Exception e) { + RuntimeException e1 = new RuntimeException("Unable to expand PrepareWrite"); + e1.addSuppressed(e); + throw e1; + } + return input + .apply( + ParDo.of( + new DoFn>() { + + @ProcessElement + public void processElement( + ProcessContext c, + @Element InputT element, + @Timestamp Instant timestamp, + BoundedWindow window, + PaneInfo pane) + throws IOException { + ValueInSingleWindow windowedElement = + ValueInSingleWindow.of(element, timestamp, window, pane); + dynamicDestinations.setSideInputProcessContext(c); + DestinationT tableDestination = + dynamicDestinations.getDestination(windowedElement); + OutputT outputValue = formatFunction.apply(element); + c.output(KV.of(tableDestination, outputValue)); + } + }) + .withSideInputs(dynamicDestinations.getSideInputs())) + .setCoder(destCoder); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriter.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriter.java new file mode 100644 index 000000000000..2f7380f1ced0 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriter.java @@ -0,0 +1,114 @@ +/* + * 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.io.iceberg; + +import java.io.IOException; +import org.apache.beam.sdk.transforms.SerializableBiFunction; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.orc.GenericOrcWriter; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; + +@SuppressWarnings("all") // TODO: Remove this once development is stable. +class RecordWriter { + + final Table table; + + final DataWriter writer; + final GenericRecord baseRecord; + final SerializableBiFunction toRecord; + + final String location; + + RecordWriter( + Table table, + String location, + Schema schema, + PartitionSpec partitionSpec, + FileFormat format, + SerializableBiFunction toRecord) + throws IOException { + this.table = table; + this.baseRecord = GenericRecord.create(schema); + this.toRecord = toRecord; + this.location = table.locationProvider().newDataLocation(partitionSpec, baseRecord, location); + + OutputFile outputFile = table.io().newOutputFile(this.location); + switch (format) { + case AVRO: + writer = + Avro.writeData(outputFile).schema(schema).withSpec(partitionSpec).overwrite().build(); + break; + case PARQUET: + writer = + Parquet.writeData(outputFile) + .createWriterFunc(GenericParquetWriter::buildWriter) + .schema(schema) + .withSpec(partitionSpec) + .overwrite() + .build(); + break; + case ORC: + writer = + ORC.writeData(outputFile) + .createWriterFunc(GenericOrcWriter::buildWriter) + .schema(schema) + .withSpec(partitionSpec) + .overwrite() + .build(); + break; + default: + throw new RuntimeException("Unrecognized File Format. This should be impossible."); + } + } + + public void write(ElementT element) throws IOException { + Record record = toRecord.apply(baseRecord, element); + writer.write(record); + } + + public void close() throws IOException { + writer.close(); + } + + public long bytesWritten() { + return writer.length(); + } + + public Table table() { + return table; + } + + public String location() { + return location; + } + + public DataFile dataFile() { + return writer.toDataFile(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriterFactory.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriterFactory.java new file mode 100644 index 000000000000..9058d150c530 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriterFactory.java @@ -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.io.iceberg; + +import java.io.Serializable; +import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.SerializableBiFunction; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.Record; + +@SuppressWarnings("all") // TODO: Remove this once development is stable. +abstract class RecordWriterFactory implements Serializable { + private RecordWriterFactory() {} + + public abstract RecordWriterFactory prepare( + DynamicDestinations destination); + + public abstract RecordWriter createWriter(String location, DestinationT destination) + throws Exception; + + static TableRecordWriterFactory tableRecords( + SerializableBiFunction toRecord, + @Nullable DynamicDestinations dynamicDestinations) { + return new TableRecordWriterFactory<>(toRecord, dynamicDestinations); + } + + static final class TableRecordWriterFactory + extends RecordWriterFactory { + + final SerializableBiFunction toRecord; + + final DynamicDestinations dynamicDestinations; + + TableRecordWriterFactory( + SerializableBiFunction toRecord, + DynamicDestinations dynamicDestinations) { + this.toRecord = toRecord; + this.dynamicDestinations = dynamicDestinations; + } + + @Override + public RecordWriterFactory prepare( + DynamicDestinations destination) { + return new TableRecordWriterFactory<>(toRecord, destination); + } + + @Override + public RecordWriter createWriter(String location, DestinationT destination) + throws Exception { + Table table = dynamicDestinations.getTable(destination); + Schema schema = dynamicDestinations.getSchema(destination); + PartitionSpec partitionSpec = dynamicDestinations.getPartitionSpec(destination); + FileFormat format = dynamicDestinations.getFileFormat(destination); + return new RecordWriter<>(table, location, schema, partitionSpec, format, toRecord); + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/TableFactory.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/TableFactory.java new file mode 100644 index 000000000000..135faf3decfe --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/TableFactory.java @@ -0,0 +1,51 @@ +/* + * 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.io.iceberg; + +import java.io.Serializable; +import java.util.Arrays; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; + +@SuppressWarnings("all") +public abstract class TableFactory implements Serializable { + + private TableFactory() {} + + public abstract Table getTable(IdentifierT id); + + public static TableFactory forCatalog(final Iceberg.Catalog catalog) { + return new TableFactory() { + @Override + public Table getTable(String id) { + TableIdentifier tableId = TableIdentifier.parse(id); + // If the first element in the namespace is our catalog, remove that. + if (tableId.hasNamespace()) { + Namespace ns = tableId.namespace(); + if (catalog.catalog().name().equals(ns.level(0))) { + String[] levels = ns.levels(); + levels = Arrays.copyOfRange(levels, 1, levels.length); + tableId = TableIdentifier.of(Namespace.of(levels), tableId.name()); + } + } + return catalog.catalog().loadTable(tableId); + } + }; + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteBundlesToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteBundlesToFiles.java new file mode 100644 index 000000000000..b9ef6fe55db9 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteBundlesToFiles.java @@ -0,0 +1,312 @@ +/* + * 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.io.iceberg; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.beam.io.iceberg.WriteBundlesToFiles.Result; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.StructuredCoder; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.ShardedKey; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Objects; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; + +@SuppressWarnings("all") // TODO: Remove this once development is stable. +public class WriteBundlesToFiles + extends DoFn, Result> { + + private transient Map> writers; + private transient Map windows; + + private static final int SPILLED_RECORD_SHARDING_FACTOR = 10; + + private final PCollectionView locationPrefixView; + + private final TupleTag, ElementT>> successfulWritesTag; + private final TupleTag, ElementT>> unwrittenRecordsTag; + private final int maxWritersPerBundle; + private final long maxFileSize; + + private final RecordWriterFactory recordWriterFactory; + + private int spilledShardNumber; + + static final class Result implements Serializable { + private static final long serialVersionUID = 1L; + + public final String tableId; + public final String location; + + public final PartitionSpec partitionSpec; + + public final MetadataUpdate update; + + public final DestinationT destination; + + public Result( + String tableId, + String location, + DataFile dataFile, + PartitionSpec partitionSpec, + DestinationT destination) { + this.tableId = tableId; + this.location = location; + this.update = MetadataUpdate.of(partitionSpec, dataFile); + this.partitionSpec = partitionSpec; + this.destination = destination; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof Result) { + Result result = (Result) obj; + return Objects.equal(result.tableId, tableId) + && Objects.equal(result.location, location) + && Objects.equal(result.partitionSpec, partitionSpec) + && Objects.equal(result.update.getDataFiles().get(0), update.getDataFiles().get(0)) + && Objects.equal(destination, result.destination); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode( + tableId, location, update.getDataFiles().get(0), partitionSpec, destination); + } + + @Override + public String toString() { + return "Result{" + + "table='" + + tableId + + '\'' + + "location='" + + location + + '\'' + + ", fileByteSize=" + + update.getDataFiles().get(0).fileSizeInBytes() + + ", destination=" + + destination + + '}'; + } + } + + public static class ResultCoder extends StructuredCoder> { + private static final StringUtf8Coder stringCoder = StringUtf8Coder.of(); + private static final Coder metadataCoder = MetadataUpdate.coder(); + + private static final SerializableCoder partitionSpecCoder = + SerializableCoder.of(PartitionSpec.class); + + private final Coder destinationCoder; + + public ResultCoder(Coder destinationCoder) { + this.destinationCoder = destinationCoder; + } + + @Override + public void encode( + Result value, @UnknownKeyFor @NonNull @Initialized OutputStream outStream) + throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull + @Initialized IOException { + + // Convert most everything to Avro for serialization + + // Table id and location are strings + stringCoder.encode(value.tableId, outStream); + stringCoder.encode(value.location, outStream); + // PartitionSpec is Java serialized because we need it to decode DataFile + destinationCoder.encode(value.destination, outStream); + metadataCoder.encode(value.update, outStream); + partitionSpecCoder.encode(value.partitionSpec, outStream); + } + + @Override + public Result decode(InputStream inStream) throws CoderException, IOException { + String tableId = stringCoder.decode(inStream); + String location = stringCoder.decode(inStream); + DestinationT dest = destinationCoder.decode(inStream); + MetadataUpdate update = metadataCoder.decode(inStream); + PartitionSpec spec = partitionSpecCoder.decode(inStream); + return new Result<>(tableId, location, update.getDataFiles().get(0), spec, dest); + } + + @Override + public List> getCoderArguments() { + return Collections.singletonList(destinationCoder); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException {} + + public static ResultCoder of( + Coder destinationCoder) { + return new ResultCoder<>(destinationCoder); + } + } + + public WriteBundlesToFiles( + PCollectionView locationPrefixView, + TupleTag, ElementT>> successfulWritesTag, + TupleTag, ElementT>> unwrittenRecordsTag, + int maximumWritersPerBundle, + long maxFileSize, + RecordWriterFactory recordWriterFactory) { + this.locationPrefixView = locationPrefixView; + this.successfulWritesTag = successfulWritesTag; + this.unwrittenRecordsTag = unwrittenRecordsTag; + this.maxWritersPerBundle = maximumWritersPerBundle; + this.maxFileSize = maxFileSize; + this.recordWriterFactory = recordWriterFactory; + } + + @StartBundle + public void startBundle() { + this.writers = Maps.newHashMap(); + this.windows = Maps.newHashMap(); + this.spilledShardNumber = ThreadLocalRandom.current().nextInt(SPILLED_RECORD_SHARDING_FACTOR); + } + + RecordWriter createWriter( + DestinationT destination, String location, BoundedWindow window) throws Exception { + Map windows = Preconditions.checkNotNull(this.windows); + Map> writers = Preconditions.checkNotNull(this.writers); + RecordWriter writer = recordWriterFactory.createWriter(location, destination); + windows.put(destination, window); + writers.put(destination, writer); + return writer; + } + + @ProcessElement + public void processElement( + ProcessContext c, @Element KV element, BoundedWindow window) + throws Exception { + Map> writers = Preconditions.checkNotNull(this.writers); + String locationPrefix = c.sideInput(locationPrefixView); + DestinationT destination = element.getKey(); + RecordWriter writer; + if (writers.containsKey(destination)) { + writer = writers.get(destination); + } else { + if (writers.size() <= maxWritersPerBundle) { + writer = createWriter(destination, locationPrefix, window); + } else { + c.output( + unwrittenRecordsTag, + KV.of( + ShardedKey.of(destination, ++spilledShardNumber % SPILLED_RECORD_SHARDING_FACTOR), + element.getValue())); + return; + } + } + + if (writer.bytesWritten() > maxFileSize) { + writer.close(); + Table t = writer.table(); + + c.output(new Result<>(t.name(), writer.location(), writer.dataFile(), t.spec(), destination)); + writer = createWriter(destination, locationPrefix, window); + } + + try { + writer.write(element.getValue()); + c.output( + successfulWritesTag, + KV.of( + ShardedKey.of(destination, spilledShardNumber % SPILLED_RECORD_SHARDING_FACTOR), + element.getValue())); + } catch (Exception e) { + try { + writer.close(); + } catch (Exception closeException) { + e.addSuppressed(closeException); + } + throw e; + } + } + + @FinishBundle + public void finishBundle(FinishBundleContext c) throws Exception { + Map windows = Preconditions.checkNotNull(this.windows); + Map> writers = Preconditions.checkNotNull(this.writers); + List exceptionList = Lists.newArrayList(); + for (RecordWriter writer : writers.values()) { + try { + writer.close(); + } catch (Exception e) { + exceptionList.add(e); + } + } + if (!exceptionList.isEmpty()) { + Exception e = new IOException("Exception closing some writers."); + for (Exception thrown : exceptionList) { + e.addSuppressed(thrown); + } + throw e; + } + + exceptionList.clear(); + for (Map.Entry> entry : writers.entrySet()) { + try { + DestinationT destination = entry.getKey(); + + RecordWriter writer = entry.getValue(); + BoundedWindow window = windows.get(destination); + Preconditions.checkNotNull(window); + Table t = writer.table(); + c.output( + new Result<>(t.name(), writer.location(), writer.dataFile(), t.spec(), destination), + window.maxTimestamp(), + window); + } catch (Exception e) { + exceptionList.add(e); + } + } + writers.clear(); + if (!exceptionList.isEmpty()) { + Exception e = new IOException("Exception emitting writer metadata."); + for (Exception thrown : exceptionList) { + e.addSuppressed(thrown); + } + throw e; + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRecordsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRecordsToFiles.java new file mode 100644 index 000000000000..53dbf43fa7f3 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRecordsToFiles.java @@ -0,0 +1,60 @@ +/* + * 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.io.iceberg; + +import org.apache.beam.io.iceberg.WriteBundlesToFiles.Result; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollectionView; + +@SuppressWarnings("all") +public class WriteGroupedRecordsToFiles + extends DoFn>, Result> { + + private final PCollectionView locationPrefixView; + private final long maxFileSize; + private final RecordWriterFactory recordWriterFactory; + + WriteGroupedRecordsToFiles( + PCollectionView locationPrefixView, + long maxFileSize, + RecordWriterFactory recordWriterFactory) { + this.locationPrefixView = locationPrefixView; + this.maxFileSize = maxFileSize; + this.recordWriterFactory = recordWriterFactory; + } + + @ProcessElement + public void processElement( + ProcessContext c, @Element KV> element) throws Exception { + String locationPrefix = c.sideInput(locationPrefixView); + DestinationT destination = element.getKey(); + RecordWriter writer = recordWriterFactory.createWriter(locationPrefix, destination); + for (ElementT e : element.getValue()) { + writer.write(e); + } + writer.close(); + c.output( + new Result<>( + writer.table().name(), + writer.location(), + writer.dataFile(), + writer.table().spec(), + destination)); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/package-info.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/package-info.java new file mode 100644 index 000000000000..835b3421ccc4 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/package-info.java @@ -0,0 +1 @@ +package org.apache.beam.io.iceberg; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/PropertyBuilder.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/PropertyBuilder.java new file mode 100644 index 000000000000..9dc67992aeae --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/PropertyBuilder.java @@ -0,0 +1,38 @@ +/* + * 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.io.iceberg.util; + +import com.google.common.collect.ImmutableMap; +import javax.annotation.Nullable; + +/** Convenience utility class to build immutable maps that drops attempts to set null values. */ +public class PropertyBuilder { + + ImmutableMap.Builder builder = ImmutableMap.builder(); + + public PropertyBuilder put(String key, @Nullable Object value) { + if (value != null) { + builder = builder.put(key, "" + value); + } + return this; + } + + public ImmutableMap build() { + return builder.build(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/RowHelper.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/RowHelper.java new file mode 100644 index 000000000000..0ad5f663a2cf --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/RowHelper.java @@ -0,0 +1,95 @@ +/* + * 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.io.iceberg.util; + +import java.util.Optional; +import org.apache.beam.sdk.transforms.SerializableBiFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types.NestedField; + +public class RowHelper { + private RowHelper() {} + + public static void copyInto(Record rec, NestedField field, Row value) { + String name = field.name(); + switch (field.type().typeId()) { + case BOOLEAN: + Optional.ofNullable(value.getBoolean(name)).ifPresent(v -> rec.setField(name, v)); + break; + case INTEGER: + Optional.ofNullable(value.getInt32(name)).ifPresent(v -> rec.setField(name, v)); + break; + case LONG: + Optional.ofNullable(value.getInt64(name)).ifPresent(v -> rec.setField(name, v)); + break; + case FLOAT: + Optional.ofNullable(value.getFloat(name)).ifPresent(v -> rec.setField(name, v)); + break; + case DOUBLE: + Optional.ofNullable(value.getDouble(name)).ifPresent(v -> rec.setField(name, v)); + break; + case DATE: + break; + case TIME: + break; + case TIMESTAMP: + break; + case STRING: + Optional.ofNullable(value.getString(name)).ifPresent(v -> rec.setField(name, v)); + break; + case UUID: + break; + case FIXED: + break; + case BINARY: + break; + case DECIMAL: + break; + case STRUCT: + Optional.ofNullable(value.getRow(name)) + .ifPresent( + row -> + rec.setField( + name, copy(GenericRecord.create(field.type().asStructType()), row))); + break; + case LIST: + break; + case MAP: + break; + } + } + + public static Record copy(Record baseRecord, Row value) { + Record rec = baseRecord.copy(); + for (NestedField f : rec.struct().fields()) { + copyInto(rec, f, value); + } + return rec; + } + + public static SerializableBiFunction recordsFromRows() { + return new SerializableBiFunction() { + @Override + public Record apply(Record record, Row row) { + return copy(record, row); + } + }; + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/ScanHelper.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/ScanHelper.java new file mode 100644 index 000000000000..268ad6b4550b --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/ScanHelper.java @@ -0,0 +1,37 @@ +/* + * 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.io.iceberg.util; + +import org.apache.beam.io.iceberg.Iceberg; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; + +public class ScanHelper { + + public static boolean isIncremental(Iceberg.Scan scan) { + if (scan.getFromSnapshotExclusive() != null) { + return true; + } + return false; + } + + public static TableScan tableScan(Table table, Iceberg.Scan scan) { + TableScan tableScan = table.newScan(); + return tableScan; + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/SchemaHelper.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/SchemaHelper.java new file mode 100644 index 000000000000..c1e25351b640 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/SchemaHelper.java @@ -0,0 +1,116 @@ +/* + * 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.io.iceberg.util; + +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +@SuppressWarnings({"dereference.of.nullable"}) +public class SchemaHelper { + + private SchemaHelper() {} + + public static String ICEBERG_TYPE_OPTION_NAME = "icebergTypeID"; + + public static Schema.FieldType fieldTypeForType(final Type type) { + switch (type.typeId()) { + case BOOLEAN: + return FieldType.BOOLEAN; + case INTEGER: + return FieldType.INT32; + case LONG: + return FieldType.INT64; + case FLOAT: + return FieldType.FLOAT; + case DOUBLE: + return FieldType.DOUBLE; + case DATE: + case TIME: + case TIMESTAMP: // TODO: Logical types? + return FieldType.DATETIME; + case STRING: + return FieldType.STRING; + case UUID: + case BINARY: + return FieldType.BYTES; + case FIXED: + case DECIMAL: + return FieldType.DECIMAL; + case STRUCT: + return FieldType.row(convert(type.asStructType())); + case LIST: + return FieldType.iterable(fieldTypeForType(type.asListType().elementType())); + case MAP: + return FieldType.map( + fieldTypeForType(type.asMapType().keyType()), + fieldTypeForType(type.asMapType().valueType())); + } + throw new RuntimeException("Unrecognized Iceberg Type"); + } + + public static Schema.Field convert(final Types.NestedField field) { + return Schema.Field.of(field.name(), fieldTypeForType(field.type())) + .withOptions( + Schema.Options.builder() + .setOption( + ICEBERG_TYPE_OPTION_NAME, Schema.FieldType.STRING, field.type().typeId().name()) + .build()) + .withNullable(field.isOptional()); + } + + public static Schema convert(final org.apache.iceberg.Schema schema) { + Schema.Builder builder = Schema.builder(); + for (Types.NestedField f : schema.columns()) { + builder.addField(convert(f)); + } + return builder.build(); + } + + public static Schema convert(final Types.StructType struct) { + Schema.Builder builder = Schema.builder(); + for (Types.NestedField f : struct.fields()) { + builder.addField(convert(f)); + } + return builder.build(); + } + + public static Types.NestedField convert(int fieldId, final Schema.Field field) { + String typeId = field.getOptions().getValue(ICEBERG_TYPE_OPTION_NAME, String.class); + if (typeId != null) { + return Types.NestedField.of( + fieldId, + field.getType().getNullable(), + field.getName(), + Types.fromPrimitiveString(typeId)); + } else { + return Types.NestedField.of( + fieldId, field.getType().getNullable(), field.getName(), Types.StringType.get()); + } + } + + public static org.apache.iceberg.Schema convert(final Schema schema) { + Types.NestedField[] fields = new Types.NestedField[schema.getFieldCount()]; + int fieldId = 0; + for (Schema.Field f : schema.getFields()) { + fields[fieldId++] = convert(fieldId, f); + } + return new org.apache.iceberg.Schema(fields); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/BoundedScanTests.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/BoundedScanTests.java new file mode 100644 index 000000000000..3a813dab930b --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/BoundedScanTests.java @@ -0,0 +1,96 @@ +/* + * 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.io.iceberg; + +import org.apache.beam.io.iceberg.util.SchemaHelper; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.Table; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@RunWith(JUnit4.class) +public class BoundedScanTests { + + private static Logger LOG = LoggerFactory.getLogger(BoundedScanTests.class); + @ClassRule public static final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(temporaryFolder, "default"); + + @Rule public TestPipeline testPipeline = TestPipeline.create(); + + static class PrintRow extends DoFn { + + @ProcessElement + public void process(@Element Row row, OutputReceiver output) throws Exception { + LOG.info("Got row {}", row); + output.output(row); + } + } + + @Test + public void testSimpleScan() throws Exception { + Table simpleTable = warehouse.createTable(TestFixtures.SCHEMA); + simpleTable + .newFastAppend() + .appendFile( + warehouse.writeRecords( + "file1s1.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT1)) + .appendFile( + warehouse.writeRecords( + "file2s1.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT1)) + .appendFile( + warehouse.writeRecords( + "file3s1.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT1)) + .commit(); + + PCollection output = + testPipeline + .apply( + Read.from( + new IcebergBoundedSource( + Iceberg.Scan.builder() + .catalog( + Iceberg.Catalog.builder() + .name("hadoop") + .icebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .warehouseLocation(warehouse.location) + .build()) + .type(Iceberg.ScanType.TABLE) + .table(simpleTable.name().replace("hadoop.", "").split("\\.")) + .schema(SchemaHelper.convert(TestFixtures.SCHEMA)) + .build()))) + .apply(ParDo.of(new PrintRow())) + .setCoder(RowCoder.of(SchemaHelper.convert(TestFixtures.SCHEMA))); + PAssert.that(output); + testPipeline.run(); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/SinkTests.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/SinkTests.java new file mode 100644 index 000000000000..8a0eaadab99b --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/SinkTests.java @@ -0,0 +1,71 @@ +/* + * 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.io.iceberg; + +import com.google.common.collect.ImmutableList; +import org.apache.beam.io.iceberg.util.RowHelper; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.Table; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@RunWith(JUnit4.class) +public class SinkTests { + private static Logger LOG = LoggerFactory.getLogger(SinkTests.class); + @ClassRule public static final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(temporaryFolder, "default"); + + @Rule public TestPipeline testPipeline = TestPipeline.create(); + + @Test + public void testSimpleAppend() throws Exception { + // Create a table and add records to it. + Table table = warehouse.createTable(TestFixtures.SCHEMA); + + Iceberg.Catalog catalog = + Iceberg.Catalog.builder() + .name("hadoop") + .icebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .warehouseLocation(warehouse.location) + .build(); + + String[] tablePath = table.name().replace("hadoop.", "").split("\\."); + DynamicDestinations destination = + DynamicDestinations.constant( + catalog.table().tablePath(ImmutableList.copyOf(tablePath)).build()); + LOG.info("Table created. Making pipeline"); + testPipeline + .apply("Records To Add", Create.of(TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT1))) + .apply( + "Append To Table", + new Iceberg.Write(catalog, destination, RowHelper.recordsFromRows())); + LOG.info("Executing pipeline"); + testPipeline.run().waitUntilFinish(); + LOG.info("Done running pipeline"); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestDataWarehouse.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestDataWarehouse.java new file mode 100644 index 000000000000..f9112e5a61a3 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestDataWarehouse.java @@ -0,0 +1,142 @@ +/* + * 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.io.iceberg; + +import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.UUID; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.orc.GenericOrcWriter; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.junit.Assert; +import org.junit.rules.ExternalResource; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TestDataWarehouse extends ExternalResource { + private static final Logger LOG = LoggerFactory.getLogger(TestDataWarehouse.class); + + protected final TemporaryFolder temporaryFolder; + protected final String database; + + protected final Configuration hadoopConf; + + protected String location; + protected Catalog catalog; + + public TestDataWarehouse(TemporaryFolder temporaryFolder, String database) { + this.temporaryFolder = temporaryFolder; + this.database = database; + this.hadoopConf = new Configuration(); + } + + @Override + protected void before() throws Throwable { + File warehouseFile = temporaryFolder.newFolder(); + Assert.assertTrue(warehouseFile.delete()); + location = "file:" + warehouseFile.toString(); + catalog = + CatalogUtil.loadCatalog( + CatalogUtil.ICEBERG_CATALOG_HADOOP, + "hadoop", + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, location), + hadoopConf); + } + + @Override + protected void after() { + List tables = catalog.listTables(Namespace.of(database)); + LOG.info("Cleaning up {} tables in test warehouse", tables.size()); + for (TableIdentifier t : tables) { + try { + LOG.info("Removing table {}", t); + catalog.dropTable(t); + } catch (Exception e) { + LOG.error("Unable to remove table", e); + } + } + try { + ((HadoopCatalog) catalog).close(); + } catch (Exception e) { + LOG.error("Unable to close catalog", e); + } + } + + public DataFile writeRecords(String filename, Schema schema, List records) + throws IOException { + Path path = new Path(location, filename); + FileFormat format = FileFormat.fromFileName(filename); + + FileAppender appender; + switch (format) { + case PARQUET: + appender = + Parquet.write(fromPath(path, hadoopConf)) + .createWriterFunc(GenericParquetWriter::buildWriter) + .schema(schema) + .overwrite() + .build(); + break; + case ORC: + appender = + ORC.write(fromPath(path, hadoopConf)) + .createWriterFunc(GenericOrcWriter::buildWriter) + .schema(schema) + .overwrite() + .build(); + break; + default: + throw new IOException("Unable to create appender for " + format); + } + appender.addAll(records); + appender.close(); + return DataFiles.builder(PartitionSpec.unpartitioned()) + .withInputFile(HadoopInputFile.fromPath(path, hadoopConf)) + .withMetrics(appender.metrics()) + .build(); + } + + public Table createTable(Schema schema) { + TableIdentifier table = + TableIdentifier.of(database, "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); + return catalog.createTable(table, schema); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestFixtures.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestFixtures.java new file mode 100644 index 000000000000..05af551f7fc1 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestFixtures.java @@ -0,0 +1,102 @@ +/* + * 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.io.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.util.ArrayList; +import org.apache.beam.io.iceberg.util.SchemaHelper; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types; + +public class TestFixtures { + public static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), optional(2, "data", Types.StringType.get())); + + private static final Record genericRecord = GenericRecord.create(SCHEMA); + + /* First file in test table */ + public static final ImmutableList FILE1SNAPSHOT1 = + ImmutableList.of( + genericRecord.copy(ImmutableMap.of("id", 0L, "data", "clarification")), + genericRecord.copy(ImmutableMap.of("id", 1L, "data", "risky")), + genericRecord.copy(ImmutableMap.of("id", 2L, "data", "falafel"))); + public static final ImmutableList FILE1SNAPSHOT2 = + ImmutableList.of( + genericRecord.copy(ImmutableMap.of("id", 4L, "data", "obscure")), + genericRecord.copy(ImmutableMap.of("id", 5L, "data", "secure")), + genericRecord.copy(ImmutableMap.of("id", 6L, "data", "feta"))); + public static final ImmutableList FILE1SNAPSHOT3 = + ImmutableList.of( + genericRecord.copy(ImmutableMap.of("id", 6L, "data", "brainy")), + genericRecord.copy(ImmutableMap.of("id", 7L, "data", "film")), + genericRecord.copy(ImmutableMap.of("id", 8L, "data", "feta"))); + + /* Second file in test table */ + public static final ImmutableList FILE2SNAPSHOT1 = + ImmutableList.of( + genericRecord.copy(ImmutableMap.of("id", 10L, "data", "clammy")), + genericRecord.copy(ImmutableMap.of("id", 11L, "data", "evacuate")), + genericRecord.copy(ImmutableMap.of("id", 12L, "data", "tissue"))); + public static final ImmutableList FILE2SNAPSHOT2 = + ImmutableList.of( + genericRecord.copy(ImmutableMap.of("id", 14L, "data", "radical")), + genericRecord.copy(ImmutableMap.of("id", 15L, "data", "collocation")), + genericRecord.copy(ImmutableMap.of("id", 16L, "data", "book"))); + public static final ImmutableList FILE2SNAPSHOT3 = + ImmutableList.of( + genericRecord.copy(ImmutableMap.of("id", 16L, "data", "cake")), + genericRecord.copy(ImmutableMap.of("id", 17L, "data", "intrinsic")), + genericRecord.copy(ImmutableMap.of("id", 18L, "data", "paper"))); + + /* Third file in test table */ + public static final ImmutableList FILE3SNAPSHOT1 = + ImmutableList.of( + genericRecord.copy(ImmutableMap.of("id", 20L, "data", "ocean")), + genericRecord.copy(ImmutableMap.of("id", 21L, "data", "holistic")), + genericRecord.copy(ImmutableMap.of("id", 22L, "data", "preventative"))); + public static final ImmutableList FILE3SNAPSHOT2 = + ImmutableList.of( + genericRecord.copy(ImmutableMap.of("id", 24L, "data", "cloud")), + genericRecord.copy(ImmutableMap.of("id", 25L, "data", "zen")), + genericRecord.copy(ImmutableMap.of("id", 26L, "data", "sky"))); + public static final ImmutableList FILE3SNAPSHOT3 = + ImmutableList.of( + genericRecord.copy(ImmutableMap.of("id", 26L, "data", "belleview")), + genericRecord.copy(ImmutableMap.of("id", 27L, "data", "overview")), + genericRecord.copy(ImmutableMap.of("id", 28L, "data", "tender"))); + + public static final ImmutableList asRows(Iterable records) { + ArrayList rows = new ArrayList<>(); + for (Record record : records) { + rows.add( + Row.withSchema(SchemaHelper.convert(SCHEMA)) + .withFieldValue("id", record.getField("id")) + .withFieldValue("data", record.getField("data")) + .build()); + } + return ImmutableList.copyOf(rows); + } +} diff --git a/settings.gradle.kts b/settings.gradle.kts index 1e52e425b215..632b283ec072 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -355,3 +355,7 @@ include("sdks:java:io:kafka:kafka-01103") findProject(":sdks:java:io:kafka:kafka-01103")?.name = "kafka-01103" include("sdks:java:managed") findProject(":sdks:java:managed")?.name = "managed" +include("sdks:java:io:iceberg") +findProject(":sdks:java:io:iceberg")?.name = "iceberg" +include("sdks:java:io:catalog") +findProject(":sdks:java:io:catalog")?.name = "catalog" From fd08eb49f315acc90f9bc3a96f48c4b0a778e4e7 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 26 Mar 2024 15:49:41 -0400 Subject: [PATCH 2/3] Fix up IcebergIO Write path - remove Read path (will propose separately) - re-enable checking, fix type errors - some style adjustments --- .../beam/gradle/BeamModulePlugin.groovy | 2 +- sdks/java/io/catalog/build.gradle | 34 -- .../apache/beam/sdk/io/catalog/Catalog.java | 21 - .../sdk/io/catalog/CatalogEnvironment.java | 29 -- .../beam/sdk/io/catalog/CatalogResource.java | 37 -- .../io/catalog/CatalogResourceIdentifier.java | 39 -- .../sdk/io/catalog/CatalogSinkResource.java | 26 -- .../sdk/io/catalog/CatalogSourceResource.java | 26 -- .../sdk/io/catalog/CatalogTableResource.java | 20 - .../beam/sdk/io/catalog/package-info.java | 1 - .../beam/io/iceberg/AppendFilesToTables.java | 102 +++++ .../beam/io/iceberg/AssignDestinations.java | 65 +++ .../beam/io/iceberg/CombinedScanReader.java | 200 --------- .../beam/io/iceberg/DynamicDestinations.java | 145 +----- .../beam/io/iceberg/FileWriteResult.java | 203 +++++++++ .../org/apache/beam/io/iceberg/Iceberg.java | 412 ------------------ .../beam/io/iceberg/IcebergBoundedSource.java | 143 ------ .../beam/io/iceberg/IcebergCatalogConfig.java | 233 ++++++++++ .../beam/io/iceberg/IcebergDestination.java | 63 ++- .../org/apache/beam/io/iceberg/IcebergIO.java | 50 +++ .../apache/beam/io/iceberg/IcebergSink.java | 229 ---------- .../io/iceberg/IcebergTableCreateConfig.java | 50 +++ .../beam/io/iceberg/IcebergWriteResult.java | 50 +-- .../beam/io/iceberg/MetadataUpdate.java | 172 -------- .../beam/io/iceberg/MetadataUpdates.java | 56 --- .../iceberg/OneTableDynamicDestinations.java | 65 +++ .../apache/beam/io/iceberg/PrepareWrite.java | 91 ---- .../iceberg/{util => }/PropertyBuilder.java | 8 +- .../apache/beam/io/iceberg/RecordWriter.java | 87 ++-- .../beam/io/iceberg/RecordWriterFactory.java | 75 ---- .../beam/io/iceberg/{util => }/RowHelper.java | 57 +-- .../io/iceberg/{util => }/SchemaHelper.java | 8 +- .../apache/beam/io/iceberg/TableFactory.java | 51 --- .../beam/io/iceberg/WriteBundlesToFiles.java | 312 ------------- .../iceberg/WriteGroupedRecordsToFiles.java | 60 --- .../io/iceberg/WriteGroupedRowsToFiles.java | 103 +++++ .../beam/io/iceberg/WriteToDestinations.java | 123 ++++++ .../io/iceberg/WriteUngroupedRowsToFiles.java | 350 +++++++++++++++ .../apache/beam/io/iceberg/package-info.java | 19 + .../beam/io/iceberg/util/ScanHelper.java | 37 -- .../beam/io/iceberg/BoundedScanTests.java | 96 ---- .../beam/io/iceberg/FileWriteResultTest.java | 147 +++++++ .../beam/io/iceberg/IcebergIOWriteTest.java | 293 +++++++++++++ .../apache/beam/io/iceberg/RowHelperTest.java | 128 ++++++ .../org/apache/beam/io/iceberg/SinkTests.java | 71 --- .../beam/io/iceberg/TestDataWarehouse.java | 14 +- .../apache/beam/io/iceberg/TestFixtures.java | 11 +- settings.gradle.kts | 2 - 48 files changed, 2081 insertions(+), 2535 deletions(-) delete mode 100644 sdks/java/io/catalog/build.gradle delete mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/Catalog.java delete mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogEnvironment.java delete mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResource.java delete mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResourceIdentifier.java delete mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSinkResource.java delete mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSourceResource.java delete mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogTableResource.java delete mode 100644 sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/package-info.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AppendFilesToTables.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AssignDestinations.java delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/CombinedScanReader.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/FileWriteResult.java delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/Iceberg.java delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergBoundedSource.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergCatalogConfig.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergSink.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergTableCreateConfig.java delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdate.java delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdates.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/OneTableDynamicDestinations.java delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PrepareWrite.java rename sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/{util => }/PropertyBuilder.java (85%) delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriterFactory.java rename sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/{util => }/RowHelper.java (65%) rename sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/{util => }/SchemaHelper.java (95%) delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/TableFactory.java delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteBundlesToFiles.java delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRecordsToFiles.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRowsToFiles.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteToDestinations.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteUngroupedRowsToFiles.java delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/ScanHelper.java delete mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/BoundedScanTests.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/FileWriteResultTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOWriteTest.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/RowHelperTest.java delete mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/SinkTests.java diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index ab002456bcf8..1a0164a0e53a 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -1151,7 +1151,7 @@ class BeamModulePlugin implements Plugin { options.compilerArgs += ([ '-parameters', '-Xlint:all', -// '-Werror' + '-Werror' ] + (defaultLintSuppressions + configuration.disableLintWarnings).collect { "-Xlint:-${it}" }) } diff --git a/sdks/java/io/catalog/build.gradle b/sdks/java/io/catalog/build.gradle deleted file mode 100644 index b00417f973a8..000000000000 --- a/sdks/java/io/catalog/build.gradle +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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. - */ - -plugins { id 'org.apache.beam.module' } -applyJavaNature( - automaticModuleName: 'org.apache.beam.sdk.io.catalog' -) - -description = "Apache Beam :: SDKs :: Java :: IO :: Catalog" -ext.summary = "Beam Catalog" - -dependencies { - implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation library.java.vendored_guava_32_1_2_jre - testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") - testImplementation library.java.junit - testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") - testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") -} \ No newline at end of file diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/Catalog.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/Catalog.java deleted file mode 100644 index 0ab87a267f03..000000000000 --- a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/Catalog.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * 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.catalog; - -/** Static Catalog class */ -public class Catalog {} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogEnvironment.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogEnvironment.java deleted file mode 100644 index 14e156fcfd28..000000000000 --- a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogEnvironment.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.catalog; - -public interface CatalogEnvironment { - - String defaultNamespace(); - - CatalogResource find(CatalogResourceIdentifier id); - - default CatalogResource find(String... path) { - return find(new CatalogResourceIdentifier(path)); - } -} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResource.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResource.java deleted file mode 100644 index 916a6bc514c7..000000000000 --- a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResource.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.catalog; - -/** Generic interface for catalog resources. */ -public interface CatalogResource { - - /** @return Whether or not you can use this resource as a source */ - default boolean isSource() { - return false; - } - - /** @return Whether or not you can use this resource as a sink */ - default boolean isSink() { - return false; - } - - /** @return Whether or not you can use this resource as a function/transform. */ - default boolean isTransform() { - return false; - } -} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResourceIdentifier.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResourceIdentifier.java deleted file mode 100644 index 31b76b470fc1..000000000000 --- a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResourceIdentifier.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.catalog; - -import java.util.Arrays; - -public class CatalogResourceIdentifier { - private String[] namespace; - private String name; - - public CatalogResourceIdentifier(String... name) { - if (name.length == 1) { - this.name = name[0]; - this.namespace = new String[0]; - } else { - this.name = name[name.length - 1]; - this.namespace = Arrays.copyOf(name, name.length - 1); - } - } - - public static CatalogResourceIdentifier of(String... name) { - return new CatalogResourceIdentifier(name); - } -} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSinkResource.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSinkResource.java deleted file mode 100644 index d3a7ccaa71da..000000000000 --- a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSinkResource.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * 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.catalog; - -public interface CatalogSinkResource extends CatalogResource { - - @Override - default boolean isSink() { - return true; - } -} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSourceResource.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSourceResource.java deleted file mode 100644 index b892a307a7ee..000000000000 --- a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSourceResource.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * 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.catalog; - -public interface CatalogSourceResource extends CatalogResource { - - @Override - default boolean isSource() { - return true; - } -} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogTableResource.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogTableResource.java deleted file mode 100644 index c75084635816..000000000000 --- a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogTableResource.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * 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.catalog; - -public interface CatalogTableResource extends CatalogSinkResource, CatalogSourceResource {} diff --git a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/package-info.java b/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/package-info.java deleted file mode 100644 index 08d120cdd5c4..000000000000 --- a/sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/package-info.java +++ /dev/null @@ -1 +0,0 @@ -package org.apache.beam.sdk.io.catalog; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AppendFilesToTables.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AppendFilesToTables.java new file mode 100644 index 000000000000..468bdcf5790d --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AppendFilesToTables.java @@ -0,0 +1,102 @@ +/* + * 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.io.iceberg; + +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; + +class AppendFilesToTables + extends PTransform, PCollection>> { + + private final IcebergCatalogConfig catalogConfig; + + AppendFilesToTables(IcebergCatalogConfig catalogConfig) { + this.catalogConfig = catalogConfig; + } + + @Override + public PCollection> expand(PCollection writtenFiles) { + + // Apply any sharded writes and flatten everything for catalog updates + return writtenFiles + .apply( + "Key metadata updates by table", + WithKeys.of( + new SerializableFunction() { + @Override + public String apply(FileWriteResult input) { + return input.getTableIdentifier().toString(); + } + })) + .apply("Group metadata updates by table", GroupByKey.create()) + .apply( + "Append metadata updates to tables", + ParDo.of(new AppendFilesToTablesDoFn(catalogConfig))) + .setCoder(KvCoder.of(StringUtf8Coder.of(), SerializableCoder.of(Snapshot.class))); + } + + private static class AppendFilesToTablesDoFn + extends DoFn>, KV> { + + private final IcebergCatalogConfig catalogConfig; + + private transient @MonotonicNonNull Catalog catalog; + + private AppendFilesToTablesDoFn(IcebergCatalogConfig catalogConfig) { + this.catalogConfig = catalogConfig; + } + + private Catalog getCatalog() { + if (catalog == null) { + catalog = catalogConfig.catalog(); + } + return catalog; + } + + @ProcessElement + public void processElement( + @Element KV> element, + OutputReceiver> out, + BoundedWindow window) { + Table table = getCatalog().loadTable(TableIdentifier.parse(element.getKey())); + AppendFiles update = table.newAppend(); + for (FileWriteResult writtenFile : element.getValue()) { + update.appendFile(writtenFile.getDataFile()); + } + update.commit(); + out.outputWithTimestamp( + KV.of(element.getKey(), table.currentSnapshot()), window.maxTimestamp()); + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AssignDestinations.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AssignDestinations.java new file mode 100644 index 000000000000..a9c4e82a7767 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AssignDestinations.java @@ -0,0 +1,65 @@ +/* + * 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.io.iceberg; + +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; + +/** + * Assigns the destination metadata for each input record. + * + *

The output record will have the format { dest: ..., data: ...} where the dest field has the + * assigned metadata and the data field has the original row. + */ +class AssignDestinations extends PTransform, PCollection> { + + private DynamicDestinations dynamicDestinations; + + public AssignDestinations(DynamicDestinations dynamicDestinations) { + this.dynamicDestinations = dynamicDestinations; + } + + @Override + public PCollection expand(PCollection input) { + + final Schema inputSchema = input.getSchema(); + final Schema outputSchema = + Schema.builder() + .addRowField("data", inputSchema) + .addRowField("dest", dynamicDestinations.getMetadataSchema()) + .build(); + + return input + .apply( + ParDo.of( + new DoFn() { + @ProcessElement + public void processElement(@Element Row data, OutputReceiver out) { + out.output( + Row.withSchema(outputSchema) + .addValues(data, dynamicDestinations.assignDestinationMetadata(data)) + .build()); + } + })) + .setRowSchema(outputSchema); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/CombinedScanReader.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/CombinedScanReader.java deleted file mode 100644 index 74ec62af66f8..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/CombinedScanReader.java +++ /dev/null @@ -1,200 +0,0 @@ -/* - * 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.io.iceberg; - -import java.io.IOException; -import java.util.LinkedList; -import java.util.NoSuchElementException; -import javax.annotation.Nullable; -import org.apache.beam.io.iceberg.util.SchemaHelper; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.values.Row; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.Table; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.avro.DataReader; -import org.apache.iceberg.data.orc.GenericOrcReader; -import org.apache.iceberg.data.parquet.GenericParquetReaders; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@SuppressWarnings("all") -public class CombinedScanReader extends BoundedSource.BoundedReader { - private static final Logger LOG = LoggerFactory.getLogger(CombinedScanReader.class); - - IcebergBoundedSource source; - - @Nullable CombinedScanTask task; - - @Nullable Schema schema; - - transient @Nullable org.apache.iceberg.Schema project; - - transient @Nullable FileIO io; - transient @Nullable EncryptionManager encryptionManager; - - transient @Nullable InputFilesDecryptor decryptor; - - transient LinkedList files = new LinkedList<>(); - - transient CloseableIterator baseIter = null; - - transient Record current; - - public CombinedScanReader( - IcebergBoundedSource source, @Nullable CombinedScanTask task, @Nullable Schema schema) { - this.source = source; - this.task = task; - this.schema = schema; - if (this.schema != null) { - project = SchemaHelper.convert(schema); - } - } - - @Override - public boolean start() throws IOException { - if (task == null) { - return false; - } - - Table table = source.table(); - - io = table.io(); - encryptionManager = table.encryption(); - decryptor = new InputFilesDecryptor(task, io, encryptionManager); - - files.addAll(task.files()); - - return advance(); - } - - @Override - public boolean advance() throws IOException { - do { - // If our current iterator is working... do that. - if (baseIter != null && baseIter.hasNext()) { - current = baseIter.next(); - return true; - } - - // Close out the current iterator and try to open a new one - if (baseIter != null) { - baseIter.close(); - baseIter = null; - } - - LOG.info("Trying to open new file."); - FileScanTask fileTask = null; - while (files.size() > 0 && fileTask == null) { - fileTask = files.removeFirst(); - if (fileTask.isDataTask()) { - LOG.error("{} is a DataTask. Skipping.", fileTask.toString()); - fileTask = null; - } - } - - // We have a new file to start reading - if (fileTask != null) { - DataFile file = fileTask.file(); - InputFile input = decryptor.getInputFile(fileTask); - - CloseableIterable iterable = null; - switch (file.format()) { - case ORC: - LOG.info("Preparing ORC input"); - iterable = - ORC.read(input) - .project(project) - .createReaderFunc( - fileSchema -> GenericOrcReader.buildReader(project, fileSchema)) - .filter(fileTask.residual()) - .build(); - break; - case PARQUET: - LOG.info("Preparing Parquet input."); - iterable = - Parquet.read(input) - .project(project) - .createReaderFunc( - fileSchema -> GenericParquetReaders.buildReader(project, fileSchema)) - .filter(fileTask.residual()) - .build(); - break; - case AVRO: - LOG.info("Preparing Avro input."); - iterable = - Avro.read(input).project(project).createReaderFunc(DataReader::create).build(); - break; - default: - throw new UnsupportedOperationException("Cannot read format: " + file.format()); - } - - if (iterable != null) { - baseIter = iterable.iterator(); - } - } else { - LOG.info("We have exhausted all available files in this CombinedScanTask"); - } - - } while (baseIter != null); - return false; - } - - private Row convert(Record record) { - Row.Builder b = Row.withSchema(schema); - for (int i = 0; i < schema.getFieldCount(); i++) { - // TODO: A lot obviously - b.addValue(record.getField(schema.getField(i).getName())); - } - return b.build(); - } - - @Override - public Row getCurrent() throws NoSuchElementException { - if (current == null) { - throw new NoSuchElementException(); - } - return convert(current); - } - - @Override - public void close() throws IOException { - if (baseIter != null) { - baseIter.close(); - } - files.clear(); - io.close(); - } - - @Override - public BoundedSource getCurrentSource() { - return source; - } -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/DynamicDestinations.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/DynamicDestinations.java index 8032e5c1fbbb..a395086403f6 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/DynamicDestinations.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/DynamicDestinations.java @@ -17,147 +17,20 @@ */ package org.apache.beam.io.iceberg; -import static org.apache.beam.sdk.values.TypeDescriptors.extractFromTypeParameters; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; - import java.io.Serializable; -import java.util.List; -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.sdk.values.TypeDescriptors; -import org.apache.beam.sdk.values.ValueInSingleWindow; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; - -@SuppressWarnings("all") // TODO: Remove this once development is stable. -public abstract class DynamicDestinations implements Serializable { - - interface SideInputAccessor { - SideInputT sideInput(PCollectionView view); - } - - private transient SideInputAccessor sideInputAccessor; - private transient PipelineOptions options; - - static class ProcessContextSideInputAccessor implements SideInputAccessor { - private DoFn.ProcessContext processContext; - - public ProcessContextSideInputAccessor(DoFn.ProcessContext processContext) { - this.processContext = processContext; - } - - @Override - public SideInputT sideInput(PCollectionView view) { - return processContext.sideInput(view); - } - } - - public PipelineOptions getOptions() { - return options; - } - - public List> getSideInputs() { - return Lists.newArrayList(); - } +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.catalog.TableIdentifier; - protected final SideInputT sideInput(PCollectionView view) { - checkState( - getSideInputs().contains(view), - "View %s not declared in getSideInputs() (%s)", - view, - getSideInputs()); - if (sideInputAccessor == null) { - throw new IllegalStateException("sideInputAccessor (transient field) is null"); - } - return sideInputAccessor.sideInput(view); - } +public interface DynamicDestinations extends Serializable { - void setSideInputProcessContext(DoFn.ProcessContext context) { - this.sideInputAccessor = new ProcessContextSideInputAccessor(context); - this.options = context.getPipelineOptions(); - } + Schema getMetadataSchema(); - public abstract DestinationT getDestination(ValueInSingleWindow element); + Row assignDestinationMetadata(Row data); - public Coder getDestinationCoder() { - return null; - } - - public abstract Table getTable(DestinationT destination); - - public abstract Schema getSchema(DestinationT destination); - - public abstract PartitionSpec getPartitionSpec(DestinationT destination); - - public abstract FileFormat getFileFormat(DestinationT destination); - - Coder getDestinationCoderWithDefault(CoderRegistry registry) - throws CannotProvideCoderException { - Coder destinationCoder = getDestinationCoder(); - if (destinationCoder != null) { - return destinationCoder; - } - TypeDescriptor descriptor = - extractFromTypeParameters( - this, - DynamicDestinations.class, - new TypeDescriptors.TypeVariableExtractor< - DynamicDestinations, DestinationT>() {}); - try { - return registry.getCoder(descriptor); - } catch (CannotProvideCoderException e) { - throw new CannotProvideCoderException( - "Failed to infer coder for DestinationT from type " - + descriptor - + ", please provide it explicitly by overriding getDestinationCoder()", - e); - } - } - - public static class StaticTableDestination - extends DynamicDestinations { - - final Iceberg.Table table; - - public StaticTableDestination(Iceberg.Table table) { - this.table = table; - } - - @Override - public String getDestination(ValueInSingleWindow element) { - return table.table().name(); - } - - @Override - public Table getTable(String destination) { - return table.table(); - } - - @Override - public Schema getSchema(String destination) { - return getTable(destination).schema(); - } - - @Override - public PartitionSpec getPartitionSpec(String destination) { - return getTable(destination).spec(); - } - - @Override - public FileFormat getFileFormat(String destination) { - return FileFormat.PARQUET; - } - } + IcebergDestination instantiateDestination(Row dest); - public static StaticTableDestination constant(Iceberg.Table table) { - return new StaticTableDestination<>(table); + static DynamicDestinations singleTable(TableIdentifier tableId) { + return new OneTableDynamicDestinations(tableId); } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/FileWriteResult.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/FileWriteResult.java new file mode 100644 index 000000000000..09645cdd827b --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/FileWriteResult.java @@ -0,0 +1,203 @@ +/* + * 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.io.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; + +import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.avro.Schema; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CoderProvider; +import org.apache.beam.sdk.coders.CoderProviders; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.StructuredCoder; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.avro.AvroEncoderUtil; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Types; +import org.checkerframework.checker.nullness.qual.Nullable; + +@AutoValue +@DefaultCoder(FileWriteResult.FileWriteResultCoder.class) +abstract class FileWriteResult { + public abstract TableIdentifier getTableIdentifier(); + + public abstract PartitionSpec getPartitionSpec(); + + public abstract DataFile getDataFile(); + + public static Builder builder() { + return new AutoValue_FileWriteResult.Builder(); + } + + @AutoValue.Builder + abstract static class Builder { + public abstract Builder setTableIdentifier(TableIdentifier tableId); + + public abstract Builder setPartitionSpec(PartitionSpec partitionSpec); + + public abstract Builder setDataFile(DataFile dataFiles); + + public abstract FileWriteResult build(); + } + + public static class FileWriteResultCoder extends StructuredCoder { + private static final FileWriteResultCoder SINGLETON = new FileWriteResultCoder(); + + private static final Coder tableIdentifierCoder = StringUtf8Coder.of(); + private static final Coder partitionSpecCoder = + SerializableCoder.of(PartitionSpec.class); + private static final Coder dataFileBytesCoder = ByteArrayCoder.of(); + + private static Schema getDataFileAvroSchema(FileWriteResult fileWriteResult) { + Types.StructType partitionType = fileWriteResult.getPartitionSpec().partitionType(); + Types.StructType dataFileStruct = DataFile.getType(partitionType); + Map dataFileNames = + ImmutableMap.of( + dataFileStruct, "org.apache.iceberg.GenericDataFile", + partitionType, "org.apache.iceberg.PartitionData"); + return AvroSchemaUtil.convert(dataFileStruct, dataFileNames); + } + + @Override + public void encode(FileWriteResult value, OutputStream outStream) + throws CoderException, IOException { + tableIdentifierCoder.encode(value.getTableIdentifier().toString(), outStream); + partitionSpecCoder.encode(value.getPartitionSpec(), outStream); + dataFileBytesCoder.encode( + AvroEncoderUtil.encode(value.getDataFile(), getDataFileAvroSchema(value)), outStream); + } + + @Override + public FileWriteResult decode(InputStream inStream) throws CoderException, IOException { + TableIdentifier tableId = TableIdentifier.parse(tableIdentifierCoder.decode(inStream)); + PartitionSpec partitionSpec = partitionSpecCoder.decode(inStream); + DataFile dataFile = + checkArgumentNotNull( + AvroEncoderUtil.decode(dataFileBytesCoder.decode(inStream)), + "Decoding of dataFile resulted in null"); + return FileWriteResult.builder() + .setTableIdentifier(tableId) + .setDataFile(dataFile) + .setPartitionSpec(partitionSpec) + .build(); + } + + @Override + public List> getCoderArguments() { + return Collections.emptyList(); + } + + @Override + public Object structuralValue(FileWriteResult fileWriteResult) { + return new FileWriteResultDeepEqualityWrapper(fileWriteResult); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException {} + + @Override + public TypeDescriptor getEncodedTypeDescriptor() { + return TypeDescriptor.of(FileWriteResult.class); + } + + public static FileWriteResultCoder of() { + return SINGLETON; + } + + @SuppressWarnings("unused") // used via `DefaultCoder` annotation + public static CoderProvider getCoderProvider() { + return CoderProviders.forCoder( + TypeDescriptor.of(FileWriteResult.class), FileWriteResultCoder.of()); + } + } + + private static class FileWriteResultDeepEqualityWrapper { + private final FileWriteResult fileWriteResult; + + private FileWriteResultDeepEqualityWrapper(FileWriteResult fileWriteResult) { + this.fileWriteResult = fileWriteResult; + } + + @Override + public boolean equals(@Nullable Object obj) { + if (obj == this) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof FileWriteResultDeepEqualityWrapper)) { + return false; + } + FileWriteResultDeepEqualityWrapper other = (FileWriteResultDeepEqualityWrapper) obj; + + return Objects.equals( + fileWriteResult.getTableIdentifier(), other.fileWriteResult.getTableIdentifier()) + && Objects.equals( + fileWriteResult.getPartitionSpec(), other.fileWriteResult.getPartitionSpec()) + && dataFilesEqual(fileWriteResult.getDataFile(), other.fileWriteResult.getDataFile()); + } + + private boolean dataFilesEqual(DataFile first, DataFile second) { + return Objects.equals(first.pos(), second.pos()) + && first.specId() == second.specId() + && Objects.equals(first.content(), second.content()) + && Objects.equals(first.path(), second.path()) + && Objects.equals(first.format(), second.format()) + && Objects.equals(first.partition(), second.partition()) + && first.recordCount() == second.recordCount() + && first.fileSizeInBytes() == second.fileSizeInBytes() + && Objects.equals(first.columnSizes(), second.columnSizes()) + && Objects.equals(first.valueCounts(), second.valueCounts()) + && Objects.equals(first.nullValueCounts(), second.nullValueCounts()) + && Objects.equals(first.nanValueCounts(), second.nanValueCounts()) + && Objects.equals(first.lowerBounds(), second.lowerBounds()) + && Objects.equals(first.upperBounds(), second.upperBounds()) + && Objects.equals(first.keyMetadata(), second.keyMetadata()) + && Objects.equals(first.splitOffsets(), second.splitOffsets()) + && Objects.equals(first.equalityFieldIds(), second.equalityFieldIds()) + && Objects.equals(first.sortOrderId(), second.sortOrderId()) + && Objects.equals(first.dataSequenceNumber(), second.dataSequenceNumber()) + && Objects.equals(first.fileSequenceNumber(), second.fileSequenceNumber()); + } + + @Override + public int hashCode() { + return Objects.hash( + fileWriteResult.getTableIdentifier(), + fileWriteResult.getPartitionSpec(), + fileWriteResult.getDataFile()); + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/Iceberg.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/Iceberg.java deleted file mode 100644 index 58dc5a6287e9..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/Iceberg.java +++ /dev/null @@ -1,412 +0,0 @@ -/* - * 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.io.iceberg; - -import com.google.auto.value.AutoValue; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import java.io.Serializable; -import java.util.List; -import java.util.Map; -import javax.annotation.Nullable; -import org.apache.beam.io.iceberg.util.PropertyBuilder; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.SerializableBiFunction; -import org.apache.beam.sdk.transforms.SerializableFunctions; -import org.apache.beam.sdk.values.PCollection; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.expressions.Expression; - -@SuppressWarnings("all") // TODO: Remove this once development is stable. -public class Iceberg { - - public static String DEFAULT_CATALOG_NAME = "default"; - - public enum ScanType { - TABLE, - BATCH - } - - public enum WriteFormat { - AVRO, - PARQUET, - ORC - } - - public static Catalog catalog(String name) { - return Catalog.builder().name(name).build(); - } - - public static Catalog catalog() { - return catalog(DEFAULT_CATALOG_NAME); - } - - @AutoValue - public abstract static class Scan implements Serializable { - - public abstract ScanType getType(); - - public abstract Catalog getCatalog(); - - public abstract ImmutableList getTable(); - - public abstract Schema getSchema(); - - public abstract @Nullable Expression getFilter(); - - public abstract @Nullable Boolean getCaseSensitive(); - - public abstract ImmutableMap getOptions(); - - public abstract @Nullable Long getSnapshot(); - - public abstract @Nullable Long getTimestamp(); - - public abstract @Nullable Long getFromSnapshotInclusive(); - - public abstract @Nullable String getFromSnapshotRefInclusive(); - - public abstract @Nullable Long getFromSnapshotExclusive(); - - public abstract @Nullable String getFromSnapshotRefExclusive(); - - public abstract @Nullable Long getToSnapshot(); - - public abstract @Nullable String getToSnapshotRef(); - - public abstract @Nullable String getTag(); - - public abstract @Nullable String getBranch(); - - public static Scan.Builder builder() { - return new AutoValue_Iceberg_Scan.Builder() - .type(ScanType.TABLE) - .filter(null) - .caseSensitive(null) - .options(ImmutableMap.of()) - .snapshot(null) - .timestamp(null) - .fromSnapshotInclusive(null) - .fromSnapshotRefInclusive(null) - .fromSnapshotExclusive(null) - .fromSnapshotRefExclusive(null) - .toSnapshot(null) - .toSnapshotRef(null) - .tag(null) - .branch(null); - } - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder type(ScanType type); - - public abstract Builder catalog(Catalog catalog); - - public abstract Builder table(ImmutableList table); - - public Builder table(String... table) { - return table(ImmutableList.copyOf(table)); - } - - public abstract Builder schema(Schema schema); - - public abstract Builder filter(@Nullable Expression filter); - - public abstract Builder caseSensitive(@Nullable Boolean caseSensitive); - - public abstract Builder options(ImmutableMap options); - - public abstract Builder snapshot(@Nullable Long snapshot); - - public abstract Builder timestamp(@Nullable Long timestamp); - - public abstract Builder fromSnapshotInclusive(@Nullable Long fromInclusive); - - public abstract Builder fromSnapshotRefInclusive(@Nullable String ref); - - public abstract Builder fromSnapshotExclusive(@Nullable Long fromExclusive); - - public abstract Builder fromSnapshotRefExclusive(@Nullable String ref); - - public abstract Builder toSnapshot(@Nullable Long snapshot); - - public abstract Builder toSnapshotRef(@Nullable String ref); - - public abstract Builder tag(@Nullable String tag); - - public abstract Builder branch(@Nullable String branch); - - public abstract Scan build(); - } - } - - @AutoValue - public abstract static class Catalog implements Serializable { - - public abstract String getName(); - - /* Core Properties */ - public abstract @Nullable String getIcebergCatalogType(); - - public abstract @Nullable String getCatalogImplementation(); - - public abstract @Nullable String getFileIOImplementation(); - - public abstract @Nullable String getWarehouseLocation(); - - public abstract @Nullable String getMetricsReporterImplementation(); - - /* Caching */ - public abstract boolean getCacheEnabled(); - - public abstract boolean getCacheCaseSensitive(); - - public abstract long getCacheExpirationIntervalMillis(); - - public abstract boolean getIOManifestCacheEnabled(); - - public abstract long getIOManifestCacheExpirationIntervalMillis(); - - public abstract long getIOManifestCacheMaxTotalBytes(); - - public abstract long getIOManifestCacheMaxContentLength(); - - public abstract @Nullable String getUri(); - - public abstract int getClientPoolSize(); - - public abstract long getClientPoolEvictionIntervalMs(); - - public abstract @Nullable String getClientPoolCacheKeys(); - - public abstract @Nullable String getLockImplementation(); - - public abstract long getLockHeartbeatIntervalMillis(); - - public abstract long getLockHeartbeatTimeoutMillis(); - - public abstract int getLockHeartbeatThreads(); - - public abstract long getLockAcquireIntervalMillis(); - - public abstract long getLockAcquireTimeoutMillis(); - - public abstract @Nullable String getAppIdentifier(); - - public abstract @Nullable String getUser(); - - public abstract long getAuthSessionTimeoutMillis(); - - public abstract @Nullable Configuration getConfiguration(); - - public static Catalog.Builder builder() { - return new AutoValue_Iceberg_Catalog.Builder() - .icebergCatalogType(null) - .catalogImplementation(null) - .fileIOImplementation(null) - .warehouseLocation(null) - .metricsReporterImplementation(null) // TODO: Set this to our implementation - .cacheEnabled(CatalogProperties.CACHE_ENABLED_DEFAULT) - .cacheCaseSensitive(CatalogProperties.CACHE_CASE_SENSITIVE_DEFAULT) - .cacheExpirationIntervalMillis(CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_DEFAULT) - .iOManifestCacheEnabled(CatalogProperties.IO_MANIFEST_CACHE_ENABLED_DEFAULT) - .iOManifestCacheExpirationIntervalMillis( - CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_DEFAULT) - .iOManifestCacheMaxTotalBytes(CatalogProperties.IO_MANIFEST_CACHE_MAX_TOTAL_BYTES_DEFAULT) - .iOManifestCacheMaxContentLength( - CatalogProperties.IO_MANIFEST_CACHE_MAX_CONTENT_LENGTH_DEFAULT) - .uri(null) - .clientPoolSize(CatalogProperties.CLIENT_POOL_SIZE_DEFAULT) - .clientPoolEvictionIntervalMs( - CatalogProperties.CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS_DEFAULT) - .clientPoolCacheKeys(null) - .lockImplementation(null) - .lockHeartbeatIntervalMillis(CatalogProperties.LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT) - .lockHeartbeatTimeoutMillis(CatalogProperties.LOCK_HEARTBEAT_TIMEOUT_MS_DEFAULT) - .lockHeartbeatThreads(CatalogProperties.LOCK_HEARTBEAT_THREADS_DEFAULT) - .lockAcquireIntervalMillis(CatalogProperties.LOCK_ACQUIRE_INTERVAL_MS_DEFAULT) - .lockAcquireTimeoutMillis(CatalogProperties.LOCK_HEARTBEAT_TIMEOUT_MS_DEFAULT) - .appIdentifier(null) - .user(null) - .authSessionTimeoutMillis(CatalogProperties.AUTH_SESSION_TIMEOUT_MS_DEFAULT) - .configuration(null); - } - - public ImmutableMap properties() { - return new PropertyBuilder() - .put(CatalogUtil.ICEBERG_CATALOG_TYPE, getIcebergCatalogType()) - .put(CatalogProperties.CATALOG_IMPL, getCatalogImplementation()) - .put(CatalogProperties.FILE_IO_IMPL, getFileIOImplementation()) - .put(CatalogProperties.WAREHOUSE_LOCATION, getWarehouseLocation()) - .put(CatalogProperties.METRICS_REPORTER_IMPL, getMetricsReporterImplementation()) - .put(CatalogProperties.CACHE_ENABLED, getCacheEnabled()) - .put(CatalogProperties.CACHE_CASE_SENSITIVE, getCacheCaseSensitive()) - .put(CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS, getCacheExpirationIntervalMillis()) - .build(); - } - - public org.apache.iceberg.catalog.Catalog catalog() { - Configuration conf = getConfiguration(); - if (conf == null) { - conf = new Configuration(); - } - return CatalogUtil.buildIcebergCatalog(getName(), properties(), conf); - } - - public Table.Builder table() { - return new AutoValue_Iceberg_Table.Builder().catalog(this); - } - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder name(String name); - - /* Core Properties */ - public abstract Builder icebergCatalogType(@Nullable String icebergType); - - public abstract Builder catalogImplementation(@Nullable String catalogImpl); - - public abstract Builder fileIOImplementation(@Nullable String fileIOImpl); - - public abstract Builder warehouseLocation(@Nullable String warehouse); - - public abstract Builder metricsReporterImplementation(@Nullable String metricsImpl); - - /* Caching */ - public abstract Builder cacheEnabled(boolean cacheEnabled); - - public abstract Builder cacheCaseSensitive(boolean cacheCaseSensitive); - - public abstract Builder cacheExpirationIntervalMillis(long expiration); - - public abstract Builder iOManifestCacheEnabled(boolean enabled); - - public abstract Builder iOManifestCacheExpirationIntervalMillis(long expiration); - - public abstract Builder iOManifestCacheMaxTotalBytes(long bytes); - - public abstract Builder iOManifestCacheMaxContentLength(long length); - - public abstract Builder uri(@Nullable String uri); - - public abstract Builder clientPoolSize(int size); - - public abstract Builder clientPoolEvictionIntervalMs(long interval); - - public abstract Builder clientPoolCacheKeys(@Nullable String keys); - - public abstract Builder lockImplementation(@Nullable String lockImplementation); - - public abstract Builder lockHeartbeatIntervalMillis(long interval); - - public abstract Builder lockHeartbeatTimeoutMillis(long timeout); - - public abstract Builder lockHeartbeatThreads(int threads); - - public abstract Builder lockAcquireIntervalMillis(long interval); - - public abstract Builder lockAcquireTimeoutMillis(long timeout); - - public abstract Builder appIdentifier(@Nullable String id); - - public abstract Builder user(@Nullable String user); - - public abstract Builder authSessionTimeoutMillis(long timeout); - - public abstract Builder configuration(@Nullable Configuration conf); - - public abstract Catalog build(); - - public Builder withProperties(Map properties) { - return this; - } - } - } - - @AutoValue - public abstract static class Table implements Serializable { - - public abstract @Nullable Catalog catalog(); - - public abstract @Nullable List tablePath(); - - public TableIdentifier identifier() { - return TableIdentifier.of(tablePath().toArray(new String[0])); - } - - public org.apache.iceberg.Table table() { - return catalog().catalog().loadTable(identifier()); - } - - @AutoValue.Builder - public abstract static class Builder { - - public abstract Builder catalog(Catalog catalog); - - public abstract Builder tablePath(List tablePath); - - public abstract Table build(); - } - } - - public static class Write - extends PTransform, IcebergWriteResult> { - - private final DynamicDestinations dynamicDestinations; - private final Catalog catalog; - - private final SerializableBiFunction toRecord; - - public Write( - Catalog catalog, - DynamicDestinations dynamicDestinations, - SerializableBiFunction toRecord) { - this.catalog = catalog; - this.dynamicDestinations = dynamicDestinations; - this.toRecord = toRecord; - } - - @Override - public IcebergWriteResult expand(PCollection input) { - try { - return input - .apply( - "Set Output Location", - new PrepareWrite( - dynamicDestinations, SerializableFunctions.identity(), input.getCoder())) - .apply( - "Dynamic Write", - new IcebergSink( - dynamicDestinations, - dynamicDestinations.getDestinationCoderWithDefault( - input.getPipeline().getCoderRegistry()), - RecordWriterFactory.tableRecords(toRecord, dynamicDestinations), - TableFactory.forCatalog(catalog))); - } catch (Exception e) { - RuntimeException e1 = new RuntimeException("Unable to expand transforms"); - e1.addSuppressed(e); - throw e1; - } - } - } -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergBoundedSource.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergBoundedSource.java deleted file mode 100644 index 0c6028c85487..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergBoundedSource.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * 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.io.iceberg; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import javax.annotation.Nullable; -import org.apache.beam.io.iceberg.util.SchemaHelper; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.RowCoder; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.values.Row; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableScan; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.io.CloseableIterable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@SuppressWarnings("all") // TODO: Remove this once development is stable. -public class IcebergBoundedSource extends BoundedSource { - private static final Logger LOG = LoggerFactory.getLogger(IcebergBoundedSource.class); - - private @Nullable CombinedScanTask task; - private Iceberg.Scan scan; - - public IcebergBoundedSource(Iceberg.Scan scan, @Nullable CombinedScanTask task) { - this.task = task; - this.scan = scan; - } - - public IcebergBoundedSource(Iceberg.Scan scan) { - this(scan, null); - } - - public @Nullable Catalog catalog() { - return scan.getCatalog().catalog(); - } - - public @Nullable Table table() { - Catalog catalog = catalog(); - if (catalog != null) { - return catalog.loadTable( - TableIdentifier.of(scan.getTable().toArray(new String[scan.getTable().size()]))); - } else { - return null; - } - } - - @Override - public List> split( - long desiredBundleSizeBytes, PipelineOptions options) throws Exception { - ArrayList tasks = new ArrayList<>(); - Table table = table(); - if (table != null) { - - switch (scan.getType()) { - case TABLE: - // Override the split size with our desired size - TableScan tableScan = table.newScan(); - - if (desiredBundleSizeBytes > 0) { - tableScan = tableScan.option(TableProperties.SPLIT_SIZE, "" + desiredBundleSizeBytes); - } - - // Always project to our destination schema - tableScan = tableScan.project(SchemaHelper.convert(scan.getSchema())); - - if (scan.getFilter() != null) { - tableScan = tableScan.filter(scan.getFilter()); - } - if (scan.getCaseSensitive() != null) { - tableScan = tableScan.caseSensitive(scan.getCaseSensitive()); - } - if (scan.getSnapshot() != null) { - tableScan = tableScan.useSnapshot(scan.getSnapshot()); - } - if (scan.getBranch() != null) { - tableScan = tableScan.useRef(scan.getBranch()); - } else if (scan.getTag() != null) { - tableScan = tableScan.useRef(scan.getTag()); - } - try (CloseableIterable t = tableScan.planTasks()) { - for (CombinedScanTask c : t) { - tasks.add(new IcebergBoundedSource(scan, c)); - } - } catch (IOException e) { - throw new RuntimeException(e); - } - break; - case BATCH: - // TODO: Add batch scan - break; - } - } - return tasks; - } - - @Override - public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { - if (task == null) { - return 0; - } else { - return task.sizeBytes(); - } - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - } - - @Override - public Coder getOutputCoder() { - return RowCoder.of(scan.getSchema()); - } - - @Override - public BoundedReader createReader(PipelineOptions options) throws IOException { - return new CombinedScanReader(this, task, scan.getSchema()); - } -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergCatalogConfig.java new file mode 100644 index 000000000000..06a29ac14652 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergCatalogConfig.java @@ -0,0 +1,233 @@ +/* + * 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.io.iceberg; + +import com.google.auto.value.AutoValue; +import java.io.Serializable; +import javax.annotation.Nullable; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.checkerframework.dataflow.qual.Pure; + +@AutoValue +public abstract class IcebergCatalogConfig implements Serializable { + + @Pure + public abstract String getName(); + + /* Core Properties */ + @Pure + public abstract @Nullable String getIcebergCatalogType(); + + @Pure + public abstract @Nullable String getCatalogImplementation(); + + @Pure + public abstract @Nullable String getFileIOImplementation(); + + @Pure + public abstract @Nullable String getWarehouseLocation(); + + @Pure + public abstract @Nullable String getMetricsReporterImplementation(); + + /* Caching */ + @Pure + public abstract boolean getCacheEnabled(); + + @Pure + public abstract boolean getCacheCaseSensitive(); + + @Pure + public abstract long getCacheExpirationIntervalMillis(); + + @Pure + public abstract boolean getIOManifestCacheEnabled(); + + @Pure + public abstract long getIOManifestCacheExpirationIntervalMillis(); + + @Pure + public abstract long getIOManifestCacheMaxTotalBytes(); + + @Pure + public abstract long getIOManifestCacheMaxContentLength(); + + @Pure + public abstract @Nullable String getUri(); + + @Pure + public abstract int getClientPoolSize(); + + @Pure + public abstract long getClientPoolEvictionIntervalMs(); + + @Pure + public abstract @Nullable String getClientPoolCacheKeys(); + + @Pure + public abstract @Nullable String getLockImplementation(); + + @Pure + public abstract long getLockHeartbeatIntervalMillis(); + + @Pure + public abstract long getLockHeartbeatTimeoutMillis(); + + @Pure + public abstract int getLockHeartbeatThreads(); + + @Pure + public abstract long getLockAcquireIntervalMillis(); + + @Pure + public abstract long getLockAcquireTimeoutMillis(); + + @Pure + public abstract @Nullable String getAppIdentifier(); + + @Pure + public abstract @Nullable String getUser(); + + @Pure + public abstract long getAuthSessionTimeoutMillis(); + + @Pure + public abstract @Nullable Configuration getConfiguration(); + + @Pure + public static Builder builder() { + return new AutoValue_IcebergCatalogConfig.Builder() + .setIcebergCatalogType(null) + .setCatalogImplementation(null) + .setFileIOImplementation(null) + .setWarehouseLocation(null) + .setMetricsReporterImplementation(null) // TODO: Set this to our implementation + .setCacheEnabled(CatalogProperties.CACHE_ENABLED_DEFAULT) + .setCacheCaseSensitive(CatalogProperties.CACHE_CASE_SENSITIVE_DEFAULT) + .setCacheExpirationIntervalMillis(CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_DEFAULT) + .setIOManifestCacheEnabled(CatalogProperties.IO_MANIFEST_CACHE_ENABLED_DEFAULT) + .setIOManifestCacheExpirationIntervalMillis( + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_DEFAULT) + .setIOManifestCacheMaxTotalBytes( + CatalogProperties.IO_MANIFEST_CACHE_MAX_TOTAL_BYTES_DEFAULT) + .setIOManifestCacheMaxContentLength( + CatalogProperties.IO_MANIFEST_CACHE_MAX_CONTENT_LENGTH_DEFAULT) + .setUri(null) + .setClientPoolSize(CatalogProperties.CLIENT_POOL_SIZE_DEFAULT) + .setClientPoolEvictionIntervalMs( + CatalogProperties.CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS_DEFAULT) + .setClientPoolCacheKeys(null) + .setLockImplementation(null) + .setLockHeartbeatIntervalMillis(CatalogProperties.LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT) + .setLockHeartbeatTimeoutMillis(CatalogProperties.LOCK_HEARTBEAT_TIMEOUT_MS_DEFAULT) + .setLockHeartbeatThreads(CatalogProperties.LOCK_HEARTBEAT_THREADS_DEFAULT) + .setLockAcquireIntervalMillis(CatalogProperties.LOCK_ACQUIRE_INTERVAL_MS_DEFAULT) + .setLockAcquireTimeoutMillis(CatalogProperties.LOCK_HEARTBEAT_TIMEOUT_MS_DEFAULT) + .setAppIdentifier(null) + .setUser(null) + .setAuthSessionTimeoutMillis(CatalogProperties.AUTH_SESSION_TIMEOUT_MS_DEFAULT) + .setConfiguration(null); + } + + @Pure + public ImmutableMap properties() { + return new PropertyBuilder() + .put(CatalogUtil.ICEBERG_CATALOG_TYPE, getIcebergCatalogType()) + .put(CatalogProperties.CATALOG_IMPL, getCatalogImplementation()) + .put(CatalogProperties.FILE_IO_IMPL, getFileIOImplementation()) + .put(CatalogProperties.WAREHOUSE_LOCATION, getWarehouseLocation()) + .put(CatalogProperties.METRICS_REPORTER_IMPL, getMetricsReporterImplementation()) + .put(CatalogProperties.CACHE_ENABLED, getCacheEnabled()) + .put(CatalogProperties.CACHE_CASE_SENSITIVE, getCacheCaseSensitive()) + .put(CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS, getCacheExpirationIntervalMillis()) + .build(); + } + + public org.apache.iceberg.catalog.Catalog catalog() { + Configuration conf = getConfiguration(); + if (conf == null) { + conf = new Configuration(); + } + return CatalogUtil.buildIcebergCatalog(getName(), properties(), conf); + } + + @AutoValue.Builder + public abstract static class Builder { + + /* Core Properties */ + public abstract Builder setName(String name); + + public abstract Builder setIcebergCatalogType(@Nullable String icebergType); + + public abstract Builder setCatalogImplementation(@Nullable String catalogImpl); + + public abstract Builder setFileIOImplementation(@Nullable String fileIOImpl); + + public abstract Builder setWarehouseLocation(@Nullable String warehouse); + + public abstract Builder setMetricsReporterImplementation(@Nullable String metricsImpl); + + /* Caching */ + public abstract Builder setCacheEnabled(boolean cacheEnabled); + + public abstract Builder setCacheCaseSensitive(boolean cacheCaseSensitive); + + public abstract Builder setCacheExpirationIntervalMillis(long expiration); + + public abstract Builder setIOManifestCacheEnabled(boolean enabled); + + public abstract Builder setIOManifestCacheExpirationIntervalMillis(long expiration); + + public abstract Builder setIOManifestCacheMaxTotalBytes(long bytes); + + public abstract Builder setIOManifestCacheMaxContentLength(long length); + + public abstract Builder setUri(@Nullable String uri); + + public abstract Builder setClientPoolSize(int size); + + public abstract Builder setClientPoolEvictionIntervalMs(long interval); + + public abstract Builder setClientPoolCacheKeys(@Nullable String keys); + + public abstract Builder setLockImplementation(@Nullable String lockImplementation); + + public abstract Builder setLockHeartbeatIntervalMillis(long interval); + + public abstract Builder setLockHeartbeatTimeoutMillis(long timeout); + + public abstract Builder setLockHeartbeatThreads(int threads); + + public abstract Builder setLockAcquireIntervalMillis(long interval); + + public abstract Builder setLockAcquireTimeoutMillis(long timeout); + + public abstract Builder setAppIdentifier(@Nullable String id); + + public abstract Builder setUser(@Nullable String user); + + public abstract Builder setAuthSessionTimeoutMillis(long timeout); + + public abstract Builder setConfiguration(@Nullable Configuration conf); + + public abstract IcebergCatalogConfig build(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergDestination.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergDestination.java index 9f9790aed4ef..dcd69c87e9d0 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergDestination.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergDestination.java @@ -17,30 +17,49 @@ */ package org.apache.beam.io.iceberg; -import java.io.Serializable; -import org.apache.beam.sdk.io.fs.ResourceId; -import org.apache.iceberg.Schema; - -public class IcebergDestination implements Serializable { - - ResourceId resourceId; - String table; - Schema schema; - Iceberg.WriteFormat writeFormat; - - public IcebergDestination( - ResourceId resourceId, String table, Schema schema, Iceberg.WriteFormat writeFormat) { - this.resourceId = resourceId; - this.table = table; - this.schema = schema; - this.writeFormat = writeFormat; - } +import com.google.auto.value.AutoValue; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.catalog.TableIdentifier; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.dataflow.qual.Pure; + +@AutoValue +public abstract class IcebergDestination { + + /** + * The iceberg table identifier to write data to. This is relative to the catalog, which is + * presumed to be configured outside of this destination specification. + */ + @Pure + public abstract TableIdentifier getTableIdentifier(); + + /** File format for created files. */ + @Pure + public abstract FileFormat getFileFormat(); - public Iceberg.WriteFormat getWriteFormat() { - return writeFormat; + /** + * Metadata and constraints for creating a new table, if it must be done dynamically. + * + *

If null, dynamic table creation will fail, and this should be disallowed at the top level + * configuration. + */ + @Pure + public abstract @Nullable IcebergTableCreateConfig getTableCreateConfig(); + + @Pure + public static Builder builder() { + return new AutoValue_IcebergDestination.Builder(); } - public Schema getSchema() { - return schema; + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setTableIdentifier(TableIdentifier tableId); + + public abstract Builder setFileFormat(FileFormat fileFormat); + + public abstract Builder setTableCreateConfig(@Nullable IcebergTableCreateConfig createConfig); + + @Pure + public abstract IcebergDestination build(); } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java new file mode 100644 index 000000000000..4b9e3102a02e --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergIO.java @@ -0,0 +1,50 @@ +/* + * 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.io.iceberg; + +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; + +public class IcebergIO { + + public static WriteRows writeToDynamicDestinations( + IcebergCatalogConfig catalog, DynamicDestinations dynamicDestinations) { + return new WriteRows(catalog, dynamicDestinations); + } + + static class WriteRows extends PTransform, IcebergWriteResult> { + + private final IcebergCatalogConfig catalog; + private final DynamicDestinations dynamicDestinations; + + private WriteRows(IcebergCatalogConfig catalog, DynamicDestinations dynamicDestinations) { + this.catalog = catalog; + this.dynamicDestinations = dynamicDestinations; + } + + @Override + public IcebergWriteResult expand(PCollection input) { + + return input + .apply("Set Destination Metadata", new AssignDestinations(dynamicDestinations)) + .apply( + "Write Rows to Destinations", new WriteToDestinations(catalog, dynamicDestinations)); + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergSink.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergSink.java deleted file mode 100644 index f8ef8721161e..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergSink.java +++ /dev/null @@ -1,229 +0,0 @@ -/* - * 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.io.iceberg; - -import com.google.common.collect.ImmutableList; -import java.util.UUID; -import org.apache.beam.io.iceberg.WriteBundlesToFiles.Result; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.IterableCoder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.SerializableCoder; -import org.apache.beam.sdk.coders.ShardedKeyCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; -import org.apache.beam.sdk.transforms.windowing.GlobalWindows; -import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionList; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.ShardedKey; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.commons.lang.NotImplementedException; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.log4j.Logger; - -public class IcebergSink - extends PTransform>, IcebergWriteResult> { - - private static final Logger LOG = Logger.getLogger(IcebergSink.class); - - @VisibleForTesting static final int DEFAULT_MAX_WRITERS_PER_BUNDLE = 20; - @VisibleForTesting static final int DEFAULT_MAX_FILES_PER_PARTITION = 10_000; - @VisibleForTesting static final long DEFAULT_MAX_BYTES_PER_PARTITION = 10L * (1L << 40); // 10TB - static final long DEFAULT_MAX_BYTES_PER_FILE = (1L << 40); // 1TB - static final int DEFAULT_NUM_FILE_SHARDS = 0; - static final int FILE_TRIGGERING_RECORD_COUNT = 50_000; - - final DynamicDestinations dynamicDestinations; - final Coder destinationCoder; - - final RecordWriterFactory recordWriterFactory; - final TableFactory tableFactory; - - boolean triggered; - - public IcebergSink( - DynamicDestinations dynamicDestinations, - Coder destinationCoder, - RecordWriterFactory recordWriterFactory, - TableFactory tableFactory) { - this.dynamicDestinations = dynamicDestinations; - this.destinationCoder = destinationCoder; - this.triggered = false; - this.recordWriterFactory = recordWriterFactory; - this.tableFactory = tableFactory; - } - - private IcebergWriteResult expandTriggered(PCollection> input) { - - throw new NotImplementedException("Not yet implemented"); - } - - private IcebergWriteResult expandUntriggered(PCollection> input) { - - final PCollectionView fileView = createJobIdPrefixView(input.getPipeline()); - // We always do the equivalent of a dynamically sharded file creation - TupleTag> writtenFilesTag = - new TupleTag<>("writtenFiles"); - TupleTag, ElementT>> successfulWritesTag = - new TupleTag<>("successfulWrites"); - TupleTag, ElementT>> failedWritesTag = - new TupleTag<>("failedWrites"); - TupleTag> snapshotsTag = new TupleTag<>("snapshots"); - - final Coder elementCoder = - ((KvCoder) input.getCoder()).getValueCoder(); - - // Write everything to files - PCollectionTuple writeBundlesToFiles = - input.apply( - "Write Bundles To Files", - ParDo.of( - new WriteBundlesToFiles<>( - fileView, - successfulWritesTag, - failedWritesTag, - DEFAULT_MAX_WRITERS_PER_BUNDLE, - DEFAULT_MAX_BYTES_PER_FILE, - recordWriterFactory)) - .withSideInputs(fileView) - .withOutputTags( - writtenFilesTag, - TupleTagList.of(ImmutableList.of(successfulWritesTag, failedWritesTag)))); - - PCollection, ElementT>> successfulWrites = - writeBundlesToFiles - .get(successfulWritesTag) - .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), elementCoder)); - - PCollection, ElementT>> failedWrites = - writeBundlesToFiles - .get(failedWritesTag) - .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), elementCoder)); - - PCollection> writtenFilesGrouped = - failedWrites - .apply("Group By Destination", GroupByKey.create()) - .apply( - "Strip Shard ID", - MapElements.via( - new SimpleFunction< - KV, Iterable>, - KV>>() { - @Override - public KV> apply( - KV, Iterable> input) { - return KV.of(input.getKey().getKey(), input.getValue()); - } - })) - .setCoder(KvCoder.of(destinationCoder, IterableCoder.of(elementCoder))) - .apply( - "Write Grouped Records", - ParDo.of( - new WriteGroupedRecordsToFiles<>( - fileView, DEFAULT_MAX_BYTES_PER_FILE, recordWriterFactory))) - .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder)); - - PCollection> catalogUpdates = - PCollectionList.of( - writeBundlesToFiles - .get(writtenFilesTag) - .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder))) - .and(writtenFilesGrouped) - .apply("Flatten Files", Flatten.pCollections()) - .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder)); - - // Apply any sharded writes and flatten everything for catalog updates - PCollection> snapshots = - catalogUpdates - .apply( - "Extract Data File", - ParDo.of( - new DoFn, KV>() { - @ProcessElement - public void processElement( - ProcessContext c, @Element Result element) { - c.output( - KV.of( - element.tableId, - MetadataUpdate.of( - element.partitionSpec, element.update.getDataFiles().get(0)))); - } - })) - .setCoder(KvCoder.of(StringUtf8Coder.of(), MetadataUpdate.coder())) - .apply(GroupByKey.create()) - .apply("Write Metadata Updates", ParDo.of(new MetadataUpdates<>(tableFactory))) - .setCoder(KvCoder.of(StringUtf8Coder.of(), SerializableCoder.of(Snapshot.class))); - - return new IcebergWriteResult( - input.getPipeline(), - successfulWrites, - catalogUpdates, - snapshots, - successfulWritesTag, - writtenFilesTag, - snapshotsTag); - } - - private PCollectionView createJobIdPrefixView(Pipeline p) { - - final String jobName = p.getOptions().getJobName(); - - return p.apply("JobIdCreationRoot_", Create.of((Void) null)) - .apply( - "CreateJobId", - ParDo.of( - new DoFn() { - @ProcessElement - public void process(ProcessContext c) { - c.output(jobName + "-" + UUID.randomUUID().toString()); - } - })) - .apply("JobIdSideInput", View.asSingleton()); - } - - public IcebergWriteResult expand(PCollection> input) { - - String jobName = input.getPipeline().getOptions().getJobName(); - - // We always window into global as far as I can tell? - PCollection> globalInput = - input.apply( - "rewindowIntoGlobal", - Window.>into(new GlobalWindows()) - .triggering(DefaultTrigger.of()) - .discardingFiredPanes()); - return triggered ? expandTriggered(input) : expandUntriggered(input); - } -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergTableCreateConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergTableCreateConfig.java new file mode 100644 index 000000000000..c1041f026c31 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergTableCreateConfig.java @@ -0,0 +1,50 @@ +/* + * 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.io.iceberg; + +import com.google.auto.value.AutoValue; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.checkerframework.dataflow.qual.Pure; + +@AutoValue +public abstract class IcebergTableCreateConfig { + + /** Schema for the destination, in the event that it must be dynamically created. */ + @Pure + public abstract Schema getSchema(); + + /** Partition spec destination, in the event that it must be dynamically created. */ + @Pure + public abstract PartitionSpec getPartitionSpec(); + + @Pure + public Builder builder() { + return new AutoValue_IcebergTableCreateConfig.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setSchema(Schema schema); + + public abstract Builder setPartitionSpec(PartitionSpec partitionSpec); + + @Pure + public abstract IcebergTableCreateConfig build(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteResult.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteResult.java index 0040e385457c..94ac576674ed 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteResult.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/IcebergWriteResult.java @@ -18,7 +18,6 @@ package org.apache.beam.io.iceberg; import java.util.Map; -import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.KV; @@ -27,41 +26,25 @@ import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.DataFile; import org.apache.iceberg.Snapshot; -@SuppressWarnings("all") -public final class IcebergWriteResult implements POutput { +public final class IcebergWriteResult implements POutput { - private final Pipeline pipeline; - @Nullable PCollection successfulInserts; - @Nullable TupleTag successfulInsertsTag; + private static final TupleTag> SNAPSHOTS_TAG = + new TupleTag>() {}; - @Nullable PCollection>> catalogUpdates; - @Nullable TupleTag>> catalogUpdatesTag; + private final Pipeline pipeline; - @Nullable PCollection> snapshots; + private final PCollection> snapshots; - @Nullable TupleTag> snapshotsTag; + public PCollection> getSnapshots() { + return snapshots; + } - public IcebergWriteResult( - Pipeline pipeline, - @Nullable PCollection successfulInserts, - @Nullable PCollection>> catalogUpdates, - @Nullable PCollection> snapshots, - @Nullable TupleTag successfulInsertsTag, - @Nullable TupleTag>> catalogUpdatesTag, - @Nullable TupleTag> snapshotsTag) { + IcebergWriteResult(Pipeline pipeline, PCollection> snapshots) { this.pipeline = pipeline; - this.successfulInserts = successfulInserts; - this.catalogUpdates = catalogUpdates; this.snapshots = snapshots; - - this.successfulInsertsTag = successfulInsertsTag; - this.catalogUpdatesTag = catalogUpdatesTag; - this.snapshotsTag = snapshotsTag; } @Override @@ -69,23 +52,10 @@ public Pipeline getPipeline() { return pipeline; } - public PCollection getSuccessfulInserts() { - return successfulInserts; - } - @Override public Map, PValue> expand() { ImmutableMap.Builder, PValue> output = ImmutableMap.builder(); - if (successfulInsertsTag != null) { - output.put(successfulInsertsTag, Preconditions.checkNotNull(successfulInserts)); - } - if (catalogUpdatesTag != null) { - output.put(catalogUpdatesTag, Preconditions.checkNotNull(catalogUpdates)); - } - if (snapshotsTag != null) { - output.put(snapshotsTag, Preconditions.checkNotNull(snapshots)); - } - + output.put(SNAPSHOTS_TAG, snapshots); return output.build(); } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdate.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdate.java deleted file mode 100644 index bfde156d5c8b..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdate.java +++ /dev/null @@ -1,172 +0,0 @@ -/* - * 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.io.iceberg; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.List; -import java.util.Map; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; -import org.apache.avro.generic.IndexedRecord; -import org.apache.avro.specific.SpecificData.SchemaConstructable; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.avro.AvroEncoderUtil; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.types.Types.StructType; -import org.checkerframework.checker.initialization.qual.Initialized; -import org.checkerframework.checker.nullness.qual.NonNull; -import org.checkerframework.checker.nullness.qual.UnknownKeyFor; - -@SuppressWarnings("all") -public class MetadataUpdate implements IndexedRecord, SchemaConstructable { - - private List dataFiles; - private List deleteFiles; - - private final Schema avroSchema; - - public MetadataUpdate(Schema avroSchema) { - this.avroSchema = avroSchema; - } - - public MetadataUpdate( - StructType partitionType, List dataFiles, List deleteFiles) { - this.dataFiles = dataFiles; - this.deleteFiles = deleteFiles; - - StructType dataFileStruct = DataFile.getType(partitionType); - Map dataFileNames = - ImmutableMap.of( - dataFileStruct, "org.apache.iceberg.GenericDataFile", - partitionType, "org.apache.iceberg.PartitionData"); - Schema dataFileSchema = AvroSchemaUtil.convert(dataFileStruct, dataFileNames); - Map deleteFileNames = - ImmutableMap.of( - dataFileStruct, "org.apache.iceberg.GenericDeleteFile", - partitionType, "org.apache.iceberg.PartitionData"); - Schema deleteFileSchema = AvroSchemaUtil.convert(dataFileStruct, deleteFileNames); - - this.avroSchema = - SchemaBuilder.builder() - .record(getClass().getName()) - .fields() - .name("dataFiles") - .prop(AvroSchemaUtil.FIELD_ID_PROP, "-1") - .type() - .nullable() - .array() - .items(dataFileSchema) - .noDefault() - .name("deleteFiles") - .prop(AvroSchemaUtil.FIELD_ID_PROP, "-1") - .type() - .nullable() - .array() - .items(deleteFileSchema) - .noDefault() - .endRecord(); - } - - public static MetadataUpdate of(PartitionSpec partitionSpec, DataFile dataFile) { - return new MetadataUpdate(partitionSpec.partitionType(), ImmutableList.of(dataFile), null); - } - - public List getDataFiles() { - return this.dataFiles; - } - - public List getDeleteFiles() { - return this.deleteFiles; - } - - @Override - public void put(int i, Object v) { - switch (i) { - case 0: - this.dataFiles = (List) v; - return; - case 1: - this.deleteFiles = (List) v; - return; - default: - } - } - - @Override - public Object get(int i) { - switch (i) { - case 0: - return this.dataFiles; - case 1: - return this.deleteFiles; - default: - throw new UnsupportedOperationException("Unknown field ordinal: " + i); - } - } - - @Override - public Schema getSchema() { - return avroSchema; - } - - protected static class MetadataUpdateCoder extends Coder { - - private static final ByteArrayCoder bytesCoder = ByteArrayCoder.of(); - - @Override - public void encode( - MetadataUpdate value, @UnknownKeyFor @NonNull @Initialized OutputStream outStream) - throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull - @Initialized IOException { - bytesCoder.encode(AvroEncoderUtil.encode(value, value.getSchema()), outStream); - } - - @Override - public MetadataUpdate decode(@UnknownKeyFor @NonNull @Initialized InputStream inStream) - throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull - @Initialized IOException { - byte[] updateBytes = bytesCoder.decode(inStream); - return AvroEncoderUtil.decode(updateBytes); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized List< - ? extends - @UnknownKeyFor @NonNull @Initialized Coder<@UnknownKeyFor @NonNull @Initialized ?>> - getCoderArguments() { - return ImmutableList.of(); - } - - @Override - public void verifyDeterministic() - throws @UnknownKeyFor @NonNull @Initialized NonDeterministicException {} - } - - public static Coder coder() { - return new MetadataUpdateCoder(); - } -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdates.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdates.java deleted file mode 100644 index a19cb3b9bc4d..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/MetadataUpdates.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.io.iceberg; - -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.KV; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; - -public class MetadataUpdates - extends DoFn>, KV> { - - final TableFactory tableFactory; - - public MetadataUpdates(TableFactory tableFactory) { - this.tableFactory = tableFactory; - } - - @ProcessElement - public void processElement( - ProcessContext c, - @Element KV> element, - BoundedWindow window) { - Table table = tableFactory.getTable(element.getKey()); - AppendFiles update = table.newAppend(); - Iterable metadataUpdates = element.getValue(); - if (metadataUpdates != null) { - for (MetadataUpdate metadata : metadataUpdates) { - for (DataFile file : metadata.getDataFiles()) { - update.appendFile(file); - } - } - update.commit(); - c.outputWithTimestamp( - KV.of(element.getKey(), table.currentSnapshot()), window.maxTimestamp()); - } - } -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/OneTableDynamicDestinations.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/OneTableDynamicDestinations.java new file mode 100644 index 000000000000..dc6815235a06 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/OneTableDynamicDestinations.java @@ -0,0 +1,65 @@ +/* + * 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.io.iceberg; + +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.catalog.TableIdentifier; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; + +class OneTableDynamicDestinations implements DynamicDestinations { + + private static final Schema EMPTY_SCHEMA = Schema.builder().build(); + private static final Row EMPTY_ROW = Row.nullRow(EMPTY_SCHEMA); + + // TableId represented as String for serializability + private final String tableIdString; + + private transient @MonotonicNonNull TableIdentifier tableId; + + private TableIdentifier getTableIdentifier() { + if (tableId == null) { + tableId = TableIdentifier.parse(tableIdString); + } + return tableId; + } + + OneTableDynamicDestinations(TableIdentifier tableId) { + this.tableIdString = tableId.toString(); + } + + @Override + public Schema getMetadataSchema() { + return EMPTY_SCHEMA; + } + + @Override + public Row assignDestinationMetadata(Row data) { + return EMPTY_ROW; + } + + @Override + public IcebergDestination instantiateDestination(Row dest) { + return IcebergDestination.builder() + .setTableIdentifier(getTableIdentifier()) + .setTableCreateConfig(null) + .setFileFormat(FileFormat.PARQUET) + .build(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PrepareWrite.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PrepareWrite.java deleted file mode 100644 index 9b380c822e58..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PrepareWrite.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * 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.io.iceberg; - -import java.io.IOException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.ValueInSingleWindow; -import org.joda.time.Instant; - -@SuppressWarnings("all") // TODO: Remove this once development is stable. -public class PrepareWrite - extends PTransform, PCollection>> { - - private DynamicDestinations dynamicDestinations; - private SerializableFunction formatFunction; - private Coder outputCoder; - - public PrepareWrite( - DynamicDestinations dynamicDestinations, - SerializableFunction formatFunction, - Coder outputCoder) { - this.dynamicDestinations = dynamicDestinations; - this.formatFunction = formatFunction; - this.outputCoder = outputCoder; - } - - @Override - public PCollection> expand(PCollection input) { - - final Coder destCoder; - try { - destCoder = - KvCoder.of( - dynamicDestinations.getDestinationCoderWithDefault( - input.getPipeline().getCoderRegistry()), - outputCoder); - } catch (Exception e) { - RuntimeException e1 = new RuntimeException("Unable to expand PrepareWrite"); - e1.addSuppressed(e); - throw e1; - } - return input - .apply( - ParDo.of( - new DoFn>() { - - @ProcessElement - public void processElement( - ProcessContext c, - @Element InputT element, - @Timestamp Instant timestamp, - BoundedWindow window, - PaneInfo pane) - throws IOException { - ValueInSingleWindow windowedElement = - ValueInSingleWindow.of(element, timestamp, window, pane); - dynamicDestinations.setSideInputProcessContext(c); - DestinationT tableDestination = - dynamicDestinations.getDestination(windowedElement); - OutputT outputValue = formatFunction.apply(element); - c.output(KV.of(tableDestination, outputValue)); - } - }) - .withSideInputs(dynamicDestinations.getSideInputs())) - .setCoder(destCoder); - } -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/PropertyBuilder.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PropertyBuilder.java similarity index 85% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/PropertyBuilder.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PropertyBuilder.java index 9dc67992aeae..53184c70dfca 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/PropertyBuilder.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/PropertyBuilder.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg.util; +package org.apache.beam.io.iceberg; -import com.google.common.collect.ImmutableMap; -import javax.annotation.Nullable; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.checkerframework.checker.nullness.qual.Nullable; /** Convenience utility class to build immutable maps that drops attempts to set null values. */ -public class PropertyBuilder { +class PropertyBuilder { ImmutableMap.Builder builder = ImmutableMap.builder(); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriter.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriter.java index 2f7380f1ced0..d9a3427c11fc 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriter.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriter.java @@ -17,98 +17,83 @@ */ package org.apache.beam.io.iceberg; +import static org.apache.beam.io.iceberg.RowHelper.rowToRecord; + import java.io.IOException; -import org.apache.beam.sdk.transforms.SerializableBiFunction; +import org.apache.beam.sdk.values.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.orc.GenericOrcWriter; import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.io.DataWriter; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; -@SuppressWarnings("all") // TODO: Remove this once development is stable. -class RecordWriter { - - final Table table; +class RecordWriter { - final DataWriter writer; - final GenericRecord baseRecord; - final SerializableBiFunction toRecord; + private final DataWriter icebergDataWriter; - final String location; + private final Table table; - RecordWriter( - Table table, - String location, - Schema schema, - PartitionSpec partitionSpec, - FileFormat format, - SerializableBiFunction toRecord) + RecordWriter(Catalog catalog, IcebergDestination destination, String filename) throws IOException { + this( + catalog.loadTable(destination.getTableIdentifier()), destination.getFileFormat(), filename); + } + + RecordWriter(Table table, FileFormat fileFormat, String filename) throws IOException { this.table = table; - this.baseRecord = GenericRecord.create(schema); - this.toRecord = toRecord; - this.location = table.locationProvider().newDataLocation(partitionSpec, baseRecord, location); - OutputFile outputFile = table.io().newOutputFile(this.location); - switch (format) { + String absoluteFilename = table.location() + "/" + filename; + OutputFile outputFile = table.io().newOutputFile(absoluteFilename); + switch (fileFormat) { case AVRO: - writer = - Avro.writeData(outputFile).schema(schema).withSpec(partitionSpec).overwrite().build(); + icebergDataWriter = + Avro.writeData(outputFile) + .createWriterFunc(org.apache.iceberg.data.avro.DataWriter::create) + .schema(table.schema()) + .withSpec(table.spec()) + .overwrite() + .build(); break; case PARQUET: - writer = + icebergDataWriter = Parquet.writeData(outputFile) .createWriterFunc(GenericParquetWriter::buildWriter) - .schema(schema) - .withSpec(partitionSpec) + .schema(table.schema()) + .withSpec(table.spec()) .overwrite() .build(); break; case ORC: - writer = - ORC.writeData(outputFile) - .createWriterFunc(GenericOrcWriter::buildWriter) - .schema(schema) - .withSpec(partitionSpec) - .overwrite() - .build(); - break; + throw new UnsupportedOperationException("ORC file format not currently supported."); default: - throw new RuntimeException("Unrecognized File Format. This should be impossible."); + throw new RuntimeException("Unknown File Format: " + fileFormat); } } - public void write(ElementT element) throws IOException { - Record record = toRecord.apply(baseRecord, element); - writer.write(record); + public void write(Row row) { + Record record = rowToRecord(table.schema(), row); + icebergDataWriter.write(record); } public void close() throws IOException { - writer.close(); - } - - public long bytesWritten() { - return writer.length(); + icebergDataWriter.close(); } - public Table table() { + public Table getTable() { return table; } - public String location() { - return location; + public long bytesWritten() { + return icebergDataWriter.length(); } public DataFile dataFile() { - return writer.toDataFile(); + return icebergDataWriter.toDataFile(); } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriterFactory.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriterFactory.java deleted file mode 100644 index 9058d150c530..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RecordWriterFactory.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.io.iceberg; - -import java.io.Serializable; -import javax.annotation.Nullable; -import org.apache.beam.sdk.transforms.SerializableBiFunction; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.data.Record; - -@SuppressWarnings("all") // TODO: Remove this once development is stable. -abstract class RecordWriterFactory implements Serializable { - private RecordWriterFactory() {} - - public abstract RecordWriterFactory prepare( - DynamicDestinations destination); - - public abstract RecordWriter createWriter(String location, DestinationT destination) - throws Exception; - - static TableRecordWriterFactory tableRecords( - SerializableBiFunction toRecord, - @Nullable DynamicDestinations dynamicDestinations) { - return new TableRecordWriterFactory<>(toRecord, dynamicDestinations); - } - - static final class TableRecordWriterFactory - extends RecordWriterFactory { - - final SerializableBiFunction toRecord; - - final DynamicDestinations dynamicDestinations; - - TableRecordWriterFactory( - SerializableBiFunction toRecord, - DynamicDestinations dynamicDestinations) { - this.toRecord = toRecord; - this.dynamicDestinations = dynamicDestinations; - } - - @Override - public RecordWriterFactory prepare( - DynamicDestinations destination) { - return new TableRecordWriterFactory<>(toRecord, destination); - } - - @Override - public RecordWriter createWriter(String location, DestinationT destination) - throws Exception { - Table table = dynamicDestinations.getTable(destination); - Schema schema = dynamicDestinations.getSchema(destination); - PartitionSpec partitionSpec = dynamicDestinations.getPartitionSpec(destination); - FileFormat format = dynamicDestinations.getFileFormat(destination); - return new RecordWriter<>(table, location, schema, partitionSpec, format, toRecord); - } - } -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/RowHelper.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RowHelper.java similarity index 65% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/RowHelper.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RowHelper.java index 0ad5f663a2cf..92f55208a0da 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/RowHelper.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/RowHelper.java @@ -15,19 +15,34 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg.util; +package org.apache.beam.io.iceberg; +import java.nio.ByteBuffer; import java.util.Optional; -import org.apache.beam.sdk.transforms.SerializableBiFunction; +import java.util.UUID; import org.apache.beam.sdk.values.Row; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.types.Types.NestedField; -public class RowHelper { +class RowHelper { + + // static helper functions only private RowHelper() {} - public static void copyInto(Record rec, NestedField field, Row value) { + public static Record rowToRecord(org.apache.iceberg.Schema schema, Row row) { + return copy(GenericRecord.create(schema), row); + } + + private static Record copy(Record baseRecord, Row value) { + Record rec = baseRecord.copy(); + for (NestedField f : rec.struct().fields()) { + copyInto(rec, f, value); + } + return rec; + } + + private static void copyInto(Record rec, NestedField field, Row value) { String name = field.name(); switch (field.type().typeId()) { case BOOLEAN: @@ -46,21 +61,28 @@ public static void copyInto(Record rec, NestedField field, Row value) { Optional.ofNullable(value.getDouble(name)).ifPresent(v -> rec.setField(name, v)); break; case DATE: - break; + throw new UnsupportedOperationException("Date fields not yet supported"); case TIME: - break; + throw new UnsupportedOperationException("Time fields not yet supported"); case TIMESTAMP: + Optional.ofNullable(value.getDateTime(name)) + .ifPresent(v -> rec.setField(name, v.getMillis())); break; case STRING: Optional.ofNullable(value.getString(name)).ifPresent(v -> rec.setField(name, v)); break; case UUID: + Optional.ofNullable(value.getBytes(name)) + .ifPresent(v -> rec.setField(name, UUID.nameUUIDFromBytes(v))); break; case FIXED: - break; + throw new UnsupportedOperationException("Fixed-precision fields are not yet supported."); case BINARY: + Optional.ofNullable(value.getBytes(name)) + .ifPresent(v -> rec.setField(name, ByteBuffer.wrap(v))); break; case DECIMAL: + Optional.ofNullable(value.getDecimal(name)).ifPresent(v -> rec.setField(name, v)); break; case STRUCT: Optional.ofNullable(value.getRow(name)) @@ -70,26 +92,9 @@ public static void copyInto(Record rec, NestedField field, Row value) { name, copy(GenericRecord.create(field.type().asStructType()), row))); break; case LIST: - break; + throw new UnsupportedOperationException("List fields are not yet supported."); case MAP: - break; - } - } - - public static Record copy(Record baseRecord, Row value) { - Record rec = baseRecord.copy(); - for (NestedField f : rec.struct().fields()) { - copyInto(rec, f, value); + throw new UnsupportedOperationException("Map fields are not yet supported."); } - return rec; - } - - public static SerializableBiFunction recordsFromRows() { - return new SerializableBiFunction() { - @Override - public Record apply(Record record, Row row) { - return copy(record, row); - } - }; } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/SchemaHelper.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaHelper.java similarity index 95% rename from sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/SchemaHelper.java rename to sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaHelper.java index c1e25351b640..bbd23239de66 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/SchemaHelper.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/SchemaHelper.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.io.iceberg.util; +package org.apache.beam.io.iceberg; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; @@ -23,11 +23,11 @@ import org.apache.iceberg.types.Types; @SuppressWarnings({"dereference.of.nullable"}) -public class SchemaHelper { +class SchemaHelper { private SchemaHelper() {} - public static String ICEBERG_TYPE_OPTION_NAME = "icebergTypeID"; + public static final String ICEBERG_TYPE_OPTION_NAME = "icebergTypeID"; public static Schema.FieldType fieldTypeForType(final Type type) { switch (type.typeId()) { @@ -62,7 +62,7 @@ public static Schema.FieldType fieldTypeForType(final Type type) { fieldTypeForType(type.asMapType().keyType()), fieldTypeForType(type.asMapType().valueType())); } - throw new RuntimeException("Unrecognized Iceberg Type"); + throw new RuntimeException("Unrecognized IcebergIO Type"); } public static Schema.Field convert(final Types.NestedField field) { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/TableFactory.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/TableFactory.java deleted file mode 100644 index 135faf3decfe..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/TableFactory.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.io.iceberg; - -import java.io.Serializable; -import java.util.Arrays; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; - -@SuppressWarnings("all") -public abstract class TableFactory implements Serializable { - - private TableFactory() {} - - public abstract Table getTable(IdentifierT id); - - public static TableFactory forCatalog(final Iceberg.Catalog catalog) { - return new TableFactory() { - @Override - public Table getTable(String id) { - TableIdentifier tableId = TableIdentifier.parse(id); - // If the first element in the namespace is our catalog, remove that. - if (tableId.hasNamespace()) { - Namespace ns = tableId.namespace(); - if (catalog.catalog().name().equals(ns.level(0))) { - String[] levels = ns.levels(); - levels = Arrays.copyOfRange(levels, 1, levels.length); - tableId = TableIdentifier.of(Namespace.of(levels), tableId.name()); - } - } - return catalog.catalog().loadTable(tableId); - } - }; - } -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteBundlesToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteBundlesToFiles.java deleted file mode 100644 index b9ef6fe55db9..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteBundlesToFiles.java +++ /dev/null @@ -1,312 +0,0 @@ -/* - * 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.io.iceberg; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ThreadLocalRandom; -import org.apache.beam.io.iceberg.WriteBundlesToFiles.Result; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.SerializableCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.StructuredCoder; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.ShardedKey; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Objects; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.checkerframework.checker.initialization.qual.Initialized; -import org.checkerframework.checker.nullness.qual.NonNull; -import org.checkerframework.checker.nullness.qual.UnknownKeyFor; - -@SuppressWarnings("all") // TODO: Remove this once development is stable. -public class WriteBundlesToFiles - extends DoFn, Result> { - - private transient Map> writers; - private transient Map windows; - - private static final int SPILLED_RECORD_SHARDING_FACTOR = 10; - - private final PCollectionView locationPrefixView; - - private final TupleTag, ElementT>> successfulWritesTag; - private final TupleTag, ElementT>> unwrittenRecordsTag; - private final int maxWritersPerBundle; - private final long maxFileSize; - - private final RecordWriterFactory recordWriterFactory; - - private int spilledShardNumber; - - static final class Result implements Serializable { - private static final long serialVersionUID = 1L; - - public final String tableId; - public final String location; - - public final PartitionSpec partitionSpec; - - public final MetadataUpdate update; - - public final DestinationT destination; - - public Result( - String tableId, - String location, - DataFile dataFile, - PartitionSpec partitionSpec, - DestinationT destination) { - this.tableId = tableId; - this.location = location; - this.update = MetadataUpdate.of(partitionSpec, dataFile); - this.partitionSpec = partitionSpec; - this.destination = destination; - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof Result) { - Result result = (Result) obj; - return Objects.equal(result.tableId, tableId) - && Objects.equal(result.location, location) - && Objects.equal(result.partitionSpec, partitionSpec) - && Objects.equal(result.update.getDataFiles().get(0), update.getDataFiles().get(0)) - && Objects.equal(destination, result.destination); - } - return false; - } - - @Override - public int hashCode() { - return Objects.hashCode( - tableId, location, update.getDataFiles().get(0), partitionSpec, destination); - } - - @Override - public String toString() { - return "Result{" - + "table='" - + tableId - + '\'' - + "location='" - + location - + '\'' - + ", fileByteSize=" - + update.getDataFiles().get(0).fileSizeInBytes() - + ", destination=" - + destination - + '}'; - } - } - - public static class ResultCoder extends StructuredCoder> { - private static final StringUtf8Coder stringCoder = StringUtf8Coder.of(); - private static final Coder metadataCoder = MetadataUpdate.coder(); - - private static final SerializableCoder partitionSpecCoder = - SerializableCoder.of(PartitionSpec.class); - - private final Coder destinationCoder; - - public ResultCoder(Coder destinationCoder) { - this.destinationCoder = destinationCoder; - } - - @Override - public void encode( - Result value, @UnknownKeyFor @NonNull @Initialized OutputStream outStream) - throws @UnknownKeyFor @NonNull @Initialized CoderException, @UnknownKeyFor @NonNull - @Initialized IOException { - - // Convert most everything to Avro for serialization - - // Table id and location are strings - stringCoder.encode(value.tableId, outStream); - stringCoder.encode(value.location, outStream); - // PartitionSpec is Java serialized because we need it to decode DataFile - destinationCoder.encode(value.destination, outStream); - metadataCoder.encode(value.update, outStream); - partitionSpecCoder.encode(value.partitionSpec, outStream); - } - - @Override - public Result decode(InputStream inStream) throws CoderException, IOException { - String tableId = stringCoder.decode(inStream); - String location = stringCoder.decode(inStream); - DestinationT dest = destinationCoder.decode(inStream); - MetadataUpdate update = metadataCoder.decode(inStream); - PartitionSpec spec = partitionSpecCoder.decode(inStream); - return new Result<>(tableId, location, update.getDataFiles().get(0), spec, dest); - } - - @Override - public List> getCoderArguments() { - return Collections.singletonList(destinationCoder); - } - - @Override - public void verifyDeterministic() throws NonDeterministicException {} - - public static ResultCoder of( - Coder destinationCoder) { - return new ResultCoder<>(destinationCoder); - } - } - - public WriteBundlesToFiles( - PCollectionView locationPrefixView, - TupleTag, ElementT>> successfulWritesTag, - TupleTag, ElementT>> unwrittenRecordsTag, - int maximumWritersPerBundle, - long maxFileSize, - RecordWriterFactory recordWriterFactory) { - this.locationPrefixView = locationPrefixView; - this.successfulWritesTag = successfulWritesTag; - this.unwrittenRecordsTag = unwrittenRecordsTag; - this.maxWritersPerBundle = maximumWritersPerBundle; - this.maxFileSize = maxFileSize; - this.recordWriterFactory = recordWriterFactory; - } - - @StartBundle - public void startBundle() { - this.writers = Maps.newHashMap(); - this.windows = Maps.newHashMap(); - this.spilledShardNumber = ThreadLocalRandom.current().nextInt(SPILLED_RECORD_SHARDING_FACTOR); - } - - RecordWriter createWriter( - DestinationT destination, String location, BoundedWindow window) throws Exception { - Map windows = Preconditions.checkNotNull(this.windows); - Map> writers = Preconditions.checkNotNull(this.writers); - RecordWriter writer = recordWriterFactory.createWriter(location, destination); - windows.put(destination, window); - writers.put(destination, writer); - return writer; - } - - @ProcessElement - public void processElement( - ProcessContext c, @Element KV element, BoundedWindow window) - throws Exception { - Map> writers = Preconditions.checkNotNull(this.writers); - String locationPrefix = c.sideInput(locationPrefixView); - DestinationT destination = element.getKey(); - RecordWriter writer; - if (writers.containsKey(destination)) { - writer = writers.get(destination); - } else { - if (writers.size() <= maxWritersPerBundle) { - writer = createWriter(destination, locationPrefix, window); - } else { - c.output( - unwrittenRecordsTag, - KV.of( - ShardedKey.of(destination, ++spilledShardNumber % SPILLED_RECORD_SHARDING_FACTOR), - element.getValue())); - return; - } - } - - if (writer.bytesWritten() > maxFileSize) { - writer.close(); - Table t = writer.table(); - - c.output(new Result<>(t.name(), writer.location(), writer.dataFile(), t.spec(), destination)); - writer = createWriter(destination, locationPrefix, window); - } - - try { - writer.write(element.getValue()); - c.output( - successfulWritesTag, - KV.of( - ShardedKey.of(destination, spilledShardNumber % SPILLED_RECORD_SHARDING_FACTOR), - element.getValue())); - } catch (Exception e) { - try { - writer.close(); - } catch (Exception closeException) { - e.addSuppressed(closeException); - } - throw e; - } - } - - @FinishBundle - public void finishBundle(FinishBundleContext c) throws Exception { - Map windows = Preconditions.checkNotNull(this.windows); - Map> writers = Preconditions.checkNotNull(this.writers); - List exceptionList = Lists.newArrayList(); - for (RecordWriter writer : writers.values()) { - try { - writer.close(); - } catch (Exception e) { - exceptionList.add(e); - } - } - if (!exceptionList.isEmpty()) { - Exception e = new IOException("Exception closing some writers."); - for (Exception thrown : exceptionList) { - e.addSuppressed(thrown); - } - throw e; - } - - exceptionList.clear(); - for (Map.Entry> entry : writers.entrySet()) { - try { - DestinationT destination = entry.getKey(); - - RecordWriter writer = entry.getValue(); - BoundedWindow window = windows.get(destination); - Preconditions.checkNotNull(window); - Table t = writer.table(); - c.output( - new Result<>(t.name(), writer.location(), writer.dataFile(), t.spec(), destination), - window.maxTimestamp(), - window); - } catch (Exception e) { - exceptionList.add(e); - } - } - writers.clear(); - if (!exceptionList.isEmpty()) { - Exception e = new IOException("Exception emitting writer metadata."); - for (Exception thrown : exceptionList) { - e.addSuppressed(thrown); - } - throw e; - } - } -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRecordsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRecordsToFiles.java deleted file mode 100644 index 53dbf43fa7f3..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRecordsToFiles.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.io.iceberg; - -import org.apache.beam.io.iceberg.WriteBundlesToFiles.Result; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollectionView; - -@SuppressWarnings("all") -public class WriteGroupedRecordsToFiles - extends DoFn>, Result> { - - private final PCollectionView locationPrefixView; - private final long maxFileSize; - private final RecordWriterFactory recordWriterFactory; - - WriteGroupedRecordsToFiles( - PCollectionView locationPrefixView, - long maxFileSize, - RecordWriterFactory recordWriterFactory) { - this.locationPrefixView = locationPrefixView; - this.maxFileSize = maxFileSize; - this.recordWriterFactory = recordWriterFactory; - } - - @ProcessElement - public void processElement( - ProcessContext c, @Element KV> element) throws Exception { - String locationPrefix = c.sideInput(locationPrefixView); - DestinationT destination = element.getKey(); - RecordWriter writer = recordWriterFactory.createWriter(locationPrefix, destination); - for (ElementT e : element.getValue()) { - writer.write(e); - } - writer.close(); - c.output( - new Result<>( - writer.table().name(), - writer.location(), - writer.dataFile(), - writer.table().spec(), - destination)); - } -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRowsToFiles.java new file mode 100644 index 000000000000..7a152b602581 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteGroupedRowsToFiles.java @@ -0,0 +1,103 @@ +/* + * 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.io.iceberg; + +import java.io.IOException; +import java.util.UUID; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.ShardedKey; +import org.apache.iceberg.catalog.Catalog; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; + +class WriteGroupedRowsToFiles + extends PTransform< + PCollection, Iterable>>, PCollection> { + + static final long DEFAULT_MAX_BYTES_PER_FILE = (1L << 40); // 1TB + + private final DynamicDestinations dynamicDestinations; + private final IcebergCatalogConfig catalogConfig; + + WriteGroupedRowsToFiles( + IcebergCatalogConfig catalogConfig, DynamicDestinations dynamicDestinations) { + this.catalogConfig = catalogConfig; + this.dynamicDestinations = dynamicDestinations; + } + + @Override + public PCollection expand( + PCollection, Iterable>> input) { + return input.apply( + ParDo.of( + new WriteGroupedRowsToFilesDoFn( + catalogConfig, dynamicDestinations, DEFAULT_MAX_BYTES_PER_FILE))); + } + + private static class WriteGroupedRowsToFilesDoFn + extends DoFn, Iterable>, FileWriteResult> { + + private final DynamicDestinations dynamicDestinations; + private final IcebergCatalogConfig catalogConfig; + private transient @MonotonicNonNull Catalog catalog; + + WriteGroupedRowsToFilesDoFn( + IcebergCatalogConfig catalogConfig, + DynamicDestinations dynamicDestinations, + long maxFileSize) { + this.catalogConfig = catalogConfig; + this.dynamicDestinations = dynamicDestinations; + } + + private org.apache.iceberg.catalog.Catalog getCatalog() { + if (catalog == null) { + this.catalog = catalogConfig.catalog(); + } + return catalog; + } + + private RecordWriter createWriter(IcebergDestination destination) throws IOException { + return new RecordWriter(getCatalog(), destination, "-" + UUID.randomUUID()); + } + + @ProcessElement + public void processElement( + ProcessContext c, @Element KV, Iterable> element) throws Exception { + + Row destMetadata = element.getKey().getKey(); + IcebergDestination destination = dynamicDestinations.instantiateDestination(destMetadata); + RecordWriter writer = createWriter(destination); + + for (Row e : element.getValue()) { + writer.write(e); + } + + writer.close(); + c.output( + FileWriteResult.builder() + .setTableIdentifier(destination.getTableIdentifier()) + .setDataFile(writer.dataFile()) + .setPartitionSpec(writer.getTable().spec()) + .build()); + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteToDestinations.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteToDestinations.java new file mode 100644 index 000000000000..22e27a4df7ca --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteToDestinations.java @@ -0,0 +1,123 @@ +/* + * 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.io.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; + +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.coders.ShardedKeyCoder; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.ShardedKey; +import org.apache.iceberg.Snapshot; + +class WriteToDestinations extends PTransform, IcebergWriteResult> { + + static final long DEFAULT_MAX_BYTES_PER_FILE = (1L << 40); // 1TB + static final int DEFAULT_NUM_FILE_SHARDS = 0; + static final int FILE_TRIGGERING_RECORD_COUNT = 50_000; + + private final IcebergCatalogConfig catalogConfig; + private final DynamicDestinations dynamicDestinations; + + WriteToDestinations(IcebergCatalogConfig catalogConfig, DynamicDestinations dynamicDestinations) { + this.dynamicDestinations = dynamicDestinations; + this.catalogConfig = catalogConfig; + } + + @Override + public IcebergWriteResult expand(PCollection input) { + + // First, attempt to write directly to files without shuffling. If there are + // too many distinct destinations in a single bundle, the remaining + // elements will be emitted to take the "slow path" that involves a shuffle + WriteUngroupedRowsToFiles.Result writeUngroupedResult = + input.apply( + "Fast-path write rows", + new WriteUngroupedRowsToFiles(catalogConfig, dynamicDestinations)); + + // Then write the rest by shuffling on the destination metadata + Schema destSchema = + checkArgumentNotNull( + writeUngroupedResult + .getSpilledRows() + .getSchema() + .getField("dest") + .getType() + .getRowSchema(), + "Input schema missing `dest` field."); + Schema dataSchema = + checkArgumentNotNull( + writeUngroupedResult + .getSpilledRows() + .getSchema() + .getField("data") + .getType() + .getRowSchema(), + "Input schema missing `data` field"); + + PCollection writeGroupedResult = + writeUngroupedResult + .getSpilledRows() + .apply( + "Key by destination and shard", + MapElements.via( + new SimpleFunction, Row>>() { + private static final int SPILLED_ROWS_SHARDING_FACTOR = 10; + private int shardNumber = 0; + + @Override + public KV, Row> apply(Row elem) { + Row data = + checkArgumentNotNull( + elem.getRow("data"), "Element missing `data` field"); + Row dest = + checkArgumentNotNull( + elem.getRow("dest"), "Element missing `dest` field"); + return KV.of( + ShardedKey.of(dest, shardNumber % SPILLED_ROWS_SHARDING_FACTOR), data); + } + })) + .setCoder( + KvCoder.of(ShardedKeyCoder.of(RowCoder.of(destSchema)), RowCoder.of(dataSchema))) + .apply("Group spilled rows by destination shard", GroupByKey.create()) + .apply( + "Write remaining rows to files", + new WriteGroupedRowsToFiles(catalogConfig, dynamicDestinations)); + + PCollection allWrittenFiles = + PCollectionList.of(writeUngroupedResult.getWrittenFiles()) + .and(writeGroupedResult) + .apply("Flatten Written Files", Flatten.pCollections()); + + // Apply any sharded writes and flatten everything for catalog updates + PCollection> snapshots = + allWrittenFiles.apply(new AppendFilesToTables(catalogConfig)); + + return new IcebergWriteResult(input.getPipeline(), snapshots); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteUngroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteUngroupedRowsToFiles.java new file mode 100644 index 000000000000..240678c83697 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/WriteUngroupedRowsToFiles.java @@ -0,0 +1,350 @@ +/* + * 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.io.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.iceberg.catalog.Catalog; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A PTransform that writes rows to files according to their dynamic destination. If there are too + * many destinations in a single bundle, some rows will be written to a secondary output and must be + * written via another method. + */ +class WriteUngroupedRowsToFiles + extends PTransform, WriteUngroupedRowsToFiles.Result> { + + /** + * Maximum number of writers that will be created per bundle. Any elements requiring more writers + * will be spilled. + */ + @VisibleForTesting static final int DEFAULT_MAX_WRITERS_PER_BUNDLE = 20; + + private static final long DEFAULT_MAX_BYTES_PER_FILE = (1L << 29); // 512mb + + private static final TupleTag WRITTEN_FILES_TAG = new TupleTag<>("writtenFiles"); + private static final TupleTag WRITTEN_ROWS_TAG = new TupleTag("writtenRows") {}; + private static final TupleTag SPILLED_ROWS_TAG = new TupleTag("spilledRows") {}; + + private final String fileSuffix; + private final DynamicDestinations dynamicDestinations; + private final IcebergCatalogConfig catalogConfig; + + WriteUngroupedRowsToFiles( + IcebergCatalogConfig catalogConfig, DynamicDestinations dynamicDestinations) { + this.catalogConfig = catalogConfig; + this.dynamicDestinations = dynamicDestinations; + this.fileSuffix = UUID.randomUUID().toString(); + } + + @Override + public Result expand(PCollection input) { + + PCollectionTuple resultTuple = + input.apply( + ParDo.of( + new WriteUngroupedRowsToFilesDoFn( + catalogConfig, + dynamicDestinations, + fileSuffix, + DEFAULT_MAX_WRITERS_PER_BUNDLE, + DEFAULT_MAX_BYTES_PER_FILE)) + .withOutputTags( + WRITTEN_FILES_TAG, + TupleTagList.of(ImmutableList.of(WRITTEN_ROWS_TAG, SPILLED_ROWS_TAG)))); + + return new Result( + input.getPipeline(), + resultTuple.get(WRITTEN_FILES_TAG), + resultTuple.get(WRITTEN_ROWS_TAG).setCoder(input.getCoder()), + resultTuple.get(SPILLED_ROWS_TAG).setCoder(input.getCoder())); + } + + /** + * The result of this transform has two components: the records that were written and the records + * that spilled over and need to be written by a subsquent method. + */ + static class Result implements POutput { + + private final Pipeline pipeline; + private final PCollection writtenRows; + private final PCollection spilledRows; + private final PCollection writtenFiles; + + private Result( + Pipeline pipeline, + PCollection writtenFiles, + PCollection writtenRows, + PCollection spilledRows) { + this.pipeline = pipeline; + this.writtenFiles = writtenFiles; + this.writtenRows = writtenRows; + this.spilledRows = spilledRows; + } + + public PCollection getWrittenRows() { + return writtenRows; + } + + public PCollection getSpilledRows() { + return spilledRows; + } + + public PCollection getWrittenFiles() { + return writtenFiles; + } + + @Override + public Pipeline getPipeline() { + return pipeline; + } + + @Override + public Map, PValue> expand() { + return ImmutableMap., PValue>builder() + .put(WRITTEN_FILES_TAG, writtenFiles) + .put(WRITTEN_ROWS_TAG, writtenRows) + .put(SPILLED_ROWS_TAG, spilledRows) + .build(); + } + + @Override + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) { + // noop + } + } + + /** + * A DoFn that writes each input row to its assigned destination and outputs a result object + * summarizing what it accomplished for a given bundle. + * + *

Specifically, the outputs are: + * + *

    + *
  • (main output) the written files + *
  • the written records + *
  • the spilled records which were not written + *
+ */ + private static class WriteUngroupedRowsToFilesDoFn extends DoFn { + + private final String filename; + private final int maxWritersPerBundle; + private final long maxFileSize; + private final DynamicDestinations dynamicDestinations; + private final IcebergCatalogConfig catalogConfig; + + private transient @MonotonicNonNull Map writers; + private transient @MonotonicNonNull Map windows; + private transient @MonotonicNonNull Catalog catalog; + + public WriteUngroupedRowsToFilesDoFn( + IcebergCatalogConfig catalogConfig, + DynamicDestinations dynamicDestinations, + String filename, + int maximumWritersPerBundle, + long maxFileSize) { + this.catalogConfig = catalogConfig; + this.dynamicDestinations = dynamicDestinations; + this.filename = filename; + this.maxWritersPerBundle = maximumWritersPerBundle; + this.maxFileSize = maxFileSize; + } + + private Map getWriters() { + if (writers == null) { + writers = Maps.newHashMap(); + } + return writers; + } + + private Map getWindows() { + if (windows == null) { + windows = Maps.newHashMap(); + } + return windows; + } + + private org.apache.iceberg.catalog.Catalog getCatalog() { + if (catalog == null) { + this.catalog = catalogConfig.catalog(); + } + return catalog; + } + + private RecordWriter createAndInsertWriter(IcebergDestination destination, BoundedWindow window) + throws IOException { + RecordWriter writer = + new RecordWriter(getCatalog(), destination, filename + "-" + UUID.randomUUID()); + getWindows().put(destination, window); + getWriters().put(destination, writer); + return writer; + } + + /** + * Returns active writer for this destination if possible. If this returns null then we have + * reached the maximum number of writers and should spill any records associated. + */ + @Nullable + RecordWriter getWriterIfPossible(IcebergDestination destination, BoundedWindow window) + throws IOException { + + RecordWriter existingWriter = getWriters().get(destination); + if (existingWriter != null) { + return existingWriter; + } + + if (getWriters().size() > maxWritersPerBundle) { + return null; + } + + return createAndInsertWriter(destination, window); + } + + @StartBundle + public void startBundle() {} + + @ProcessElement + public void processElement(@Element Row element, BoundedWindow window, MultiOutputReceiver out) + throws Exception { + + Row data = checkArgumentNotNull(element.getRow("data"), "Input row missing `data` field."); + Row destMetadata = + checkArgumentNotNull(element.getRow("dest"), "Input row missing `dest` field."); + IcebergDestination destination = dynamicDestinations.instantiateDestination(destMetadata); + + // Spill record if writer cannot be created + RecordWriter writer = getWriterIfPossible(destination, window); + if (writer == null) { + out.get(SPILLED_ROWS_TAG).output(element); + return; + } + + // Reset writer if max file size reached + if (writer.bytesWritten() > maxFileSize) { + writer.close(); + out.get(WRITTEN_FILES_TAG) + .output( + FileWriteResult.builder() + .setDataFile(writer.dataFile()) + .setTableIdentifier(destination.getTableIdentifier()) + .build()); + writer = createAndInsertWriter(destination, window); + } + + // Actually write the data + try { + writer.write(data); + out.get(WRITTEN_ROWS_TAG).output(element); + } catch (Exception e) { + try { + writer.close(); + } catch (Exception closeException) { + e.addSuppressed(closeException); + } + throw e; + } + } + + @FinishBundle + public void finishBundle(FinishBundleContext c) throws Exception { + closeAllWriters(); + outputFinalWrittenFiles(c); + getWriters().clear(); + } + + private void outputFinalWrittenFiles(DoFn.FinishBundleContext c) + throws Exception { + List exceptionList = Lists.newArrayList(); + for (Map.Entry entry : getWriters().entrySet()) { + try { + IcebergDestination destination = entry.getKey(); + + RecordWriter writer = entry.getValue(); + BoundedWindow window = + checkStateNotNull( + getWindows().get(destination), "internal error: no windows for destination"); + c.output( + FileWriteResult.builder() + .setDataFile(writer.dataFile()) + .setTableIdentifier(destination.getTableIdentifier()) + .setPartitionSpec(writer.getTable().spec()) + .build(), + window.maxTimestamp(), + window); + } catch (Exception e) { + exceptionList.add(e); + } + } + + if (!exceptionList.isEmpty()) { + Exception e = + new IOException("Exception emitting writer metadata. See suppressed exceptions"); + for (Exception thrown : exceptionList) { + e.addSuppressed(thrown); + } + throw e; + } + } + + private void closeAllWriters() throws Exception { + List exceptionList = Lists.newArrayList(); + for (RecordWriter writer : getWriters().values()) { + try { + writer.close(); + } catch (Exception e) { + exceptionList.add(e); + } + } + + if (!exceptionList.isEmpty()) { + Exception e = new IOException("Exception closing some writers. See suppressed exceptions."); + for (Exception thrown : exceptionList) { + e.addSuppressed(thrown); + } + throw e; + } + } + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/package-info.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/package-info.java index 835b3421ccc4..f97ff98677f9 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/package-info.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/package-info.java @@ -1 +1,20 @@ +/* + * 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. + */ + +/** Iceberg connectors. */ package org.apache.beam.io.iceberg; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/ScanHelper.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/ScanHelper.java deleted file mode 100644 index 268ad6b4550b..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/util/ScanHelper.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.io.iceberg.util; - -import org.apache.beam.io.iceberg.Iceberg; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableScan; - -public class ScanHelper { - - public static boolean isIncremental(Iceberg.Scan scan) { - if (scan.getFromSnapshotExclusive() != null) { - return true; - } - return false; - } - - public static TableScan tableScan(Table table, Iceberg.Scan scan) { - TableScan tableScan = table.newScan(); - return tableScan; - } -} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/BoundedScanTests.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/BoundedScanTests.java deleted file mode 100644 index 3a813dab930b..000000000000 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/BoundedScanTests.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * 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.io.iceberg; - -import org.apache.beam.io.iceberg.util.SchemaHelper; -import org.apache.beam.sdk.coders.RowCoder; -import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.Row; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.Table; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@RunWith(JUnit4.class) -public class BoundedScanTests { - - private static Logger LOG = LoggerFactory.getLogger(BoundedScanTests.class); - @ClassRule public static final TemporaryFolder temporaryFolder = new TemporaryFolder(); - - @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(temporaryFolder, "default"); - - @Rule public TestPipeline testPipeline = TestPipeline.create(); - - static class PrintRow extends DoFn { - - @ProcessElement - public void process(@Element Row row, OutputReceiver output) throws Exception { - LOG.info("Got row {}", row); - output.output(row); - } - } - - @Test - public void testSimpleScan() throws Exception { - Table simpleTable = warehouse.createTable(TestFixtures.SCHEMA); - simpleTable - .newFastAppend() - .appendFile( - warehouse.writeRecords( - "file1s1.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT1)) - .appendFile( - warehouse.writeRecords( - "file2s1.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT1)) - .appendFile( - warehouse.writeRecords( - "file3s1.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT1)) - .commit(); - - PCollection output = - testPipeline - .apply( - Read.from( - new IcebergBoundedSource( - Iceberg.Scan.builder() - .catalog( - Iceberg.Catalog.builder() - .name("hadoop") - .icebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .warehouseLocation(warehouse.location) - .build()) - .type(Iceberg.ScanType.TABLE) - .table(simpleTable.name().replace("hadoop.", "").split("\\.")) - .schema(SchemaHelper.convert(TestFixtures.SCHEMA)) - .build()))) - .apply(ParDo.of(new PrintRow())) - .setCoder(RowCoder.of(SchemaHelper.convert(TestFixtures.SCHEMA))); - PAssert.that(output); - testPipeline.run(); - } -} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/FileWriteResultTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/FileWriteResultTest.java new file mode 100644 index 000000000000..2499331beadc --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/FileWriteResultTest.java @@ -0,0 +1,147 @@ +/* + * 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.io.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +import java.io.Serializable; +import java.util.List; +import java.util.UUID; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.testing.CoderProperties; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.commons.compress.utils.Lists; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Types; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class FileWriteResultTest implements Serializable { + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); + + private static final Coder TEST_CODER = + FileWriteResult.FileWriteResultCoder.of(); + + private List getTestValues() throws Exception { + TableIdentifier tableId = + TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); + + // Create a table so we can have some DataFile objects + Table table = warehouse.createTable(tableId, TestFixtures.SCHEMA); + List values = Lists.newArrayList(); + + // A parquet file + RecordWriter writer = + new RecordWriter(table, FileFormat.PARQUET, TEMPORARY_FOLDER.newFile().toString()); + writer.write( + Row.withSchema(SchemaHelper.convert(TestFixtures.SCHEMA)).addValues(42L, "bizzle").build()); + writer.close(); + DataFile dataFile = writer.dataFile(); + values.add( + FileWriteResult.builder() + .setDataFile(dataFile) + .setPartitionSpec(table.spec()) + .setTableIdentifier(tableId) + .build()); + + // An avro file + writer = new RecordWriter(table, FileFormat.AVRO, TEMPORARY_FOLDER.newFile().toString()); + writer.write( + Row.withSchema(SchemaHelper.convert(TestFixtures.SCHEMA)).addValues(42L, "bizzle").build()); + writer.close(); + dataFile = writer.dataFile(); + values.add( + FileWriteResult.builder() + .setDataFile(dataFile) + .setPartitionSpec(table.spec()) + .setTableIdentifier(tableId) + .build()); + + // Parquet file with a different schema + TableIdentifier tableId2 = + TableIdentifier.of( + "default", "othertable" + Long.toString(UUID.randomUUID().hashCode(), 16)); + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + optional( + 3, + "extra", + Types.StructType.of( + Types.NestedField.required(4, "inner", Types.BinaryType.get())))); + Table table2 = warehouse.createTable(tableId2, schema); + + // A parquet file in this other table + writer = new RecordWriter(table2, FileFormat.PARQUET, TEMPORARY_FOLDER.newFile().toString()); + writer.write( + Row.withSchema(SchemaHelper.convert(schema)) + .addValues( + 42L, + "bizzle", + Row.withSchema( + org.apache.beam.sdk.schemas.Schema.of( + org.apache.beam.sdk.schemas.Schema.Field.of( + "inner", org.apache.beam.sdk.schemas.Schema.FieldType.BYTES))) + .addValues(new byte[] {0xa}) + .build()) + .build()); + writer.close(); + DataFile dataFile2 = writer.dataFile(); + values.add( + FileWriteResult.builder() + .setDataFile(dataFile2) + .setPartitionSpec(table2.spec()) + .setTableIdentifier(tableId2) + .build()); + + return values; + } + + @Test + public void testDecodeEncodeEqual() throws Exception { + for (FileWriteResult value : getTestValues()) { + CoderProperties.structuralValueDecodeEncodeEqual(TEST_CODER, value); + } + } + + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testEncodedTypeDescriptor() throws Exception { + assertThat( + TEST_CODER.getEncodedTypeDescriptor(), equalTo(TypeDescriptor.of(FileWriteResult.class))); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOWriteTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOWriteTest.java new file mode 100644 index 000000000000..c77d162aafd4 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/IcebergIOWriteTest.java @@ -0,0 +1,293 @@ +/* + * 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.io.iceberg; + +import static org.apache.beam.io.iceberg.RowHelper.rowToRecord; +import static org.hamcrest.MatcherAssert.assertThat; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.commons.compress.utils.Lists; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.hamcrest.Matchers; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@RunWith(JUnit4.class) +public class IcebergIOWriteTest implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(IcebergIOWriteTest.class); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public transient TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); + + @Rule public transient TestPipeline testPipeline = TestPipeline.create(); + + @Test + public void testSimpleAppend() throws Exception { + TableIdentifier tableId = + TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); + + // Create a table and add records to it. + Table table = warehouse.createTable(tableId, TestFixtures.SCHEMA); + + IcebergCatalogConfig catalog = + IcebergCatalogConfig.builder() + .setName("hadoop") + .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .setWarehouseLocation(warehouse.location) + .build(); + + DynamicDestinations destination = DynamicDestinations.singleTable(tableId); + + testPipeline + .apply("Records To Add", Create.of(TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT1))) + .setRowSchema(SchemaHelper.convert(TestFixtures.SCHEMA)) + .apply("Append To Table", IcebergIO.writeToDynamicDestinations(catalog, destination)); + + LOG.info("Executing pipeline"); + testPipeline.run().waitUntilFinish(); + LOG.info("Done running pipeline"); + + List writtenRecords = ImmutableList.copyOf(IcebergGenerics.read(table).build()); + + assertThat(writtenRecords, Matchers.containsInAnyOrder(TestFixtures.FILE1SNAPSHOT1.toArray())); + } + + /** Tests that a small write to three different tables with dynamic destinations works. */ + @Test + public void testDynamicDestinationsWithoutSpillover() throws Exception { + final String salt = Long.toString(UUID.randomUUID().hashCode(), 16); + final TableIdentifier table1Id = TableIdentifier.of("default", "table1-" + salt); + final TableIdentifier table2Id = TableIdentifier.of("default", "table2-" + salt); + final TableIdentifier table3Id = TableIdentifier.of("default", "table3-" + salt); + + // Create a table and add records to it. + Table table1 = warehouse.createTable(table1Id, TestFixtures.SCHEMA); + Table table2 = warehouse.createTable(table2Id, TestFixtures.SCHEMA); + Table table3 = warehouse.createTable(table3Id, TestFixtures.SCHEMA); + + IcebergCatalogConfig catalog = + IcebergCatalogConfig.builder() + .setName("hadoop") + .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .setWarehouseLocation(warehouse.location) + .build(); + + DynamicDestinations destination = + new DynamicDestinations() { + private final Schema schema = Schema.builder().addInt64Field("tableNumber").build(); + + @Override + public Schema getMetadataSchema() { + return schema; + } + + @Override + public Row assignDestinationMetadata(Row data) { + long rowId = data.getInt64("id"); + return Row.withSchema(schema).addValues((rowId / 3) + 1).build(); + } + + @Override + public IcebergDestination instantiateDestination(Row dest) { + return IcebergDestination.builder() + .setTableIdentifier( + TableIdentifier.of( + "default", "table" + dest.getInt64("tableNumber") + "-" + salt)) + .setFileFormat(FileFormat.PARQUET) + .build(); + } + }; + + testPipeline + .apply( + "Records To Add", + Create.of( + TestFixtures.asRows( + Iterables.concat( + TestFixtures.FILE1SNAPSHOT1, + TestFixtures.FILE1SNAPSHOT2, + TestFixtures.FILE1SNAPSHOT3)))) + .setRowSchema(SchemaHelper.convert(TestFixtures.SCHEMA)) + .apply("Append To Table", IcebergIO.writeToDynamicDestinations(catalog, destination)); + + LOG.info("Executing pipeline"); + testPipeline.run().waitUntilFinish(); + LOG.info("Done running pipeline"); + + List writtenRecords1 = ImmutableList.copyOf(IcebergGenerics.read(table1).build()); + List writtenRecords2 = ImmutableList.copyOf(IcebergGenerics.read(table2).build()); + List writtenRecords3 = ImmutableList.copyOf(IcebergGenerics.read(table3).build()); + + assertThat(writtenRecords1, Matchers.containsInAnyOrder(TestFixtures.FILE1SNAPSHOT1.toArray())); + assertThat(writtenRecords2, Matchers.containsInAnyOrder(TestFixtures.FILE1SNAPSHOT2.toArray())); + assertThat(writtenRecords3, Matchers.containsInAnyOrder(TestFixtures.FILE1SNAPSHOT3.toArray())); + } + + /** + * Tests writing to enough destinations to spill over to the "slow" write path. + * + *

Note that we could have added a config to lower the spill number but exceeding the default + * is fine + */ + @Test + public void testDynamicDestinationsWithSpillover() throws Exception { + final String salt = Long.toString(UUID.randomUUID().hashCode(), 16); + + // Create far more tables than the max writers per bundle + int numDestinations = 5 * WriteUngroupedRowsToFiles.DEFAULT_MAX_WRITERS_PER_BUNDLE; + List tableIdentifiers = Lists.newArrayList(); + List tables = Lists.newArrayList(); + for (int i = 0; i < numDestinations; ++i) { + TableIdentifier id = TableIdentifier.of("default", "table" + i + "-" + salt); + tableIdentifiers.add(id); + tables.add(warehouse.createTable(id, TestFixtures.SCHEMA)); + } + + // Create plenty of data to hit each table + int numElements = 10 * numDestinations; + List elements = Lists.newArrayList(); + final Record genericRecord = GenericRecord.create(TestFixtures.SCHEMA); + Map> elementsPerTable = Maps.newHashMap(); + for (int i = 0; i < numElements; ++i) { + Record element = genericRecord.copy(ImmutableMap.of("id", (long) i, "data", "data for " + i)); + TableIdentifier tableId = tableIdentifiers.get(i % numDestinations); + elements.add(element); + elementsPerTable.computeIfAbsent(tableId, ignored -> Lists.newArrayList()).add(element); + } + + IcebergCatalogConfig catalog = + IcebergCatalogConfig.builder() + .setName("hadoop") + .setIcebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .setWarehouseLocation(warehouse.location) + .build(); + + DynamicDestinations destination = + new DynamicDestinations() { + private final Schema schema = Schema.builder().addInt64Field("tableNumber").build(); + + @Override + public Schema getMetadataSchema() { + return schema; + } + + @Override + public Row assignDestinationMetadata(Row data) { + long rowId = data.getInt64("id"); + return Row.withSchema(schema).addValues(rowId % numDestinations).build(); + } + + @Override + public IcebergDestination instantiateDestination(Row dest) { + return IcebergDestination.builder() + .setTableIdentifier( + TableIdentifier.of( + "default", "table" + dest.getInt64("tableNumber") + "-" + salt)) + .setFileFormat(FileFormat.PARQUET) + .build(); + } + }; + + testPipeline + .apply("Records To Add", Create.of(TestFixtures.asRows(elements))) + .setRowSchema(SchemaHelper.convert(TestFixtures.SCHEMA)) + .apply("Append To Table", IcebergIO.writeToDynamicDestinations(catalog, destination)); + + LOG.info("Executing pipeline"); + testPipeline.run().waitUntilFinish(); + LOG.info("Done running pipeline"); + + for (int i = 0; i < numDestinations; ++i) { + TableIdentifier tableId = tableIdentifiers.get(i); + Table table = tables.get(i); + List writtenRecords = ImmutableList.copyOf(IcebergGenerics.read(table).build()); + assertThat( + writtenRecords, Matchers.containsInAnyOrder(elementsPerTable.get(tableId).toArray())); + } + } + + /** + * A test of our assumptions about how two commits of the same file work in iceberg's Java API. + */ + @Test + public void testIdempotentCommit() throws Exception { + TableIdentifier tableId = + TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); + + // Create a table and add records to it. + Table table = warehouse.createTable(tableId, TestFixtures.SCHEMA); + Record record = + rowToRecord( + table.schema(), + Row.withSchema(SchemaHelper.convert(TestFixtures.SCHEMA)) + .addValues(42L, "bizzle") + .build()); + + OutputFile outputFile = table.io().newOutputFile(TEMPORARY_FOLDER.newFile().toString()); + DataWriter icebergDataWriter = + Parquet.writeData(outputFile) + .createWriterFunc(GenericParquetWriter::buildWriter) + .schema(table.schema()) + .withSpec(table.spec()) + .overwrite() + .build(); + + icebergDataWriter.write(record); + icebergDataWriter.close(); + DataFile dataFile = icebergDataWriter.toDataFile(); + + AppendFiles update = table.newAppend(); + update.appendFile(dataFile); + update.commit(); + + AppendFiles secondUpdate = table.newAppend(); + secondUpdate.appendFile(dataFile); + secondUpdate.commit(); + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/RowHelperTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/RowHelperTest.java new file mode 100644 index 000000000000..931937f407dd --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/RowHelperTest.java @@ -0,0 +1,128 @@ +/* + * 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.io.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +import java.io.Serializable; +import java.nio.ByteBuffer; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class RowHelperTest implements Serializable { + + /** + * Checks a value that when converted to Iceberg type is the same value when interpreted in Java. + */ + private void checkTypeConversion(Schema.FieldType sourceType, Type destType, Object value) { + checkTypeConversion(sourceType, value, destType, value); + } + + private void checkTypeConversion( + Schema.FieldType sourceType, Object sourceValue, Type destType, Object destValue) { + Schema beamSchema = Schema.of(Schema.Field.of("v", sourceType)); + Row row = Row.withSchema(beamSchema).addValues(sourceValue).build(); + + org.apache.iceberg.Schema icebergSchema = + new org.apache.iceberg.Schema(required(0, "v", destType)); + Record record = RowHelper.rowToRecord(icebergSchema, row); + + assertThat(record.getField("v"), equalTo(destValue)); + } + + @Test + public void testBoolean() throws Exception { + checkTypeConversion(Schema.FieldType.BOOLEAN, Types.BooleanType.get(), true); + checkTypeConversion(Schema.FieldType.BOOLEAN, Types.BooleanType.get(), false); + } + + @Test + public void testInteger() throws Exception { + checkTypeConversion(Schema.FieldType.INT32, Types.IntegerType.get(), -13); + checkTypeConversion(Schema.FieldType.INT32, Types.IntegerType.get(), 42); + checkTypeConversion(Schema.FieldType.INT32, Types.IntegerType.get(), 0); + } + + @Test + public void testLong() throws Exception { + checkTypeConversion(Schema.FieldType.INT64, Types.LongType.get(), 13L); + checkTypeConversion(Schema.FieldType.INT64, Types.LongType.get(), 42L); + } + + @Test + public void testFloat() throws Exception { + checkTypeConversion(Schema.FieldType.FLOAT, Types.FloatType.get(), 3.14159f); + checkTypeConversion(Schema.FieldType.FLOAT, Types.FloatType.get(), 42.0f); + } + + @Test + public void testDouble() throws Exception { + checkTypeConversion(Schema.FieldType.DOUBLE, Types.DoubleType.get(), 3.14159); + } + + @Test + public void testDate() throws Exception {} + + @Test + public void testTime() throws Exception {} + + @Test + public void testTimestamp() throws Exception { + DateTime dateTime = + new DateTime().withDate(1979, 03, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); + + checkTypeConversion( + Schema.FieldType.DATETIME, + dateTime.toInstant(), + Types.TimestampType.withoutZone(), + dateTime.getMillis()); + } + + @Test + public void testFixed() throws Exception {} + + @Test + public void testBinary() throws Exception { + byte[] bytes = new byte[] {1, 2, 3, 4}; + checkTypeConversion( + Schema.FieldType.BYTES, bytes, Types.BinaryType.get(), ByteBuffer.wrap(bytes)); + } + + @Test + public void testDecimal() throws Exception {} + + @Test + public void testStruct() throws Exception {} + + @Test + public void testMap() throws Exception {} + + @Test + public void testList() throws Exception {} +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/SinkTests.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/SinkTests.java deleted file mode 100644 index 8a0eaadab99b..000000000000 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/SinkTests.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.io.iceberg; - -import com.google.common.collect.ImmutableList; -import org.apache.beam.io.iceberg.util.RowHelper; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.values.Row; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.Table; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@RunWith(JUnit4.class) -public class SinkTests { - private static Logger LOG = LoggerFactory.getLogger(SinkTests.class); - @ClassRule public static final TemporaryFolder temporaryFolder = new TemporaryFolder(); - - @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(temporaryFolder, "default"); - - @Rule public TestPipeline testPipeline = TestPipeline.create(); - - @Test - public void testSimpleAppend() throws Exception { - // Create a table and add records to it. - Table table = warehouse.createTable(TestFixtures.SCHEMA); - - Iceberg.Catalog catalog = - Iceberg.Catalog.builder() - .name("hadoop") - .icebergCatalogType(CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .warehouseLocation(warehouse.location) - .build(); - - String[] tablePath = table.name().replace("hadoop.", "").split("\\."); - DynamicDestinations destination = - DynamicDestinations.constant( - catalog.table().tablePath(ImmutableList.copyOf(tablePath)).build()); - LOG.info("Table created. Making pipeline"); - testPipeline - .apply("Records To Add", Create.of(TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT1))) - .apply( - "Append To Table", - new Iceberg.Write(catalog, destination, RowHelper.recordsFromRows())); - LOG.info("Executing pipeline"); - testPipeline.run().waitUntilFinish(); - LOG.info("Done running pipeline"); - } -} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestDataWarehouse.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestDataWarehouse.java index f9112e5a61a3..0fc704cfc087 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestDataWarehouse.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestDataWarehouse.java @@ -22,7 +22,6 @@ import java.io.File; import java.io.IOException; import java.util.List; -import java.util.UUID; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -61,6 +60,7 @@ public class TestDataWarehouse extends ExternalResource { protected String location; protected Catalog catalog; + protected boolean someTableHasBeenCreated = false; public TestDataWarehouse(TemporaryFolder temporaryFolder, String database) { this.temporaryFolder = temporaryFolder; @@ -83,7 +83,12 @@ protected void before() throws Throwable { @Override protected void after() { + if (!someTableHasBeenCreated) { + return; + } + List tables = catalog.listTables(Namespace.of(database)); + LOG.info("Cleaning up {} tables in test warehouse", tables.size()); for (TableIdentifier t : tables) { try { @@ -134,9 +139,8 @@ public DataFile writeRecords(String filename, Schema schema, List record .build(); } - public Table createTable(Schema schema) { - TableIdentifier table = - TableIdentifier.of(database, "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); - return catalog.createTable(table, schema); + public Table createTable(TableIdentifier tableId, Schema schema) { + someTableHasBeenCreated = true; + return catalog.createTable(tableId, schema); } } diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestFixtures.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestFixtures.java index 05af551f7fc1..a39754fb7149 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestFixtures.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/io/iceberg/TestFixtures.java @@ -20,11 +20,10 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import java.util.ArrayList; -import org.apache.beam.io.iceberg.util.SchemaHelper; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.iceberg.Schema; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; @@ -45,9 +44,9 @@ public class TestFixtures { genericRecord.copy(ImmutableMap.of("id", 2L, "data", "falafel"))); public static final ImmutableList FILE1SNAPSHOT2 = ImmutableList.of( - genericRecord.copy(ImmutableMap.of("id", 4L, "data", "obscure")), - genericRecord.copy(ImmutableMap.of("id", 5L, "data", "secure")), - genericRecord.copy(ImmutableMap.of("id", 6L, "data", "feta"))); + genericRecord.copy(ImmutableMap.of("id", 3L, "data", "obscure")), + genericRecord.copy(ImmutableMap.of("id", 4L, "data", "secure")), + genericRecord.copy(ImmutableMap.of("id", 5L, "data", "feta"))); public static final ImmutableList FILE1SNAPSHOT3 = ImmutableList.of( genericRecord.copy(ImmutableMap.of("id", 6L, "data", "brainy")), diff --git a/settings.gradle.kts b/settings.gradle.kts index 632b283ec072..cca547c9e04e 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -357,5 +357,3 @@ include("sdks:java:managed") findProject(":sdks:java:managed")?.name = "managed" include("sdks:java:io:iceberg") findProject(":sdks:java:io:iceberg")?.name = "iceberg" -include("sdks:java:io:catalog") -findProject(":sdks:java:io:catalog")?.name = "catalog" From a7a6515cf3ffea1c49dbb2f22ac5828dc22ea226 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 8 Apr 2024 13:48:15 -0400 Subject: [PATCH 3/3] Add IcebergIO GitHub Action workflow --- .github/workflows/IO_Iceberg.yml | 118 +++++++++++++++++++++++++++++++ 1 file changed, 118 insertions(+) create mode 100644 .github/workflows/IO_Iceberg.yml diff --git a/.github/workflows/IO_Iceberg.yml b/.github/workflows/IO_Iceberg.yml new file mode 100644 index 000000000000..abc75836322c --- /dev/null +++ b/.github/workflows/IO_Iceberg.yml @@ -0,0 +1,118 @@ +# 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. + +name: IcebergIO Unit Tests + +on: + push: + tags: ['v*'] + branches: ['master', 'release-*'] + paths: + - "sdks/java/io/iceberg/**" + - ".github/workflows/IO_Iceberg.yml" + pull_request_target: + branches: ['master', 'release-*'] + paths: + - "sdks/java/io/iceberg/**" + - 'release/trigger_all_tests.json' + - '.github/trigger_files/IO_Iceberg.json' + issue_comment: + types: [created] + schedule: + - cron: '15 1/6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + IO_Iceberg: + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["IO_Iceberg"] + timeout-minutes: 60 + if: | + github.event_name == 'push' || + github.event_name == 'pull_request_target' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || + github.event_name == 'workflow_dispatch' || + github.event.comment.body == 'Run Java_Amqp_IO_Direct PreCommit' + runs-on: [self-hosted, ubuntu-20.04, main] + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + - name: run Amqp IO build script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:iceberg:build + arguments: | + -PdisableSpotlessCheck=true \ + -PdisableCheckStyle=true \ + - name: Archive JUnit Test Results + uses: actions/upload-artifact@v4 + if: ${{ !success() }} + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' + - name: Archive SpotBugs Results + uses: actions/upload-artifact@v4 + if: always() + with: + name: SpotBugs Results + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html'