Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix for #22951 #22953

Closed
wants to merge 15 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;

import java.util.Set;
import java.util.function.Function;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.GroupIntoBatchesPayload;
import org.apache.beam.sdk.runners.AppliedPTransform;
Expand All @@ -32,6 +34,8 @@
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PValues;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets;
import org.joda.time.Duration;
import org.junit.Test;
import org.junit.runner.RunWith;
Expand All @@ -43,12 +47,45 @@
@RunWith(Parameterized.class)
public class GroupIntoBatchesTranslationTest {
@Parameters(name = "{index}: {0}")
public static Iterable<GroupIntoBatches<?, ?>> transform() {
return ImmutableList.of(
GroupIntoBatches.ofSize(5),
GroupIntoBatches.ofSize(5).withMaxBufferingDuration(Duration.ZERO),
GroupIntoBatches.ofSize(5).withMaxBufferingDuration(Duration.standardSeconds(10)),
GroupIntoBatches.ofByteSize(10).withMaxBufferingDuration(Duration.standardSeconds(10)));
public static Iterable<GroupIntoBatches<String, Integer>> transform() {
Set<Function<GroupIntoBatches<String, Integer>, GroupIntoBatches<String, Integer>>> creators =
ImmutableSet.of( //
$ -> GroupIntoBatches.ofSize(5), //
$ -> GroupIntoBatches.ofByteSize(10));

Set<Function<GroupIntoBatches<String, Integer>, GroupIntoBatches<String, Integer>>>
sizeModifiers =
ImmutableSet.of( //
gib -> gib, //
gib -> gib.withSize(5));

Set<Function<GroupIntoBatches<String, Integer>, GroupIntoBatches<String, Integer>>>
byteSizeModifiers =
ImmutableSet.of( //
gib -> gib, //
gib -> gib.withByteSize(10), //
gib -> gib.withByteSize(10, i -> (long) i));

Set<Function<GroupIntoBatches<String, Integer>, GroupIntoBatches<String, Integer>>>
maxBufferingDurationModifiers =
ImmutableSet.of( //
gib -> gib, //
gib -> gib.withMaxBufferingDuration(Duration.ZERO), //
gib -> gib.withMaxBufferingDuration(Duration.standardSeconds(10)));

return Sets.cartesianProduct(
creators, sizeModifiers, byteSizeModifiers, maxBufferingDurationModifiers)
.stream()
.map(
product -> {
GroupIntoBatches<String, Integer> groupIntoBatches = null;
for (Function<GroupIntoBatches<String, Integer>, GroupIntoBatches<String, Integer>>
fn : product) {
groupIntoBatches = fn.apply(groupIntoBatches);
}
return groupIntoBatches;
})
.collect(ImmutableList.toImmutableList());
}

@Parameter(0)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,7 @@ public abstract class WriteFiles<UserT, DestinationT, OutputT>
// The record count and buffering duration to trigger flushing records to a tmp file. Mainly used
// for writing unbounded data to avoid generating too many small files.
private static final int FILE_TRIGGERING_RECORD_COUNT = 100000;
private static final int FILE_TRIGGERING_BYTE_COUNT = 64 * 1024 * 1024; // 64MiB as of now
private static final Duration FILE_TRIGGERING_RECORD_BUFFERING_DURATION =
Duration.standardSeconds(5);

Expand Down Expand Up @@ -767,6 +768,7 @@ public void processElement(@Element UserT element, ProcessContext context)
.apply(
"ShardAndBatch",
GroupIntoBatches.<Integer, UserT>ofSize(FILE_TRIGGERING_RECORD_COUNT)
.withByteSize(FILE_TRIGGERING_BYTE_COUNT)
.withMaxBufferingDuration(FILE_TRIGGERING_RECORD_BUFFERING_DURATION)
.withShardedKey())
.setCoder(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,11 @@ public class GroupIntoBatches<K, InputT>
*/
@AutoValue
public abstract static class BatchingParams<InputT> implements Serializable {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It looks like your adding support for GroupIntoBatches to limit on count and byte size at the same time.

Can you add tests that cover this new scenario to:

  • GroupIntoBatchesTest
  • GroupIntoBatchesTranslationTest

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

private static <InputT> BatchingParams<InputT> createDefault() {
return new AutoValue_GroupIntoBatches_BatchingParams(
Long.MAX_VALUE, Long.MAX_VALUE, null, Duration.ZERO);
}

public static <InputT> BatchingParams<InputT> create(
long batchSize,
long batchSizeBytes,
Expand Down Expand Up @@ -170,8 +175,7 @@ private GroupIntoBatches(BatchingParams<InputT> params) {
/** Aim to create batches each with the specified element count. */
public static <K, InputT> GroupIntoBatches<K, InputT> ofSize(long batchSize) {
Preconditions.checkState(batchSize < Long.MAX_VALUE);
return new GroupIntoBatches<>(
BatchingParams.create(batchSize, Long.MAX_VALUE, null, Duration.ZERO));
return new GroupIntoBatches<K, InputT>(BatchingParams.createDefault()).withSize(batchSize);
}

/**
Expand All @@ -185,9 +189,8 @@ public static <K, InputT> GroupIntoBatches<K, InputT> ofSize(long batchSize) {
* {@link #ofByteSize(long, SerializableFunction)} to specify code to calculate the byte size.
*/
public static <K, InputT> GroupIntoBatches<K, InputT> ofByteSize(long batchSizeBytes) {
Preconditions.checkState(batchSizeBytes < Long.MAX_VALUE);
return new GroupIntoBatches<>(
BatchingParams.create(Long.MAX_VALUE, batchSizeBytes, null, Duration.ZERO));
return new GroupIntoBatches<K, InputT>(BatchingParams.createDefault())
.withByteSize(batchSizeBytes);
}

/**
Expand All @@ -196,16 +199,54 @@ public static <K, InputT> GroupIntoBatches<K, InputT> ofByteSize(long batchSizeB
*/
public static <K, InputT> GroupIntoBatches<K, InputT> ofByteSize(
long batchSizeBytes, SerializableFunction<InputT, Long> getElementByteSize) {
Preconditions.checkState(batchSizeBytes < Long.MAX_VALUE);
return new GroupIntoBatches<>(
BatchingParams.create(Long.MAX_VALUE, batchSizeBytes, getElementByteSize, Duration.ZERO));
return new GroupIntoBatches<K, InputT>(BatchingParams.createDefault())
.withByteSize(batchSizeBytes, getElementByteSize);
}

/** Returns user supplied parameters for batching. */
public BatchingParams<InputT> getBatchingParams() {
return params;
}

@Override
public String toString() {
return super.toString() + ", params=" + params;
}

/** @see #ofSize(long) */
public GroupIntoBatches<K, InputT> withSize(long batchSize) {
Preconditions.checkState(batchSize < Long.MAX_VALUE);
return new GroupIntoBatches<>(
BatchingParams.create(
batchSize,
params.getBatchSizeBytes(),
params.getElementByteSize(),
params.getMaxBufferingDuration()));
}

/** @see #ofByteSize(long) */
public GroupIntoBatches<K, InputT> withByteSize(long batchSizeBytes) {
Preconditions.checkState(batchSizeBytes < Long.MAX_VALUE);
return new GroupIntoBatches<>(
BatchingParams.create(
params.getBatchSize(),
batchSizeBytes,
params.getElementByteSize(),
params.getMaxBufferingDuration()));
}

/** @see #ofByteSize(long, SerializableFunction) */
public GroupIntoBatches<K, InputT> withByteSize(
long batchSizeBytes, SerializableFunction<InputT, Long> getElementByteSize) {
Preconditions.checkState(batchSizeBytes < Long.MAX_VALUE);
return new GroupIntoBatches<>(
BatchingParams.create(
params.getBatchSize(),
batchSizeBytes,
getElementByteSize,
params.getMaxBufferingDuration()));
}

/**
* Sets a time limit (in processing time) on how long an incomplete batch of elements is allowed
* to be buffered. Once a batch is flushed to output, the timer is reset. The provided limit must
Expand Down Expand Up @@ -423,20 +464,56 @@ public void processElement(
@Timestamp Instant elementTs,
BoundedWindow window,
OutputReceiver<KV<K, Iterable<InputT>>> receiver) {

final boolean shouldCareAboutWeight = weigher != null && batchSizeBytes != Long.MAX_VALUE;
final boolean shouldCareAboutMaxBufferingDuration =
maxBufferingDuration.isLongerThan(Duration.ZERO);

if (shouldCareAboutWeight) {
storedBatchSizeBytes.readLater();
}
storedBatchSize.readLater();
if (shouldCareAboutMaxBufferingDuration) {
minBufferedTs.readLater();
}

LOG.debug("*** BATCH *** Add element for window {} ", window);
if (shouldCareAboutWeight) {
final long elementWeight = weigher.apply(element.getValue());
if (elementWeight + storedBatchSizeBytes.read() > batchSizeBytes) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this defeats the readLater optimization, since you're eagerly reading the value here (meaning also there's no point in the below readLater). you should add readLaters for minBufferedTs (if needed) and storedBatchSize earlier in the function.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TBH I wasn't sure how read()/readLater() implementation works. Like if we read it once, then will it be cached for the whole duration already or will it fetch it again.. but I assumed readLater() - as every prefetching method should be - is already optimized to be noop for already present values, so worst case is that we have an unnecessary noop call.

So to sum things up does that mean that a value returned by a read() call will be always available from that point forward? Anyway modified the PR/code to reflect this.

// Firing by count and size limits behave differently.
//
// We always increase the count by one, so we will fire at the exact limit without any
// overflow.
// before the addition: x < limit
// after the addition: x < limit OR x == limit(triggered)
//
// Meanwhile when we increase the batched byte size we do it with an undeterministic
// amount, so if we only check the limit after we already increased the batch size it
// could mean we fire over the limit.
// before the addition: x < limit
// after the addition: x < limit OR x == limit(triggered) OR x > limit(triggered)
// We shouldn't trigger a batch with bigger size than the config to limit it contains, so
// we fire it early if necessary.
LOG.debug("*** EARLY FIRE OF BATCH *** for window {}", window.toString());
flushBatch(
receiver,
element.getKey(),
batch,
storedBatchSize,
storedBatchSizeBytes,
timerTs,
minBufferedTs);
bufferingTimer.clear();
}
storedBatchSizeBytes.add(elementWeight);
}
batch.add(element.getValue());
// Blind add is supported with combiningState
storedBatchSize.add(1L);
if (weigher != null) {
storedBatchSizeBytes.add(weigher.apply(element.getValue()));
storedBatchSizeBytes.readLater();
}

long num;
if (maxBufferingDuration.isLongerThan(Duration.ZERO)) {
minBufferedTs.readLater();
num = storedBatchSize.read();

final long num = storedBatchSize.read();
if (shouldCareAboutMaxBufferingDuration) {
long oldOutputTs =
MoreObjects.firstNonNull(
minBufferedTs.read(), BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis());
Expand All @@ -455,15 +532,14 @@ public void processElement(
.set(Instant.ofEpochMilli(targetTs));
}
}
num = storedBatchSize.read();

if (num % prefetchFrequency == 0) {
// Prefetch data and modify batch state (readLater() modifies this)
batch.readLater();
}

if (num >= batchSize
|| (batchSizeBytes != Long.MAX_VALUE && storedBatchSizeBytes.read() >= batchSizeBytes)) {
|| (shouldCareAboutWeight && storedBatchSizeBytes.read() >= batchSizeBytes)) {
LOG.debug("*** END OF BATCH *** for window {}", window.toString());
flushBatch(
receiver,
Expand Down