Skip to content

Commit

Permalink
Dynamically choose number of shards in the InProcessPipelineRunner (a…
Browse files Browse the repository at this point in the history
…pache#400)

* Add Additional Exists check to FileIOChannelFactory#create

This ensures that if the folder did not exist when first checked, but
did by the time mkdirs was executed (and thus mkdirs returned false) the
create will not fail.

* Dynamically choose number of shards in the InProcessPipelineRunner

Add a Write Override Factory that limits the number of shards if
unspecified. This ensures that we will not write an output file per-key
due to bundling.

Do so by obtaining a count of the elements and obtaining the number of
shards based on the number of outputs.
  • Loading branch information
tgroh authored and dhalperi committed Aug 26, 2016
1 parent b79bb74 commit f8f4701
Show file tree
Hide file tree
Showing 6 changed files with 441 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import com.google.cloud.dataflow.sdk.Pipeline.PipelineExecutionException;
import com.google.cloud.dataflow.sdk.PipelineResult;
import com.google.cloud.dataflow.sdk.annotations.Experimental;
import com.google.cloud.dataflow.sdk.io.Write;
import com.google.cloud.dataflow.sdk.options.PipelineOptions;
import com.google.cloud.dataflow.sdk.runners.AggregatorPipelineExtractor;
import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
Expand Down Expand Up @@ -50,9 +51,7 @@
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;

import org.joda.time.Instant;

import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
Expand All @@ -78,6 +77,7 @@ public class InProcessPipelineRunner
.put(Create.Values.class, new InProcessCreateOverrideFactory())
.put(GroupByKey.class, new InProcessGroupByKeyOverrideFactory())
.put(CreatePCollectionView.class, new InProcessViewOverrideFactory())
.put(Write.Bound.class, new WriteWithShardingFactory())
.build();

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,134 @@
/*
* Copyright (C) 2016 Google Inc.
*
* Licensed 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 com.google.cloud.dataflow.sdk.runners.inprocess;

import static com.google.common.base.Preconditions.checkArgument;

import com.google.cloud.dataflow.sdk.io.Write;
import com.google.cloud.dataflow.sdk.transforms.Count;
import com.google.cloud.dataflow.sdk.transforms.DoFn;
import com.google.cloud.dataflow.sdk.transforms.Flatten;
import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
import com.google.cloud.dataflow.sdk.transforms.PTransform;
import com.google.cloud.dataflow.sdk.transforms.ParDo;
import com.google.cloud.dataflow.sdk.transforms.Values;
import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger;
import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows;
import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
import com.google.cloud.dataflow.sdk.values.KV;
import com.google.cloud.dataflow.sdk.values.PCollection;
import com.google.cloud.dataflow.sdk.values.PCollectionView;
import com.google.cloud.dataflow.sdk.values.PDone;
import com.google.cloud.dataflow.sdk.values.PInput;
import com.google.cloud.dataflow.sdk.values.POutput;
import com.google.common.annotations.VisibleForTesting;
import org.joda.time.Duration;
import java.util.concurrent.ThreadLocalRandom;

/**
* A {@link PTransformOverrideFactory} that overrides {@link Write} {@link PTransform PTransforms}
* with an unspecified number of shards with a write with a specified number of shards. The number
* of shards is the log base 10 of the number of input records, with up to 2 additional shards.
*/
class WriteWithShardingFactory implements PTransformOverrideFactory {
static final int MAX_RANDOM_EXTRA_SHARDS = 3;

@Override
public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
PTransform<InputT, OutputT> transform) {
if (transform instanceof Write.Bound) {
Write.Bound<InputT> that = (Write.Bound<InputT>) transform;
if (that.getNumShards() == 0) {
return (PTransform<InputT, OutputT>) new DynamicallyReshardedWrite<InputT>(that);
}
}
return transform;
}

private static class DynamicallyReshardedWrite <T> extends PTransform<PCollection<T>, PDone> {
private final transient Write.Bound<T> original;

private DynamicallyReshardedWrite(Write.Bound<T> original) {
this.original = original;
}

@Override
public PDone apply(PCollection<T> input) {
PCollection<T> records = input.apply("RewindowInputs",
Window.<T>into(new GlobalWindows()).triggering(DefaultTrigger.of())
.withAllowedLateness(Duration.ZERO)
.discardingFiredPanes());
final PCollectionView<Long> numRecords = records.apply(Count.<T>globally().asSingletonView());
PCollection<T> resharded =
records
.apply(
"ApplySharding",
ParDo.withSideInputs(numRecords)
.of(
new KeyBasedOnCountFn<T>(
numRecords,
ThreadLocalRandom.current().nextInt(MAX_RANDOM_EXTRA_SHARDS))))
.apply("GroupIntoShards", GroupByKey.<Integer, T>create())
.apply("DropShardingKeys", Values.<Iterable<T>>create())
.apply("FlattenShardIterables", Flatten.<T>iterables());
// This is an inverted application to apply the expansion of the original Write PTransform
// without adding a new Write Transform Node, which would be overwritten the same way, leading
// to an infinite recursion. We cannot modify the number of shards, because that is determined
// at runtime.
return original.apply(resharded);
}
}

@VisibleForTesting
static class KeyBasedOnCountFn<T> extends DoFn<T, KV<Integer, T>> {
@VisibleForTesting
static final int MIN_SHARDS_FOR_LOG = 3;

private final PCollectionView<Long> numRecords;
private final int randomExtraShards;
private int currentShard;
private int maxShards;

KeyBasedOnCountFn(PCollectionView<Long> numRecords, int extraShards) {
this.numRecords = numRecords;
this.randomExtraShards = extraShards;
}

@Override
public void processElement(ProcessContext c) throws Exception {
if (maxShards == 0L) {
maxShards = calculateShards(c.sideInput(numRecords));
currentShard = ThreadLocalRandom.current().nextInt(maxShards);
}
int shard = currentShard;
currentShard = (currentShard + 1) % maxShards;
c.output(KV.of(shard, c.element()));
}

private int calculateShards(long totalRecords) {
checkArgument(
totalRecords > 0,
"KeyBasedOnCountFn cannot be invoked on an element if there are no elements");
if (totalRecords < MIN_SHARDS_FOR_LOG + randomExtraShards) {
return (int) totalRecords;
}
// 100mil records before >7 output files
int floorLogRecs = Double.valueOf(Math.log10(totalRecords)).intValue();
int shards = Math.max(floorLogRecs, MIN_SHARDS_FOR_LOG) + randomExtraShards;
return shards;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,8 @@ public WritableByteChannel create(String spec, String mimeType)
File file = new File(spec);
if (file.getAbsoluteFile().getParentFile() != null
&& !file.getAbsoluteFile().getParentFile().exists()
&& !file.getAbsoluteFile().getParentFile().mkdirs()) {
&& !file.getAbsoluteFile().getParentFile().mkdirs()
&& !file.getAbsoluteFile().getParentFile().exists()) {
throw new IOException("Unable to create parent directories for '" + spec + "'");
}
return Channels.newChannel(
Expand Down
18 changes: 13 additions & 5 deletions sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -269,11 +269,19 @@ public static <T> void assertOutputFiles(
String shardNameTemplate)
throws Exception {
List<File> expectedFiles = new ArrayList<>();
for (int i = 0; i < numShards; i++) {
expectedFiles.add(
new File(
rootLocation.getRoot(),
IOChannelUtils.constructName(outputName, shardNameTemplate, "", i, numShards)));
if (numShards == 0) {
String pattern =
IOChannelUtils.resolve(rootLocation.getRoot().getAbsolutePath(), outputName + "*");
for (String expected : IOChannelUtils.getFactory(pattern).match(pattern)) {
expectedFiles.add(new File(expected));
}
} else {
for (int i = 0; i < numShards; i++) {
expectedFiles.add(
new File(
rootLocation.getRoot(),
IOChannelUtils.constructName(outputName, shardNameTemplate, "", i, numShards)));
}
}

List<String> actual = new ArrayList<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.is;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
Expand Down Expand Up @@ -137,7 +138,7 @@ public void testWrite() {
public void testEmptyWrite() {
runWrite(Collections.<String>emptyList(), IDENTITY_MAP);
// Note we did not request a sharded write, so runWrite will not validate the number of shards.
assertEquals(1, numShards.intValue());
assertThat(numShards.intValue(), greaterThan(0));
}

/**
Expand Down
Loading

0 comments on commit f8f4701

Please sign in to comment.