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

Filter by extension

Filter by extension

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

import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;

import java.util.ArrayList;
import java.util.List;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver;
import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
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.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.TupleTagList;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* A PTransform that buffers elements and outputs them to one of two TupleTags based on the total
* size of the bundle in finish_bundle.
*
* <p>This is the Java equivalent of the BundleLifter PTransform in Python.
*
* @param <T> The type of elements in the input PCollection.
*/
public class BundleLifter<T> extends PTransform<PCollection<T>, PCollectionTuple> {

final TupleTag<T> smallBatchTag;
final TupleTag<T> largeBatchTag;
final int threshold;
final SerializableFunction<T, Integer> elementSizer;

/**
* A private, static DoFn that buffers elements within a bundle and outputs them to different tags
* in finish_bundle based on the total bundle size.
*
* <p>This is the Java equivalent of the _BundleLiftDoFn in Python, now merged inside the
* PTransform.
*
* @param <T> The type of elements being processed.
*/
private static class BundleLiftDoFn<T> extends DoFn<T, Void> {
private static final Logger LOG = LoggerFactory.getLogger(BundleLiftDoFn.class);

final TupleTag<T> smallBatchTag;
final TupleTag<T> largeBatchTag;
final int threshold;
final SerializableFunction<T, Integer> elementSizer;

private transient @MonotonicNonNull List<T> buffer;
private transient long bundleSize;
private transient @Nullable MultiOutputReceiver receiver;

BundleLiftDoFn(
TupleTag<T> smallBatchTag,
TupleTag<T> largeBatchTag,
int threshold,
SerializableFunction<T, Integer> elementSizer) {
this.smallBatchTag = smallBatchTag;
this.largeBatchTag = largeBatchTag;
this.threshold = threshold;
this.elementSizer = elementSizer;
}

@StartBundle
public void startBundle() {
buffer = new ArrayList<>();
receiver = null;
bundleSize = 0L;
}

@ProcessElement
public void processElement(@Element T element, MultiOutputReceiver mor) {
if (receiver == null) {
receiver = mor;
}
checkArgumentNotNull(buffer, "Buffer should be set by startBundle.");
buffer.add(element);
bundleSize += elementSizer.apply(element);
}

@FinishBundle
public void finishBundle() {
checkArgumentNotNull(buffer, "Buffer should be set by startBundle.");
if (buffer.isEmpty()) {
return;
}

TupleTag<T> targetTag;

// Select the target tag based on the bundle size
if (bundleSize < threshold) {
targetTag = smallBatchTag;
LOG.debug("Emitting {} elements to small tag: '{}'", bundleSize, targetTag.getId());
} else {
targetTag = largeBatchTag;
LOG.debug("Emitting {} elements to large tag: '{}'", bundleSize, targetTag.getId());
}

checkArgumentNotNull(receiver, "Receiver should be set by startBundle.");
OutputReceiver<T> taggedOutput = receiver.get(targetTag);

for (T element : buffer) {
taggedOutput.output(element);
}
}
}

public BundleLifter(TupleTag<T> smallBatchTag, TupleTag<T> largeBatchTag, int threshold) {
this(smallBatchTag, largeBatchTag, threshold, x -> 1);
}

public BundleLifter(
TupleTag<T> smallBatchTag,
TupleTag<T> largeBatchTag,
int threshold,
SerializableFunction<T, Integer> elementSizer) {
if (smallBatchTag == null || largeBatchTag == null) {
throw new IllegalArgumentException("smallBatchTag and largeBatchTag must not be null");
}
if (smallBatchTag.getId().equals(largeBatchTag.getId())) {
throw new IllegalArgumentException("smallBatchTag and largeBatchTag must be different");
}
if (threshold <= 0) {
throw new IllegalArgumentException("Threshold must be a positive integer");
}

this.smallBatchTag = smallBatchTag;
this.largeBatchTag = largeBatchTag;
this.threshold = threshold;
this.elementSizer = elementSizer;
}

public static <T> BundleLifter<T> of(
TupleTag<T> smallBatchTag, TupleTag<T> largeBatchTag, int threshold) {
return new BundleLifter<>(smallBatchTag, largeBatchTag, threshold);
}

public static <T> BundleLifter<T> of(
TupleTag<T> smallBatchTag,
TupleTag<T> largeBatchTag,
int threshold,
SerializableFunction<T, Integer> elementSizer) {
return new BundleLifter<>(smallBatchTag, largeBatchTag, threshold, elementSizer);
}

@Override
public PCollectionTuple expand(PCollection<T> input) {
final TupleTag<Void> mainOutputTag = new TupleTag<Void>() {};

return input.apply(
"BundleLiftDoFn",
ParDo.of(new BundleLiftDoFn<>(smallBatchTag, largeBatchTag, threshold, elementSizer))
.withOutputTags(mainOutputTag, TupleTagList.of(smallBatchTag).and(largeBatchTag)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -395,6 +395,8 @@ public abstract static class WriteRows extends PTransform<PCollection<Row>, Iceb

abstract @Nullable Duration getTriggeringFrequency();

abstract @Nullable Integer getDirectWriteByteLimit();

abstract Builder toBuilder();

@AutoValue.Builder
Expand All @@ -407,6 +409,8 @@ abstract static class Builder {

abstract Builder setTriggeringFrequency(Duration triggeringFrequency);

abstract Builder setDirectWriteByteLimit(Integer directWriteByteLimit);

abstract WriteRows build();
}

Expand Down Expand Up @@ -435,6 +439,10 @@ public WriteRows withTriggeringFrequency(Duration triggeringFrequency) {
return toBuilder().setTriggeringFrequency(triggeringFrequency).build();
}

public WriteRows withDirectWriteByteLimit(Integer directWriteByteLimit) {
return toBuilder().setDirectWriteByteLimit(directWriteByteLimit).build();
}

@Override
public IcebergWriteResult expand(PCollection<Row> input) {
List<?> allToArgs = Arrays.asList(getTableIdentifier(), getDynamicDestinations());
Expand All @@ -455,7 +463,11 @@ public IcebergWriteResult expand(PCollection<Row> input) {
.apply("Assign Table Destinations", new AssignDestinations(destinations))
.apply(
"Write Rows to Destinations",
new WriteToDestinations(getCatalogConfig(), destinations, getTriggeringFrequency()));
new WriteToDestinations(
getCatalogConfig(),
destinations,
getTriggeringFrequency(),
getDirectWriteByteLimit()));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,10 @@ public static Builder builder() {
"For a streaming pipeline, sets the frequency at which snapshots are produced.")
public abstract @Nullable Integer getTriggeringFrequencySeconds();

@SchemaFieldDescription(
"For a streaming pipeline, sets the limit for lifting bundles into the direct write path.")
public abstract @Nullable Integer getDirectWriteByteLimit();

@SchemaFieldDescription(
"A list of field names to keep in the input record. All other fields are dropped before writing. "
+ "Is mutually exclusive with 'drop' and 'only'.")
Expand Down Expand Up @@ -142,6 +146,8 @@ public abstract static class Builder {

public abstract Builder setTriggeringFrequencySeconds(Integer triggeringFrequencySeconds);

public abstract Builder setDirectWriteByteLimit(Integer directWriteByteLimit);

public abstract Builder setKeep(List<String> keep);

public abstract Builder setDrop(List<String> drop);
Expand Down Expand Up @@ -227,6 +233,11 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) {
writeTransform = writeTransform.withTriggeringFrequency(Duration.standardSeconds(trigFreq));
}

Integer directWriteByteLimit = configuration.getDirectWriteByteLimit();
if (directWriteByteLimit != null) {
writeTransform = writeTransform.withDirectWriteByteLimit(directWriteByteLimit);
}

// TODO: support dynamic destinations
IcebergWriteResult result = rows.apply(writeTransform);

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

import java.util.List;
import java.util.Map;
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.transforms.windowing.PaneInfo;
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.WindowedValue;
import org.apache.beam.sdk.values.WindowedValues;
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.Iterables;
import org.apache.iceberg.catalog.Catalog;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import org.checkerframework.checker.nullness.qual.Nullable;

class WriteDirectRowsToFiles
extends PTransform<PCollection<KV<String, Row>>, PCollection<FileWriteResult>> {

private final DynamicDestinations dynamicDestinations;
private final IcebergCatalogConfig catalogConfig;
private final String filePrefix;
private final long maxBytesPerFile;

WriteDirectRowsToFiles(
IcebergCatalogConfig catalogConfig,
DynamicDestinations dynamicDestinations,
String filePrefix,
long maxBytesPerFile) {
this.catalogConfig = catalogConfig;
this.dynamicDestinations = dynamicDestinations;
this.filePrefix = filePrefix;
this.maxBytesPerFile = maxBytesPerFile;
}

@Override
public PCollection<FileWriteResult> expand(PCollection<KV<String, Row>> input) {
return input.apply(
ParDo.of(
new WriteDirectRowsToFilesDoFn(
catalogConfig, dynamicDestinations, maxBytesPerFile, filePrefix)));
}

private static class WriteDirectRowsToFilesDoFn extends DoFn<KV<String, Row>, FileWriteResult> {

private final DynamicDestinations dynamicDestinations;
private final IcebergCatalogConfig catalogConfig;
private transient @MonotonicNonNull Catalog catalog;
private final String filePrefix;
private final long maxFileSize;
private transient @Nullable RecordWriterManager recordWriterManager;

WriteDirectRowsToFilesDoFn(
IcebergCatalogConfig catalogConfig,
DynamicDestinations dynamicDestinations,
long maxFileSize,
String filePrefix) {
this.catalogConfig = catalogConfig;
this.dynamicDestinations = dynamicDestinations;
this.filePrefix = filePrefix;
this.maxFileSize = maxFileSize;
this.recordWriterManager = null;
}

private org.apache.iceberg.catalog.Catalog getCatalog() {
if (catalog == null) {
this.catalog = catalogConfig.catalog();
}
return catalog;
}

@StartBundle
public void startBundle() {
recordWriterManager =
new RecordWriterManager(getCatalog(), filePrefix, maxFileSize, Integer.MAX_VALUE);
}

@ProcessElement
public void processElement(
ProcessContext context,
@Element KV<String, Row> element,
BoundedWindow window,
PaneInfo paneInfo)
throws Exception {
String tableIdentifier = element.getKey();
IcebergDestination destination = dynamicDestinations.instantiateDestination(tableIdentifier);
WindowedValue<IcebergDestination> windowedDestination =
WindowedValues.of(destination, window.maxTimestamp(), window, paneInfo);
Preconditions.checkNotNull(recordWriterManager)
.write(windowedDestination, element.getValue());
}

@FinishBundle
public void finishBundle(FinishBundleContext context) throws Exception {
if (recordWriterManager == null) {
return;
}
recordWriterManager.close();

for (Map.Entry<WindowedValue<IcebergDestination>, List<SerializableDataFile>>
destinationAndFiles :
Preconditions.checkNotNull(recordWriterManager)
.getSerializableDataFiles()
.entrySet()) {
WindowedValue<IcebergDestination> windowedDestination = destinationAndFiles.getKey();

for (SerializableDataFile dataFile : destinationAndFiles.getValue()) {
context.output(
FileWriteResult.builder()
.setSerializableDataFile(dataFile)
.setTableIdentifier(windowedDestination.getValue().getTableIdentifier())
.build(),
windowedDestination.getTimestamp(),
Iterables.getFirst(windowedDestination.getWindows(), null));
}
}
recordWriterManager = null;
}
}
}
Loading
Loading