forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 5
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- remove Read path (will propose separately) - re-enable checking, fix type errors - some style adjustments
- Loading branch information
1 parent
0437a8d
commit fd08eb4
Showing
48 changed files
with
2,081 additions
and
2,535 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file was deleted.
Oops, something went wrong.
21 changes: 0 additions & 21 deletions
21
sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/Catalog.java
This file was deleted.
Oops, something went wrong.
29 changes: 0 additions & 29 deletions
29
sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogEnvironment.java
This file was deleted.
Oops, something went wrong.
37 changes: 0 additions & 37 deletions
37
sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResource.java
This file was deleted.
Oops, something went wrong.
39 changes: 0 additions & 39 deletions
39
...va/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogResourceIdentifier.java
This file was deleted.
Oops, something went wrong.
26 changes: 0 additions & 26 deletions
26
sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSinkResource.java
This file was deleted.
Oops, something went wrong.
26 changes: 0 additions & 26 deletions
26
sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogSourceResource.java
This file was deleted.
Oops, something went wrong.
20 changes: 0 additions & 20 deletions
20
sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/CatalogTableResource.java
This file was deleted.
Oops, something went wrong.
1 change: 0 additions & 1 deletion
1
sdks/java/io/catalog/src/main/java/org/apache/beam/sdk/io/catalog/package-info.java
This file was deleted.
Oops, something went wrong.
102 changes: 102 additions & 0 deletions
102
sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/AppendFilesToTables.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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<FileWriteResult>, PCollection<KV<String, Snapshot>>> { | ||
|
||
private final IcebergCatalogConfig catalogConfig; | ||
|
||
AppendFilesToTables(IcebergCatalogConfig catalogConfig) { | ||
this.catalogConfig = catalogConfig; | ||
} | ||
|
||
@Override | ||
public PCollection<KV<String, Snapshot>> expand(PCollection<FileWriteResult> writtenFiles) { | ||
|
||
// Apply any sharded writes and flatten everything for catalog updates | ||
return writtenFiles | ||
.apply( | ||
"Key metadata updates by table", | ||
WithKeys.of( | ||
new SerializableFunction<FileWriteResult, String>() { | ||
@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<String, Iterable<FileWriteResult>>, KV<String, Snapshot>> { | ||
|
||
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<String, Iterable<FileWriteResult>> element, | ||
OutputReceiver<KV<String, Snapshot>> 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()); | ||
} | ||
} | ||
} |
Oops, something went wrong.