Skip to content

Commit

Permalink
Merge pull request apache#4717: Make Impulse#create() visible
Browse files Browse the repository at this point in the history
[BEAM-2421] Make Impulse#create() visible
  • Loading branch information
tgroh authored Mar 6, 2018
2 parents 5b99fe2 + adcf951 commit d51b288
Show file tree
Hide file tree
Showing 8 changed files with 485 additions and 90 deletions.
Original file line number Diff line number Diff line change
@@ -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.runners.core.construction;

import com.google.common.annotations.VisibleForTesting;
import java.io.IOException;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.SerializableCoder;
import org.apache.beam.sdk.io.BoundedSource;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.Impulse;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.Reshuffle;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;

/**
* Read from a Java {@link BoundedSource} via the {@link Impulse} and {@link ParDo} primitive
* transforms.
*/
public class JavaReadViaImpulse {
private static final long DEFAULT_BUNDLE_SIZE = 64L << 20;

public static <T> PTransform<PBegin, PCollection<T>> bounded(BoundedSource<T> source) {
return new BoundedReadViaImpulse<>(source);
}

private static class BoundedReadViaImpulse<T> extends PTransform<PBegin, PCollection<T>> {
private final BoundedSource<T> source;

private BoundedReadViaImpulse(BoundedSource<T> source) {
this.source = source;
}

@Override
public PCollection<T> expand(PBegin input) {
return input
.apply(Impulse.create())
.apply(ParDo.of(new SplitBoundedSourceFn<>(source, DEFAULT_BUNDLE_SIZE)))
.setCoder((Coder<BoundedSource<T>>) SerializableCoder.of((Class) BoundedSource.class))
.apply(Reshuffle.viaRandomKey())
.apply(ParDo.of(new ReadFromBoundedSourceFn<>()))
.setCoder(source.getOutputCoder());
}
}

@VisibleForTesting
static class SplitBoundedSourceFn<T> extends DoFn<byte[], BoundedSource<T>> {
private final BoundedSource<T> source;
private final long bundleSize;

public SplitBoundedSourceFn(BoundedSource<T> source, long bundleSize) {
this.source = source;
this.bundleSize = bundleSize;
}

@ProcessElement
public void splitSource(ProcessContext ctxt) throws Exception {
for (BoundedSource<T> split : source.split(bundleSize, ctxt.getPipelineOptions())) {
ctxt.output(split);
}
}
}

/** Reads elements contained within an input {@link BoundedSource}. */
// TODO: Extend to be a Splittable DoFn.
@VisibleForTesting
static class ReadFromBoundedSourceFn<T> extends DoFn<BoundedSource<T>, T> {
@ProcessElement
public void readSoruce(ProcessContext ctxt) throws IOException {
BoundedSource.BoundedReader<T> reader =
ctxt.element().createReader(ctxt.getPipelineOptions());
for (boolean more = reader.start(); more; more = reader.advance()) {
ctxt.outputWithTimestamp(reader.getCurrent(), reader.getCurrentTimestamp());
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,131 @@
/*
* 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.runners.core.construction;

import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.junit.Assert.assertThat;

import com.google.common.collect.Iterables;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import org.apache.beam.runners.core.construction.JavaReadViaImpulse.ReadFromBoundedSourceFn;
import org.apache.beam.runners.core.construction.JavaReadViaImpulse.SplitBoundedSourceFn;
import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.SerializableCoder;
import org.apache.beam.sdk.io.BoundedSource;
import org.apache.beam.sdk.io.CountingSource;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.testing.NeedsRunner;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.Impulse;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.PCollection;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/** Tests for {@link JavaReadViaImpulse}. */
@RunWith(JUnit4.class)
public class JavaReadViaImpulseTest {
@Rule public TestPipeline p = TestPipeline.create();

@Test
@Category(NeedsRunner.class)
public void testBoundedRead() {
PCollection<Long> read = p.apply(JavaReadViaImpulse.bounded(CountingSource.upTo(10L)));

PAssert.that(read).containsInAnyOrder(0L, 9L, 8L, 1L, 2L, 7L, 6L, 3L, 4L, 5L);
p.run();
}

@Test
@Category(NeedsRunner.class)
public void testSplitSourceFn() {
PCollection<BoundedSource<Long>> splits =
p.apply(Impulse.create())
.apply(
"SplitSource",
/*
* Split the source of 1 million longs into bundles of size 300 thousand bytes.
* This should produce some small number of bundles, but more than one.
*/
ParDo.of(new SplitBoundedSourceFn<>(CountingSource.upTo(1_000_000L), 300_000L)));

PAssert.that(splits)
.satisfies(
input -> {
assertThat(Iterables.size(input), greaterThan(1));
return null;
});

p.run();
}

@Test
@Category(NeedsRunner.class)
public void testReadFromSourceFn() {
BoundedSource<Long> source = CountingSource.upTo(10L);
PCollection<BoundedSource<Long>> sourcePC =
(PCollection)
p.apply(Create.of(source).withCoder(SerializableCoder.of((Class) BoundedSource.class)));
PCollection<Long> elems = sourcePC.apply(ParDo.of(new ReadFromBoundedSourceFn<>()));

PAssert.that(elems).containsInAnyOrder(0L, 9L, 8L, 1L, 2L, 7L, 6L, 3L, 4L, 5L);
p.run();
}

@Test
public void testOutputCoder() {
p.enableAbandonedNodeEnforcement(false);
BoundedSource<Integer> fixedCoderSource = new BigEndianIntegerSource();
assertThat(
p.apply(JavaReadViaImpulse.bounded(fixedCoderSource)).getCoder(),
equalTo(BigEndianIntegerCoder.of()));
}

private static class BigEndianIntegerSource extends BoundedSource<Integer> {
@Override
public List<? extends BoundedSource<Integer>> split(
long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
return Collections.singletonList(this);
}

@Override
public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
return 0;
}

@Override
public BoundedReader<Integer> createReader(PipelineOptions options) throws IOException {
throw new AssertionError("Not the point");
}

@Override
public Coder<Integer> getOutputCoder() {
return BigEndianIntegerCoder.of();
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
/*
* 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.runners.direct;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Iterables;
import java.util.Collection;
import java.util.Collections;
import javax.annotation.Nullable;
import org.apache.beam.sdk.runners.AppliedPTransform;
import org.apache.beam.sdk.transforms.Impulse;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;

/** The evaluator for the {@link Impulse} transform. Produces only empty byte arrays. */
class ImpulseEvaluatorFactory implements TransformEvaluatorFactory {
private final EvaluationContext ctxt;

ImpulseEvaluatorFactory(EvaluationContext ctxt) {
this.ctxt = ctxt;
}

@Nullable
@Override
public <InputT> TransformEvaluator<InputT> forApplication(
AppliedPTransform<?, ?, ?> application, CommittedBundle<?> inputBundle) {
return (TransformEvaluator<InputT>) new ImpulseEvaluator(ctxt, (AppliedPTransform) application);
}

@Override
public void cleanup() {
// Impulse has no state, so do nothing.
}

private static class ImpulseEvaluator implements TransformEvaluator<ImpulseShard> {
private final EvaluationContext ctxt;
private final AppliedPTransform<?, PCollection<byte[]>, Impulse> transform;
private final StepTransformResult.Builder<ImpulseShard> result;

private ImpulseEvaluator(
EvaluationContext ctxt, AppliedPTransform<?, PCollection<byte[]>, Impulse> transform) {
this.ctxt = ctxt;
this.transform = transform;
this.result = StepTransformResult.withoutHold(transform);
}

@Override
public void processElement(WindowedValue<ImpulseShard> element) throws Exception {
PCollection<byte[]> outputPCollection =
(PCollection<byte[]>) Iterables.getOnlyElement(transform.getOutputs().values());
result.addOutput(
ctxt.createBundle(outputPCollection).add(WindowedValue.valueInGlobalWindow(new byte[0])));
}

@Override
public TransformResult<ImpulseShard> finishBundle() throws Exception {
return result.build();
}
}

/**
* The {@link RootInputProvider} for the {@link Impulse} {@link PTransform}. Produces a single
* {@link ImpulseShard}.
*/
static class ImpulseRootProvider implements RootInputProvider<byte[], ImpulseShard, PBegin> {
private final EvaluationContext ctxt;

ImpulseRootProvider(EvaluationContext ctxt) {
this.ctxt = ctxt;
}

@Override
public Collection<CommittedBundle<ImpulseShard>> getInitialInputs(
AppliedPTransform<PBegin, PCollection<byte[]>, PTransform<PBegin, PCollection<byte[]>>>
transform,
int targetParallelism) {
return Collections.singleton(
ctxt.<ImpulseShard>createRootBundle()
.add(WindowedValue.valueInGlobalWindow(new ImpulseShard()))
.commit(BoundedWindow.TIMESTAMP_MIN_VALUE));
}
}

@VisibleForTesting
static class ImpulseShard {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.beam.runners.core.construction.PTransformTranslation.FLATTEN_TRANSFORM_URN;
import static org.apache.beam.runners.core.construction.PTransformTranslation.IMPULSE_TRANSFORM_URN;
import static org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory.DIRECT_TEST_STREAM_URN;

import com.google.common.collect.ImmutableMap;
Expand All @@ -37,6 +38,7 @@ public static RootProviderRegistry defaultRegistry(EvaluationContext context) {
ImmutableMap.Builder<String, RootInputProvider<?, ?, ?>>
defaultProviders = ImmutableMap.builder();
defaultProviders
.put(IMPULSE_TRANSFORM_URN, new ImpulseEvaluatorFactory.ImpulseRootProvider(context))
.put(PTransformTranslation.READ_TRANSFORM_URN, ReadEvaluatorFactory.inputProvider(context))
.put(DIRECT_TEST_STREAM_URN, new TestStreamEvaluatorFactory.InputProvider(context))
.put(FLATTEN_TRANSFORM_URN, new EmptyInputProvider());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import static com.google.common.base.Preconditions.checkState;
import static org.apache.beam.runners.core.construction.PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN;
import static org.apache.beam.runners.core.construction.PTransformTranslation.FLATTEN_TRANSFORM_URN;
import static org.apache.beam.runners.core.construction.PTransformTranslation.IMPULSE_TRANSFORM_URN;
import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN;
import static org.apache.beam.runners.core.construction.PTransformTranslation.READ_TRANSFORM_URN;
import static org.apache.beam.runners.core.construction.SplittableParDo.SPLITTABLE_PROCESS_URN;
Expand Down Expand Up @@ -69,6 +70,7 @@ public static TransformEvaluatorRegistry defaultRegistry(EvaluationContext ctxt)
ParDoEvaluatorFactory.basicDoFnCacheLoader()))
.put(FLATTEN_TRANSFORM_URN, new FlattenEvaluatorFactory(ctxt))
.put(ASSIGN_WINDOWS_TRANSFORM_URN, new WindowEvaluatorFactory(ctxt))
.put(IMPULSE_TRANSFORM_URN, new ImpulseEvaluatorFactory(ctxt))

// Runner-specific primitives
.put(DIRECT_WRITE_VIEW_URN, new ViewEvaluatorFactory(ctxt))
Expand Down
Loading

0 comments on commit d51b288

Please sign in to comment.