diff --git a/pom.xml b/pom.xml index 4ecfdd7c268c0..09f39859b0416 100644 --- a/pom.xml +++ b/pom.xml @@ -119,7 +119,7 @@ 1.4.5 0.5.160304 20.0 - 1.0.1 + 1.2.0 0.1.0 1.3 2.7.2 @@ -127,9 +127,9 @@ 2.4 4.12 1.9.5 - 4.1.6.Final + 4.1.8.Final 1.5.0.Final - 3.1.0 + 3.2.0 v1-rev10-1.22.0 1.7.14 1.6.2 @@ -446,6 +446,12 @@ ${project.version} + + org.apache.beam + beam-sdks-java-io-hadoop-input-format + ${project.version} + + org.apache.beam beam-runners-core-construction-java diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java index 763436642f70e..c59afc5961ca8 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java @@ -43,8 +43,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.MapState; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.SetState; @@ -139,12 +139,12 @@ public MapState bindMap( } @Override - public AccumulatorCombiningState + public CombiningState bindCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, final CombineFn combineFn) { - return new ApexAccumulatorCombiningState<>( + return new ApexCombiningState<>( namespace, address, accumCoder, @@ -161,12 +161,12 @@ public WatermarkHoldState bindWatermark( } @Override - public AccumulatorCombiningState + public CombiningState bindKeyedCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, KeyedCombineFn combineFn) { - return new ApexAccumulatorCombiningState<>( + return new ApexCombiningState<>( namespace, address, accumCoder, @@ -174,9 +174,9 @@ public WatermarkHoldState bindWatermark( } @Override - public AccumulatorCombiningState + public CombiningState bindKeyedCombiningValueWithContext( - StateTag> address, + StateTag> address, Coder accumCoder, KeyedCombineFnWithContext combineFn) { return bindKeyedCombiningValue(address, accumCoder, CombineFnUtil.bindContext(combineFn, c)); @@ -323,14 +323,14 @@ public OutputTimeFn getOutputTimeFn() { } - private final class ApexAccumulatorCombiningState + private final class ApexCombiningState extends AbstractState - implements AccumulatorCombiningState { + implements CombiningState { private final K key; private final KeyedCombineFn combineFn; - private ApexAccumulatorCombiningState(StateNamespace namespace, - StateTag> address, + private ApexCombiningState(StateNamespace namespace, + StateTag> address, Coder coder, K key, KeyedCombineFn combineFn) { super(namespace, address, coder); @@ -339,7 +339,7 @@ private ApexAccumulatorCombiningState(StateNamespace namespace, } @Override - public ApexAccumulatorCombiningState readLater() { + public ApexCombiningState readLater() { return this; } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java index 3e83a7fd89023..4f4ecfb3fc81c 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java @@ -35,9 +35,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; import org.apache.beam.sdk.util.state.CombiningState; +import org.apache.beam.sdk.util.state.GroupingState; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.util.state.WatermarkHoldState; @@ -58,7 +58,7 @@ public class ApexStateInternalsTest { private static final StateTag> STRING_VALUE_ADDR = StateTags.value("stringValue", StringUtf8Coder.of()); - private static final StateTag> + private static final StateTag> SUM_INTEGER_ADDR = StateTags.combiningValueFromInputInternal( "sumInteger", VarIntCoder.of(), Sum.ofIntegers()); private static final StateTag> STRING_BAG_ADDR = @@ -148,7 +148,7 @@ public void testMergeBagIntoNewNamespace() throws Exception { @Test public void testCombiningValue() throws Exception { - CombiningState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); + GroupingState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); // State instances are cached, but depend on the namespace. assertEquals(value, underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR)); @@ -168,7 +168,7 @@ public void testCombiningValue() throws Exception { @Test public void testCombiningIsEmpty() throws Exception { - CombiningState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); + GroupingState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); assertThat(value.isEmpty().read(), Matchers.is(true)); ReadableState readFuture = value.isEmpty(); @@ -181,9 +181,9 @@ public void testCombiningIsEmpty() throws Exception { @Test public void testMergeCombiningValueIntoSource() throws Exception { - AccumulatorCombiningState value1 = + CombiningState value1 = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); - AccumulatorCombiningState value2 = + CombiningState value2 = underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR); value1.add(5); @@ -202,11 +202,11 @@ public void testMergeCombiningValueIntoSource() throws Exception { @Test public void testMergeCombiningValueIntoNewNamespace() throws Exception { - AccumulatorCombiningState value1 = + CombiningState value1 = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); - AccumulatorCombiningState value2 = + CombiningState value2 = underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR); - AccumulatorCombiningState value3 = + CombiningState value3 = underTest.state(NAMESPACE_3, SUM_INTEGER_ADDR); value1.add(5); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java index 1d12194fc4b39..d053f62add9b4 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java @@ -38,6 +38,8 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.Materialization; +import org.apache.beam.sdk.transforms.Materializations; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Sum; @@ -349,6 +351,14 @@ public void createViewWithViewFnDifferentViewFn() { PCollectionViews.iterableView(input, input.getWindowingStrategy(), input.getCoder()); ViewFn>, Iterable> viewFn = new ViewFn>, Iterable>() { + @Override + public Materialization>> getMaterialization() { + @SuppressWarnings({"rawtypes", "unchecked"}) + Materialization>> materialization = + (Materialization) Materializations.iterable(); + return materialization; + } + @Override public Iterable apply(Iterable> contents) { return Collections.emptyList(); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java index b4b2b3860bf03..55b7fc2967bca 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java @@ -17,8 +17,6 @@ */ package org.apache.beam.runners.core; -import static com.google.common.base.Preconditions.checkNotNull; - import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -38,10 +36,11 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.MapState; import org.apache.beam.sdk.util.state.ReadableState; +import org.apache.beam.sdk.util.state.ReadableStates; import org.apache.beam.sdk.util.state.SetState; import org.apache.beam.sdk.util.state.State; import org.apache.beam.sdk.util.state.StateContext; @@ -148,12 +147,12 @@ public MapState bindMap( } @Override - public AccumulatorCombiningState + public CombiningState bindCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, final CombineFn combineFn) { - return new InMemoryCombiningValue(key, combineFn.asKeyedFn()); + return new InMemoryCombiningState(key, combineFn.asKeyedFn()); } @Override @@ -164,18 +163,18 @@ public WatermarkHoldState bindWatermark( } @Override - public AccumulatorCombiningState + public CombiningState bindKeyedCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, KeyedCombineFn combineFn) { - return new InMemoryCombiningValue(key, combineFn); + return new InMemoryCombiningState(key, combineFn); } @Override - public AccumulatorCombiningState + public CombiningState bindKeyedCombiningValueWithContext( - StateTag> address, + StateTag> address, Coder accumCoder, KeyedCombineFnWithContext combineFn) { return bindKeyedCombiningValue(address, accumCoder, CombineFnUtil.bindContext(combineFn, c)); @@ -307,17 +306,17 @@ public InMemoryWatermarkHold copy() { } /** - * An {@link InMemoryState} implementation of {@link AccumulatorCombiningState}. + * An {@link InMemoryState} implementation of {@link CombiningState}. */ - public static final class InMemoryCombiningValue - implements AccumulatorCombiningState, - InMemoryState> { + public static final class InMemoryCombiningState + implements CombiningState, + InMemoryState> { private final K key; private boolean isCleared = true; private final KeyedCombineFn combineFn; private AccumT accum; - public InMemoryCombiningValue( + public InMemoryCombiningState( K key, KeyedCombineFn combineFn) { this.key = key; this.combineFn = combineFn; @@ -325,7 +324,7 @@ public InMemoryCombiningValue( } @Override - public InMemoryCombiningValue readLater() { + public InMemoryCombiningState readLater() { return this; } @@ -384,9 +383,9 @@ public boolean isCleared() { } @Override - public InMemoryCombiningValue copy() { - InMemoryCombiningValue that = - new InMemoryCombiningValue<>(key, combineFn); + public InMemoryCombiningState copy() { + InMemoryCombiningState that = + new InMemoryCombiningState<>(key, combineFn); if (!this.isCleared) { that.isCleared = this.isCleared; that.addAccum(accum); @@ -468,13 +467,15 @@ public void clear() { } @Override - public boolean contains(T t) { - return contents.contains(t); + public ReadableState contains(T t) { + return ReadableStates.immediate(contents.contains(t)); } @Override - public boolean addIfAbsent(T t) { - return contents.add(t); + public ReadableState addIfAbsent(T t) { + boolean alreadyContained = contents.contains(t); + contents.add(t); + return ReadableStates.immediate(!alreadyContained); } @Override @@ -482,33 +483,6 @@ public void remove(T t) { contents.remove(t); } - @Override - public SetState readLater(Iterable elements) { - return this; - } - - @Override - public boolean containsAny(Iterable elements) { - elements = checkNotNull(elements); - for (T t : elements) { - if (contents.contains(t)) { - return true; - } - } - return false; - } - - @Override - public boolean containsAll(Iterable elements) { - elements = checkNotNull(elements); - for (T t : elements) { - if (!contents.contains(t)) { - return false; - } - } - return true; - } - @Override public InMemorySet readLater() { return this; @@ -565,8 +539,8 @@ public void clear() { } @Override - public V get(K key) { - return contents.get(key); + public ReadableState get(K key) { + return ReadableStates.immediate(contents.get(key)); } @Override @@ -575,13 +549,13 @@ public void put(K key, V value) { } @Override - public V putIfAbsent(K key, V value) { + public ReadableState putIfAbsent(K key, V value) { V v = contents.get(key); if (v == null) { v = contents.put(key, value); } - return v; + return ReadableStates.immediate(v); } @Override @@ -590,42 +564,18 @@ public void remove(K key) { } @Override - public Iterable get(Iterable keys) { - List values = new ArrayList<>(); - for (K k : keys) { - values.add(contents.get(k)); - } - return values; + public ReadableState> keys() { + return ReadableStates.immediate((Iterable) contents.keySet()); } @Override - public MapState getLater(K k) { - return this; - } - - @Override - public MapState getLater(Iterable keys) { - return this; - } - - @Override - public Iterable keys() { - return contents.keySet(); - } - - @Override - public Iterable values() { - return contents.values(); - } - - @Override - public MapState iterateLater() { - return this; + public ReadableState> values() { + return ReadableStates.immediate((Iterable) contents.values()); } @Override - public Iterable> iterate() { - return contents.entrySet(); + public ReadableState>> entries() { + return ReadableStates.immediate((Iterable>) contents.entrySet()); } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java index aa033ce56203d..3e875c2c07b81 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java @@ -22,7 +22,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.ReadableState; /** @@ -113,7 +113,7 @@ public void onMerge(MergingStateAccessor context) { private static class GeneralNonEmptyPanes extends NonEmptyPanes { - private static final StateTag> + private static final StateTag> PANE_ADDITIONS_TAG = StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal( "count", VarLongCoder.of(), Sum.ofLongs())); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java index 5aa76052f4496..357094c2e0122 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java @@ -226,8 +226,7 @@ public InputT element() { public T sideInput(PCollectionView view) { return sideInputReader.get( view, - view.getWindowingStrategyInternal() - .getWindowFn() + view.getWindowMappingFn() .getSideInputWindow(Iterables.getOnlyElement(element.getWindows()))); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java index 2962832a5096c..4ad20b59201eb 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java @@ -98,7 +98,7 @@ private boolean isReady(BoundedWindow mainInputWindow) { } for (PCollectionView view : views) { BoundedWindow sideInputWindow = - view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow); + view.getWindowMappingFn().getSideInputWindow(mainInputWindow); if (!sideInputReader.isReady(view, sideInputWindow)) { return false; } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java index 66a6ef8ecf3a0..8493474c8d285 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java @@ -514,8 +514,7 @@ public PipelineOptions getPipelineOptions() { public T sideInput(PCollectionView view) { return sideInputReader.get( view, - view.getWindowingStrategyInternal() - .getWindowFn() + view.getWindowMappingFn() .getSideInputWindow(mainInputWindow)); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java index 24f326dd4bb0a..26e920abd1bc5 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java @@ -31,7 +31,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.PCollectionView; @@ -71,10 +71,10 @@ public class SideInputHandler implements ReadyCheckingSideInputReader { PCollectionView, StateTag< Object, - AccumulatorCombiningState< - BoundedWindow, - Set, - Set>>> availableWindowsTags; + CombiningState< + BoundedWindow, + Set, + Set>>> availableWindowsTags; /** * State tag for the actual contents of each side input per window. @@ -106,10 +106,10 @@ public SideInputHandler( StateTag< Object, - AccumulatorCombiningState< - BoundedWindow, - Set, - Set>> availableTag = StateTags.combiningValue( + CombiningState< + BoundedWindow, + Set, + Set>> availableTag = StateTags.combiningValue( "side-input-available-windows-" + sideInput.getTagInternal().getId(), SetCoder.of(windowCoder), new WindowSetCombineFn()); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index dfa96450f1607..77286b257a2b5 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -533,7 +533,7 @@ public T sideInput(PCollectionView view) { } } return context.sideInput( - view, view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(window)); + view, view.getWindowMappingFn().getSideInputWindow(window)); } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java index c21ed7775574d..c88f1c9e42ab2 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java @@ -389,7 +389,7 @@ public T sideInput(PCollectionView view) { } } return context.sideInput( - view, view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(window)); + view, view.getWindowMappingFn().getSideInputWindow(window)); } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java index e98d0982a90aa..34108507d0512 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java @@ -24,9 +24,9 @@ import java.util.List; import java.util.Map; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; import org.apache.beam.sdk.util.state.CombiningState; +import org.apache.beam.sdk.util.state.GroupingState; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.SetState; import org.apache.beam.sdk.util.state.State; @@ -159,7 +159,7 @@ public static void mergeSets( * Prefetch all combining value state for {@code address} across all merging windows in {@code * context}. */ - public static , W extends BoundedWindow> void + public static , W extends BoundedWindow> void prefetchCombiningValues(MergingStateAccessor context, StateTag address) { for (StateT state : context.accessInEachMergingWindow(address).values()) { @@ -172,7 +172,7 @@ public static void mergeSets( */ public static void mergeCombiningValues( MergingStateAccessor context, - StateTag> address) { + StateTag> address) { mergeCombiningValues( context.accessInEachMergingWindow(address).values(), context.access(address)); } @@ -182,8 +182,8 @@ public static void mergeCo * {@code result}. */ public static void mergeCombiningValues( - Collection> sources, - AccumulatorCombiningState result) { + Collection> sources, + CombiningState result) { if (sources.isEmpty()) { // Nothing to merge. return; @@ -194,18 +194,18 @@ public static void mergeCombi } // Prefetch. List> futures = new ArrayList<>(sources.size()); - for (AccumulatorCombiningState source : sources) { + for (CombiningState source : sources) { prefetchRead(source); } // Read. List accumulators = new ArrayList<>(futures.size()); - for (AccumulatorCombiningState source : sources) { + for (CombiningState source : sources) { accumulators.add(source.getAccum()); } // Merge (possibly update and return one of the existing accumulators). AccumT merged = result.mergeAccumulators(accumulators); // Clear sources. - for (AccumulatorCombiningState source : sources) { + for (CombiningState source : sources) { source.clear(); } // Update result. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTag.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTag.java index 802aedea832b8..12c59adb8e449 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTag.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTag.java @@ -28,8 +28,8 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.MapState; import org.apache.beam.sdk.util.state.SetState; import org.apache.beam.sdk.util.state.State; @@ -94,20 +94,20 @@ MapState bindMap( StateTag> spec, Coder mapKeyCoder, Coder mapValueCoder); - AccumulatorCombiningState bindCombiningValue( - StateTag> spec, + CombiningState bindCombiningValue( + StateTag> spec, Coder accumCoder, CombineFn combineFn); - AccumulatorCombiningState bindKeyedCombiningValue( - StateTag> spec, + CombiningState bindKeyedCombiningValue( + StateTag> spec, Coder accumCoder, KeyedCombineFn combineFn); - AccumulatorCombiningState bindKeyedCombiningValueWithContext( - StateTag> spec, + CombiningState bindKeyedCombiningValueWithContext( + StateTag> spec, Coder accumCoder, KeyedCombineFnWithContext combineFn); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java index 1c70dff5f386e..77ae8f533d173 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java @@ -30,8 +30,8 @@ import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.MapState; import org.apache.beam.sdk.util.state.SetState; import org.apache.beam.sdk.util.state.State; @@ -84,9 +84,9 @@ public MapState bindMap( @Override public - AccumulatorCombiningState bindCombiningValue( + CombiningState bindCombining( String id, - StateSpec> spec, + StateSpec> spec, Coder accumCoder, CombineFn combineFn) { return binder.bindCombiningValue(tagForSpec(id, spec), accumCoder, combineFn); @@ -94,9 +94,9 @@ AccumulatorCombiningState bindCombiningValue( @Override public - AccumulatorCombiningState bindKeyedCombiningValue( + CombiningState bindKeyedCombining( String id, - StateSpec> spec, + StateSpec> spec, Coder accumCoder, KeyedCombineFn combineFn) { return binder.bindKeyedCombiningValue(tagForSpec(id, spec), accumCoder, combineFn); @@ -104,9 +104,9 @@ AccumulatorCombiningState bindKeyedCombiningValue( @Override public - AccumulatorCombiningState bindKeyedCombiningValueWithContext( + CombiningState bindKeyedCombiningWithContext( String id, - StateSpec> spec, + StateSpec> spec, Coder accumCoder, KeyedCombineFnWithContext combineFn) { return binder.bindKeyedCombiningValueWithContext( @@ -158,11 +158,11 @@ public static StateTag> value(String id, Coder valu * multiple {@code InputT}s into a single {@code OutputT}. */ public static - StateTag> + StateTag> combiningValue( String id, Coder accumCoder, CombineFn combineFn) { return new SimpleStateTag<>( - new StructuredId(id), StateSpecs.combiningValue(accumCoder, combineFn)); + new StructuredId(id), StateSpecs.combining(accumCoder, combineFn)); } /** @@ -170,11 +170,11 @@ public static StateTag> value(String id, Coder valu * multiple {@code InputT}s into a single {@code OutputT}. */ public static StateTag> + OutputT> StateTag> keyedCombiningValue(String id, Coder accumCoder, KeyedCombineFn combineFn) { return new SimpleStateTag<>( - new StructuredId(id), StateSpecs.keyedCombiningValue(accumCoder, combineFn)); + new StructuredId(id), StateSpecs.keyedCombining(accumCoder, combineFn)); } /** @@ -182,13 +182,13 @@ public static StateTag> value(String id, Coder valu * merge multiple {@code InputT}s into a single {@code OutputT}. */ public static - StateTag> + StateTag> keyedCombiningValueWithContext( String id, Coder accumCoder, KeyedCombineFnWithContext combineFn) { return new SimpleStateTag<>( - new StructuredId(id), StateSpecs.keyedCombiningValueWithContext(accumCoder, combineFn)); + new StructuredId(id), StateSpecs.keyedCombiningWithContext(accumCoder, combineFn)); } /** @@ -199,11 +199,11 @@ public static StateTag> value(String id, Coder valu * should only be used to initialize static values. */ public static - StateTag> + StateTag> combiningValueFromInputInternal( String id, Coder inputCoder, CombineFn combineFn) { return new SimpleStateTag<>( - new StructuredId(id), StateSpecs.combiningValueFromInputInternal(inputCoder, combineFn)); + new StructuredId(id), StateSpecs.combiningFromInputInternal(inputCoder, combineFn)); } /** @@ -255,7 +255,7 @@ public static StateTag makeSystemTagInterna public static StateTag> convertToBagTagInternal( - StateTag> combiningTag) { + StateTag> combiningTag) { return new SimpleStateTag<>( new StructuredId(combiningTag.getId()), StateSpecs.convertToBagSpecInternal(combiningTag.getSpec())); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java index bb7e4a9922154..f618d889d7b8d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java @@ -25,9 +25,9 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; import org.apache.beam.sdk.util.state.CombiningState; +import org.apache.beam.sdk.util.state.GroupingState; import org.apache.beam.sdk.util.state.ReadableState; /** @@ -71,7 +71,7 @@ public void onMerge(OnMergeContext c) throws Exception { AccumT, OutputT, W> combining( final Coder keyCoder, final AppliedCombineFn combineFn) { - final StateTag> bufferTag; + final StateTag> bufferTag; if (combineFn.getFn() instanceof KeyedCombineFnWithContext) { bufferTag = StateTags.makeSystemTagInternal( StateTags.keyedCombiningValueWithContext( @@ -97,10 +97,10 @@ public void onMerge(OnMergeContext c) throws Exception { }; } - private StateTag> bufferTag; + private StateTag> bufferTag; public SystemReduceFn( - StateTag> bufferTag) { + StateTag> bufferTag) { this.bufferTag = bufferTag; } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java index 4444c227a1fa3..b416788fd06f8 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java @@ -30,12 +30,12 @@ import org.apache.beam.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.InstantCoder; -import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Combine.Holder; import org.apache.beam.sdk.transforms.Min; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.CombiningState; +import org.apache.beam.sdk.util.state.GroupingState; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.PeriodFormat; @@ -55,8 +55,8 @@ public abstract class AfterDelayFromFirstElementStateMachine extends OnceTrigger protected static final List> IDENTITY = ImmutableList.>of(); - protected static final StateTag, Instant>> DELAYED_UNTIL_TAG = + protected static final StateTag, Instant>> DELAYED_UNTIL_TAG = StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal( "delayed", InstantCoder.of(), Min.naturalOrder())); @@ -169,7 +169,7 @@ public void prefetchOnElement(StateAccessor state) { @Override public void onElement(OnElementContext c) throws Exception { - CombiningState delayUntilState = c.state().access(DELAYED_UNTIL_TAG); + GroupingState delayUntilState = c.state().access(DELAYED_UNTIL_TAG); Instant oldDelayUntil = delayUntilState.read(); // Since processing time can only advance, resulting in target wake-up times we would diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachine.java index 1dd5b6534fcbc..11323cc69eb27 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachine.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachine.java @@ -27,7 +27,7 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; +import org.apache.beam.sdk.util.state.CombiningState; /** * {@link TriggerStateMachine}s that fire based on properties of the elements in the current pane. @@ -35,7 +35,7 @@ @Experimental(Experimental.Kind.TRIGGER) public class AfterPaneStateMachine extends OnceTriggerStateMachine { -private static final StateTag> +private static final StateTag> ELEMENTS_IN_PANE_TAG = StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal( "count", VarLongCoder.of(), Sum.ofLongs())); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryStateInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryStateInternalsTest.java index 1da946f7880d7..34ddae6fc47a3 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryStateInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryStateInternalsTest.java @@ -17,7 +17,9 @@ */ package org.apache.beam.runners.core; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasItems; import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -26,7 +28,6 @@ import static org.junit.Assert.assertTrue; import java.util.Arrays; -import java.util.Collections; import java.util.Map; import java.util.Objects; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -35,9 +36,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; import org.apache.beam.sdk.util.state.CombiningState; +import org.apache.beam.sdk.util.state.GroupingState; import org.apache.beam.sdk.util.state.MapState; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.SetState; @@ -61,7 +62,7 @@ public class InMemoryStateInternalsTest { private static final StateTag> STRING_VALUE_ADDR = StateTags.value("stringValue", StringUtf8Coder.of()); - private static final StateTag> + private static final StateTag> SUM_INTEGER_ADDR = StateTags.combiningValueFromInputInternal( "sumInteger", VarIntCoder.of(), Sum.ofIntegers()); private static final StateTag> STRING_BAG_ADDR = @@ -112,10 +113,10 @@ public void testBag() throws Exception { assertThat(value.read(), Matchers.emptyIterable()); value.add("hello"); - assertThat(value.read(), Matchers.containsInAnyOrder("hello")); + assertThat(value.read(), containsInAnyOrder("hello")); value.add("world"); - assertThat(value.read(), Matchers.containsInAnyOrder("hello", "world")); + assertThat(value.read(), containsInAnyOrder("hello", "world")); value.clear(); assertThat(value.read(), Matchers.emptyIterable()); @@ -147,7 +148,7 @@ public void testMergeBagIntoSource() throws Exception { StateMerging.mergeBags(Arrays.asList(bag1, bag2), bag1); // Reading the merged bag gets both the contents - assertThat(bag1.read(), Matchers.containsInAnyOrder("Hello", "World", "!")); + assertThat(bag1.read(), containsInAnyOrder("Hello", "World", "!")); assertThat(bag2.read(), Matchers.emptyIterable()); } @@ -164,7 +165,7 @@ public void testMergeBagIntoNewNamespace() throws Exception { StateMerging.mergeBags(Arrays.asList(bag1, bag2, bag3), bag3); // Reading the merged bag gets both the contents - assertThat(bag3.read(), Matchers.containsInAnyOrder("Hello", "World", "!")); + assertThat(bag3.read(), containsInAnyOrder("Hello", "World", "!")); assertThat(bag1.read(), Matchers.emptyIterable()); assertThat(bag2.read(), Matchers.emptyIterable()); } @@ -179,41 +180,32 @@ public void testSet() throws Exception { // empty assertThat(value.read(), Matchers.emptyIterable()); - assertFalse(value.contains("A")); - assertFalse(value.containsAny(Collections.singletonList("A"))); + assertFalse(value.contains("A").read()); // add value.add("A"); value.add("B"); value.add("A"); - assertFalse(value.addIfAbsent("B")); - assertThat(value.read(), Matchers.containsInAnyOrder("A", "B")); + assertFalse(value.addIfAbsent("B").read()); + assertThat(value.read(), containsInAnyOrder("A", "B")); // remove value.remove("A"); - assertThat(value.read(), Matchers.containsInAnyOrder("B")); + assertThat(value.read(), containsInAnyOrder("B")); value.remove("C"); - assertThat(value.read(), Matchers.containsInAnyOrder("B")); + assertThat(value.read(), containsInAnyOrder("B")); // contains - assertFalse(value.contains("A")); - assertTrue(value.contains("B")); + assertFalse(value.contains("A").read()); + assertTrue(value.contains("B").read()); value.add("C"); value.add("D"); - // containsAny - assertTrue(value.containsAny(Arrays.asList("A", "C"))); - assertFalse(value.containsAny(Arrays.asList("A", "E"))); - - // containsAll - assertTrue(value.containsAll(Arrays.asList("B", "C"))); - assertFalse(value.containsAll(Arrays.asList("A", "B"))); - // readLater - assertThat(value.readLater().read(), Matchers.containsInAnyOrder("B", "C", "D")); - SetState later = value.readLater(Arrays.asList("A", "C", "D")); - assertTrue(later.containsAll(Arrays.asList("C", "D"))); - assertFalse(later.contains("A")); + assertThat(value.readLater().read(), containsInAnyOrder("B", "C", "D")); + SetState later = value.readLater(); + assertThat(later.read(), hasItems("C", "D")); + assertFalse(later.contains("A").read()); // clear value.clear(); @@ -248,7 +240,7 @@ public void testMergeSetIntoSource() throws Exception { StateMerging.mergeSets(Arrays.asList(set1, set2), set1); // Reading the merged set gets both the contents - assertThat(set1.read(), Matchers.containsInAnyOrder("Hello", "World", "!")); + assertThat(set1.read(), containsInAnyOrder("Hello", "World", "!")); assertThat(set2.read(), Matchers.emptyIterable()); } @@ -266,7 +258,7 @@ public void testMergeSetIntoNewNamespace() throws Exception { StateMerging.mergeSets(Arrays.asList(set1, set2, set3), set3); // Reading the merged set gets both the contents - assertThat(set3.read(), Matchers.containsInAnyOrder("Hello", "World", "!")); + assertThat(set3.read(), containsInAnyOrder("Hello", "World", "!")); assertThat(set1.read(), Matchers.emptyIterable()); assertThat(set2.read(), Matchers.emptyIterable()); } @@ -330,55 +322,52 @@ public void testMap() throws Exception { assertThat(value, not(equalTo(underTest.state(NAMESPACE_2, STRING_MAP_ADDR)))); // put - assertThat(value.iterate(), Matchers.emptyIterable()); + assertThat(value.entries().read(), Matchers.emptyIterable()); value.put("A", 1); value.put("B", 2); value.put("A", 11); - assertThat(value.putIfAbsent("B", 22), equalTo(2)); - assertThat(value.iterate(), Matchers.containsInAnyOrder(MapEntry.of("A", 11), + assertThat(value.putIfAbsent("B", 22).read(), equalTo(2)); + assertThat(value.entries().read(), containsInAnyOrder(MapEntry.of("A", 11), MapEntry.of("B", 2))); // remove value.remove("A"); - assertThat(value.iterate(), Matchers.containsInAnyOrder(MapEntry.of("B", 2))); + assertThat(value.entries().read(), containsInAnyOrder(MapEntry.of("B", 2))); value.remove("C"); - assertThat(value.iterate(), Matchers.containsInAnyOrder(MapEntry.of("B", 2))); + assertThat(value.entries().read(), containsInAnyOrder(MapEntry.of("B", 2))); // get - assertNull(value.get("A")); - assertThat(value.get("B"), equalTo(2)); + assertNull(value.get("A").read()); + assertThat(value.get("B").read(), equalTo(2)); value.put("C", 3); value.put("D", 4); - assertThat(value.get("C"), equalTo(3)); - assertThat(value.get(Collections.singletonList("D")), Matchers.containsInAnyOrder(4)); - assertThat(value.get(Arrays.asList("B", "C")), Matchers.containsInAnyOrder(2, 3)); + assertThat(value.get("C").read(), equalTo(3)); // iterate value.put("E", 5); value.remove("C"); - assertThat(value.keys(), Matchers.containsInAnyOrder("B", "D", "E")); - assertThat(value.values(), Matchers.containsInAnyOrder(2, 4, 5)); - assertThat(value.iterate(), Matchers.containsInAnyOrder( - MapEntry.of("B", 2), MapEntry.of("D", 4), MapEntry.of("E", 5))); + assertThat(value.keys().read(), containsInAnyOrder("B", "D", "E")); + assertThat(value.values().read(), containsInAnyOrder(2, 4, 5)); + assertThat( + value.entries().read(), + containsInAnyOrder(MapEntry.of("B", 2), MapEntry.of("D", 4), MapEntry.of("E", 5))); // readLater - assertThat(value.getLater("B").get("B"), equalTo(2)); - assertNull(value.getLater("A").get("A")); - MapState later = value.getLater(Arrays.asList("C", "D")); - assertNull(later.get("C")); - assertThat(later.get("D"), equalTo(4)); - assertThat(value.iterateLater().iterate(), Matchers.containsInAnyOrder( - MapEntry.of("B", 2), MapEntry.of("D", 4), MapEntry.of("E", 5))); + assertThat(value.get("B").readLater().read(), equalTo(2)); + assertNull(value.get("A").readLater().read()); + assertThat( + value.entries().readLater().read(), + containsInAnyOrder(MapEntry.of("B", 2), MapEntry.of("D", 4), MapEntry.of("E", 5))); // clear value.clear(); - assertThat(value.iterate(), Matchers.emptyIterable()); + assertThat(value.entries().read(), Matchers.emptyIterable()); assertThat(underTest.state(NAMESPACE_1, STRING_MAP_ADDR), Matchers.sameInstance(value)); } @Test public void testCombiningValue() throws Exception { - CombiningState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); + GroupingState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); // State instances are cached, but depend on the namespace. assertEquals(value, underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR)); @@ -398,7 +387,7 @@ public void testCombiningValue() throws Exception { @Test public void testCombiningIsEmpty() throws Exception { - CombiningState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); + GroupingState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); assertThat(value.isEmpty().read(), Matchers.is(true)); ReadableState readFuture = value.isEmpty(); @@ -411,9 +400,9 @@ public void testCombiningIsEmpty() throws Exception { @Test public void testMergeCombiningValueIntoSource() throws Exception { - AccumulatorCombiningState value1 = + CombiningState value1 = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); - AccumulatorCombiningState value2 = + CombiningState value2 = underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR); value1.add(5); @@ -432,11 +421,11 @@ public void testMergeCombiningValueIntoSource() throws Exception { @Test public void testMergeCombiningValueIntoNewNamespace() throws Exception { - AccumulatorCombiningState value1 = + CombiningState value1 = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); - AccumulatorCombiningState value2 = + CombiningState value2 = underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR); - AccumulatorCombiningState value3 = + CombiningState value3 = underTest.state(NAMESPACE_3, SUM_INTEGER_ADDR); value1.add(5); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java index 1bd717fc10486..0d4d992a9386f 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java @@ -65,6 +65,7 @@ import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; @@ -360,8 +361,9 @@ public void testOnElementCombiningWithContext() throws Exception { WindowingStrategy.of(FixedWindows.of(Duration.millis(2))) .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES); - WindowingStrategy sideInputWindowingStrategy = - WindowingStrategy.of(FixedWindows.of(Duration.millis(4))); + WindowMappingFn sideInputWindowMappingFn = + FixedWindows.of(Duration.millis(4)).getDefaultWindowMappingFn(); + when(mockView.getWindowMappingFn()).thenReturn((WindowMappingFn) sideInputWindowMappingFn); TestOptions options = PipelineOptionsFactory.as(TestOptions.class); options.setValue(expectedValue); @@ -384,10 +386,6 @@ public Integer answer(InvocationOnMock invocation) throws Throwable { } }); - @SuppressWarnings({"rawtypes", "unchecked", "unused"}) - Object suppressWarningsVar = when(mockView.getWindowingStrategyInternal()) - .thenReturn((WindowingStrategy) sideInputWindowingStrategy); - SumAndVerifyContextFn combineFn = new SumAndVerifyContextFn(mockView, expectedValue); ReduceFnTester tester = ReduceFnTester.combining( mainInputWindowingStrategy, mockTriggerStateMachine, combineFn.asKeyedFn(), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java index ff5c23c2d05a9..0665812e685f1 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java @@ -26,7 +26,7 @@ import java.util.Map; import javax.annotation.Nullable; import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryBag; -import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryCombiningValue; +import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryCombiningState; import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryMap; import org.apache.beam.runners.core.InMemoryStateInternals.InMemorySet; import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryState; @@ -45,8 +45,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.MapState; import org.apache.beam.sdk.util.state.SetState; import org.apache.beam.sdk.util.state.State; @@ -306,19 +306,18 @@ public ValueState bindValue( } @Override - public AccumulatorCombiningState + public CombiningState bindCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, CombineFn combineFn) { if (containedInUnderlying(namespace, address)) { @SuppressWarnings("unchecked") - InMemoryState> - existingState = ( - InMemoryState>) underlying.get().get(namespace, address, c); + InMemoryState> existingState = + (InMemoryState>) + underlying.get().get(namespace, address, c); return existingState.copy(); } else { - return new InMemoryCombiningValue<>( + return new InMemoryCombiningState<>( key, combineFn.asKeyedFn()); } } @@ -367,27 +366,26 @@ public MapState bindMap( } @Override - public AccumulatorCombiningState + public CombiningState bindKeyedCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, KeyedCombineFn combineFn) { if (containedInUnderlying(namespace, address)) { @SuppressWarnings("unchecked") - InMemoryState> - existingState = ( - InMemoryState>) underlying.get().get(namespace, address, c); + InMemoryState> existingState = + (InMemoryState>) + underlying.get().get(namespace, address, c); return existingState.copy(); } else { - return new InMemoryCombiningValue<>(key, combineFn); + return new InMemoryCombiningState<>(key, combineFn); } } @Override - public AccumulatorCombiningState + public CombiningState bindKeyedCombiningValueWithContext( - StateTag> address, + StateTag> address, Coder accumCoder, KeyedCombineFnWithContext combineFn) { return bindKeyedCombiningValue( @@ -449,9 +447,9 @@ public ValueState bindValue( } @Override - public AccumulatorCombiningState + public CombiningState bindCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, CombineFn combineFn) { return underlying.get(namespace, address, c); } @@ -476,18 +474,18 @@ public MapState bindMap( } @Override - public AccumulatorCombiningState + public CombiningState bindKeyedCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, KeyedCombineFn combineFn) { return underlying.get(namespace, address, c); } @Override - public AccumulatorCombiningState + public CombiningState bindKeyedCombiningValueWithContext( - StateTag> address, + StateTag> address, Coder accumCoder, KeyedCombineFnWithContext combineFn) { return bindKeyedCombiningValue( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java index c7409bbe05965..68c6613848796 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java @@ -45,9 +45,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; import org.apache.beam.sdk.util.state.CombiningState; +import org.apache.beam.sdk.util.state.GroupingState; import org.apache.beam.sdk.util.state.MapState; import org.apache.beam.sdk.util.state.SetState; import org.apache.beam.sdk.util.state.ValueState; @@ -201,24 +201,24 @@ public void testMapStateWithUnderlying() { StateTag> valueTag = StateTags.map("foo", StringUtf8Coder.of(), VarIntCoder.of()); MapState underlyingValue = underlying.state(namespace, valueTag); - assertThat(underlyingValue.iterate(), emptyIterable()); + assertThat(underlyingValue.entries().read(), emptyIterable()); underlyingValue.put("hello", 1); - assertThat(underlyingValue.get("hello"), equalTo(1)); + assertThat(underlyingValue.get("hello").read(), equalTo(1)); CopyOnAccessInMemoryStateInternals internals = CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying); MapState copyOnAccessState = internals.state(namespace, valueTag); - assertThat(copyOnAccessState.get("hello"), equalTo(1)); + assertThat(copyOnAccessState.get("hello").read(), equalTo(1)); copyOnAccessState.put("world", 4); - assertThat(copyOnAccessState.get("hello"), equalTo(1)); - assertThat(copyOnAccessState.get("world"), equalTo(4)); - assertThat(underlyingValue.get("hello"), equalTo(1)); - assertNull(underlyingValue.get("world")); + assertThat(copyOnAccessState.get("hello").read(), equalTo(1)); + assertThat(copyOnAccessState.get("world").read(), equalTo(4)); + assertThat(underlyingValue.get("hello").read(), equalTo(1)); + assertNull(underlyingValue.get("world").read()); MapState reReadUnderlyingValue = underlying.state(namespace, valueTag); - assertThat(underlyingValue.iterate(), equalTo(reReadUnderlyingValue.iterate())); + assertThat(underlyingValue.entries().read(), equalTo(reReadUnderlyingValue.entries().read())); } @Test @@ -229,10 +229,10 @@ public void testAccumulatorCombiningStateWithUnderlying() throws CannotProvideCo StateNamespace namespace = new StateNamespaceForTest("foo"); CoderRegistry reg = pipeline.getCoderRegistry(); - StateTag> stateTag = + StateTag> stateTag = StateTags.combiningValue("summer", sumLongFn.getAccumulatorCoder(reg, reg.getDefaultCoder(Long.class)), sumLongFn); - CombiningState underlyingValue = underlying.state(namespace, stateTag); + GroupingState underlyingValue = underlying.state(namespace, stateTag); assertThat(underlyingValue.read(), equalTo(0L)); underlyingValue.add(1L); @@ -240,14 +240,14 @@ public void testAccumulatorCombiningStateWithUnderlying() throws CannotProvideCo CopyOnAccessInMemoryStateInternals internals = CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying); - CombiningState copyOnAccessState = internals.state(namespace, stateTag); + GroupingState copyOnAccessState = internals.state(namespace, stateTag); assertThat(copyOnAccessState.read(), equalTo(1L)); copyOnAccessState.add(4L); assertThat(copyOnAccessState.read(), equalTo(5L)); assertThat(underlyingValue.read(), equalTo(1L)); - CombiningState reReadUnderlyingValue = underlying.state(namespace, stateTag); + GroupingState reReadUnderlyingValue = underlying.state(namespace, stateTag); assertThat(underlyingValue.read(), equalTo(reReadUnderlyingValue.read())); } @@ -259,13 +259,13 @@ public void testKeyedAccumulatorCombiningStateWithUnderlying() throws Exception StateNamespace namespace = new StateNamespaceForTest("foo"); CoderRegistry reg = pipeline.getCoderRegistry(); - StateTag> stateTag = + StateTag> stateTag = StateTags.keyedCombiningValue( "summer", sumLongFn.getAccumulatorCoder( reg, StringUtf8Coder.of(), reg.getDefaultCoder(Long.class)), sumLongFn); - CombiningState underlyingValue = underlying.state(namespace, stateTag); + GroupingState underlyingValue = underlying.state(namespace, stateTag); assertThat(underlyingValue.read(), equalTo(0L)); underlyingValue.add(1L); @@ -273,14 +273,14 @@ public void testKeyedAccumulatorCombiningStateWithUnderlying() throws Exception CopyOnAccessInMemoryStateInternals internals = CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying); - CombiningState copyOnAccessState = internals.state(namespace, stateTag); + GroupingState copyOnAccessState = internals.state(namespace, stateTag); assertThat(copyOnAccessState.read(), equalTo(1L)); copyOnAccessState.add(4L); assertThat(copyOnAccessState.read(), equalTo(5L)); assertThat(underlyingValue.read(), equalTo(1L)); - CombiningState reReadUnderlyingValue = underlying.state(namespace, stateTag); + GroupingState reReadUnderlyingValue = underlying.state(namespace, stateTag); assertThat(underlyingValue.read(), equalTo(reReadUnderlyingValue.read())); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java index bcc3660ea9b90..320344608d22f 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java @@ -37,8 +37,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.util.CombineContextFactory; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.MapState; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.SetState; @@ -133,23 +133,23 @@ public MapState bindMap( @Override public - AccumulatorCombiningState + CombiningState bindCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, Combine.CombineFn combineFn) { - return new FlinkAccumulatorCombiningState<>( + return new FlinkCombiningState<>( stateBackend, address, combineFn, namespace, accumCoder); } @Override public - AccumulatorCombiningState bindKeyedCombiningValue( - StateTag> address, + CombiningState bindKeyedCombiningValue( + StateTag> address, Coder accumCoder, final Combine.KeyedCombineFn combineFn) { - return new FlinkKeyedAccumulatorCombiningState<>( + return new FlinkKeyedCombiningState<>( stateBackend, address, combineFn, @@ -160,12 +160,12 @@ AccumulatorCombiningState bindKeyedCombiningValue( @Override public - AccumulatorCombiningState bindKeyedCombiningValueWithContext( - StateTag> address, + CombiningState bindKeyedCombiningValueWithContext( + StateTag> address, Coder accumCoder, CombineWithContext.KeyedCombineFnWithContext< ? super K, InputT, AccumT, OutputT> combineFn) { - return new FlinkAccumulatorCombiningStateWithContext<>( + return new FlinkCombiningStateWithContext<>( stateBackend, address, combineFn, @@ -464,17 +464,17 @@ public int hashCode() { } } - private class FlinkAccumulatorCombiningState + private class FlinkCombiningState extends AbstractBroadcastState - implements AccumulatorCombiningState { + implements CombiningState { private final StateNamespace namespace; - private final StateTag> address; + private final StateTag> address; private final Combine.CombineFn combineFn; - FlinkAccumulatorCombiningState( + FlinkCombiningState( DefaultOperatorStateBackend flinkStateBackend, - StateTag> address, + StateTag> address, Combine.CombineFn combineFn, StateNamespace namespace, Coder accumCoder) { @@ -486,7 +486,7 @@ private class FlinkAccumulatorCombiningState } @Override - public AccumulatorCombiningState readLater() { + public CombiningState readLater() { return this; } @@ -566,8 +566,8 @@ public boolean equals(Object o) { return false; } - FlinkAccumulatorCombiningState that = - (FlinkAccumulatorCombiningState) o; + FlinkCombiningState that = + (FlinkCombiningState) o; return namespace.equals(that.namespace) && address.equals(that.address); @@ -581,18 +581,18 @@ public int hashCode() { } } - private class FlinkKeyedAccumulatorCombiningState + private class FlinkKeyedCombiningState extends AbstractBroadcastState - implements AccumulatorCombiningState { + implements CombiningState { private final StateNamespace namespace; - private final StateTag> address; + private final StateTag> address; private final Combine.KeyedCombineFn combineFn; private final FlinkBroadcastStateInternals flinkStateInternals; - FlinkKeyedAccumulatorCombiningState( + FlinkKeyedCombiningState( DefaultOperatorStateBackend flinkStateBackend, - StateTag> address, + StateTag> address, Combine.KeyedCombineFn combineFn, StateNamespace namespace, Coder accumCoder, @@ -607,7 +607,7 @@ private class FlinkKeyedAccumulatorCombiningState } @Override - public AccumulatorCombiningState readLater() { + public CombiningState readLater() { return this; } @@ -706,8 +706,8 @@ public boolean equals(Object o) { return false; } - FlinkKeyedAccumulatorCombiningState that = - (FlinkKeyedAccumulatorCombiningState) o; + FlinkKeyedCombiningState that = + (FlinkKeyedCombiningState) o; return namespace.equals(that.namespace) && address.equals(that.address); @@ -721,20 +721,20 @@ public int hashCode() { } } - private class FlinkAccumulatorCombiningStateWithContext + private class FlinkCombiningStateWithContext extends AbstractBroadcastState - implements AccumulatorCombiningState { + implements CombiningState { private final StateNamespace namespace; - private final StateTag> address; + private final StateTag> address; private final CombineWithContext.KeyedCombineFnWithContext< ? super K, InputT, AccumT, OutputT> combineFn; private final FlinkBroadcastStateInternals flinkStateInternals; private final CombineWithContext.Context context; - FlinkAccumulatorCombiningStateWithContext( + FlinkCombiningStateWithContext( DefaultOperatorStateBackend flinkStateBackend, - StateTag> address, + StateTag> address, CombineWithContext.KeyedCombineFnWithContext< ? super K, InputT, AccumT, OutputT> combineFn, StateNamespace namespace, @@ -752,7 +752,7 @@ private class FlinkAccumulatorCombiningStateWithContext readLater() { + public CombiningState readLater() { return this; } @@ -847,8 +847,8 @@ public boolean equals(Object o) { return false; } - FlinkAccumulatorCombiningStateWithContext that = - (FlinkAccumulatorCombiningStateWithContext) o; + FlinkCombiningStateWithContext that = + (FlinkCombiningStateWithContext) o; return namespace.equals(that.namespace) && address.equals(that.address); diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java index a29b1b2d55cd6..24b340ead0f93 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java @@ -40,8 +40,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.MapState; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.SetState; @@ -156,9 +156,9 @@ public MapState bindMap( @Override public - AccumulatorCombiningState + CombiningState bindCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, Combine.CombineFn combineFn) { throw new UnsupportedOperationException("bindCombiningValue is not supported."); @@ -166,8 +166,8 @@ public MapState bindMap( @Override public - AccumulatorCombiningState bindKeyedCombiningValue( - StateTag> address, + CombiningState bindKeyedCombiningValue( + StateTag> address, Coder accumCoder, final Combine.KeyedCombineFn combineFn) { throw new UnsupportedOperationException("bindKeyedCombiningValue is not supported."); @@ -176,8 +176,8 @@ AccumulatorCombiningState bindKeyedCombiningValue( @Override public - AccumulatorCombiningState bindKeyedCombiningValueWithContext( - StateTag> address, + CombiningState bindKeyedCombiningValueWithContext( + StateTag> address, Coder accumCoder, CombineWithContext.KeyedCombineFnWithContext< ? super K, InputT, AccumT, OutputT> combineFn) { @@ -190,7 +190,7 @@ public WatermarkHoldState bindWatermark( StateTag> address, OutputTimeFn outputTimeFn) { throw new UnsupportedOperationException( - String.format("%s is not supported", AccumulatorCombiningState.class.getSimpleName())); + String.format("%s is not supported", CombiningState.class.getSimpleName())); } }); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java index d9e87d190309a..2bf0bf1e1ef9a 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java @@ -28,8 +28,8 @@ import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.MapState; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.SetState; @@ -116,9 +116,9 @@ public MapState bindMap( @Override public - AccumulatorCombiningState + CombiningState bindCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, Combine.CombineFn combineFn) { throw new UnsupportedOperationException("bindCombiningValue is not supported."); @@ -126,8 +126,8 @@ public MapState bindMap( @Override public - AccumulatorCombiningState bindKeyedCombiningValue( - StateTag> address, + CombiningState bindKeyedCombiningValue( + StateTag> address, Coder accumCoder, final Combine.KeyedCombineFn combineFn) { throw new UnsupportedOperationException("bindKeyedCombiningValue is not supported."); @@ -136,8 +136,8 @@ AccumulatorCombiningState bindKeyedCombiningValue( @Override public - AccumulatorCombiningState bindKeyedCombiningValueWithContext( - StateTag> address, + CombiningState bindKeyedCombiningValueWithContext( + StateTag> address, Coder accumCoder, CombineWithContext.KeyedCombineFnWithContext< ? super K, InputT, AccumT, OutputT> combineFn) { @@ -150,7 +150,7 @@ public WatermarkHoldState bindWatermark( StateTag> address, OutputTimeFn outputTimeFn) { throw new UnsupportedOperationException( - String.format("%s is not supported", AccumulatorCombiningState.class.getSimpleName())); + String.format("%s is not supported", CombiningState.class.getSimpleName())); } }); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java index 9033ba748e575..4f961e5229c5e 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java @@ -35,8 +35,8 @@ import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.CombineContextFactory; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.MapState; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.SetState; @@ -142,23 +142,23 @@ public MapState bindMap( @Override public - AccumulatorCombiningState + CombiningState bindCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, Combine.CombineFn combineFn) { - return new FlinkAccumulatorCombiningState<>( + return new FlinkCombiningState<>( flinkStateBackend, address, combineFn, namespace, accumCoder); } @Override public - AccumulatorCombiningState bindKeyedCombiningValue( - StateTag> address, + CombiningState bindKeyedCombiningValue( + StateTag> address, Coder accumCoder, final Combine.KeyedCombineFn combineFn) { - return new FlinkKeyedAccumulatorCombiningState<>( + return new FlinkKeyedCombiningState<>( flinkStateBackend, address, combineFn, @@ -169,12 +169,12 @@ AccumulatorCombiningState bindKeyedCombiningValue( @Override public - AccumulatorCombiningState bindKeyedCombiningValueWithContext( - StateTag> address, + CombiningState bindKeyedCombiningValueWithContext( + StateTag> address, Coder accumCoder, CombineWithContext.KeyedCombineFnWithContext< ? super K, InputT, AccumT, OutputT> combineFn) { - return new FlinkAccumulatorCombiningStateWithContext<>( + return new FlinkCombiningStateWithContext<>( flinkStateBackend, address, combineFn, @@ -393,18 +393,18 @@ public int hashCode() { } } - private static class FlinkAccumulatorCombiningState - implements AccumulatorCombiningState { + private static class FlinkCombiningState + implements CombiningState { private final StateNamespace namespace; - private final StateTag> address; + private final StateTag> address; private final Combine.CombineFn combineFn; private final ValueStateDescriptor flinkStateDescriptor; private final KeyedStateBackend flinkStateBackend; - FlinkAccumulatorCombiningState( + FlinkCombiningState( KeyedStateBackend flinkStateBackend, - StateTag> address, + StateTag> address, Combine.CombineFn combineFn, StateNamespace namespace, Coder accumCoder) { @@ -420,7 +420,7 @@ private static class FlinkAccumulatorCombiningState } @Override - public AccumulatorCombiningState readLater() { + public CombiningState readLater() { return this; } @@ -546,8 +546,8 @@ public boolean equals(Object o) { return false; } - FlinkAccumulatorCombiningState that = - (FlinkAccumulatorCombiningState) o; + FlinkCombiningState that = + (FlinkCombiningState) o; return namespace.equals(that.namespace) && address.equals(that.address); @@ -561,19 +561,19 @@ public int hashCode() { } } - private static class FlinkKeyedAccumulatorCombiningState - implements AccumulatorCombiningState { + private static class FlinkKeyedCombiningState + implements CombiningState { private final StateNamespace namespace; - private final StateTag> address; + private final StateTag> address; private final Combine.KeyedCombineFn combineFn; private final ValueStateDescriptor flinkStateDescriptor; private final KeyedStateBackend flinkStateBackend; private final FlinkStateInternals flinkStateInternals; - FlinkKeyedAccumulatorCombiningState( + FlinkKeyedCombiningState( KeyedStateBackend flinkStateBackend, - StateTag> address, + StateTag> address, Combine.KeyedCombineFn combineFn, StateNamespace namespace, Coder accumCoder, @@ -591,7 +591,7 @@ private static class FlinkKeyedAccumulatorCombiningState readLater() { + public CombiningState readLater() { return this; } @@ -721,8 +721,8 @@ public boolean equals(Object o) { return false; } - FlinkKeyedAccumulatorCombiningState that = - (FlinkKeyedAccumulatorCombiningState) o; + FlinkKeyedCombiningState that = + (FlinkKeyedCombiningState) o; return namespace.equals(that.namespace) && address.equals(that.address); @@ -736,11 +736,11 @@ public int hashCode() { } } - private static class FlinkAccumulatorCombiningStateWithContext - implements AccumulatorCombiningState { + private static class FlinkCombiningStateWithContext + implements CombiningState { private final StateNamespace namespace; - private final StateTag> address; + private final StateTag> address; private final CombineWithContext.KeyedCombineFnWithContext< ? super K, InputT, AccumT, OutputT> combineFn; private final ValueStateDescriptor flinkStateDescriptor; @@ -748,9 +748,9 @@ private static class FlinkAccumulatorCombiningStateWithContext flinkStateInternals; private final CombineWithContext.Context context; - FlinkAccumulatorCombiningStateWithContext( + FlinkCombiningStateWithContext( KeyedStateBackend flinkStateBackend, - StateTag> address, + StateTag> address, CombineWithContext.KeyedCombineFnWithContext< ? super K, InputT, AccumT, OutputT> combineFn, StateNamespace namespace, @@ -771,7 +771,7 @@ private static class FlinkAccumulatorCombiningStateWithContext readLater() { + public CombiningState readLater() { return this; } @@ -896,8 +896,8 @@ public boolean equals(Object o) { return false; } - FlinkAccumulatorCombiningStateWithContext that = - (FlinkAccumulatorCombiningStateWithContext) o; + FlinkCombiningStateWithContext that = + (FlinkCombiningStateWithContext) o; return namespace.equals(that.namespace) && address.equals(that.address); diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java index db02cb34ab0af..7e7d1e1e84dc6 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java @@ -32,9 +32,9 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; import org.apache.beam.sdk.util.state.CombiningState; +import org.apache.beam.sdk.util.state.GroupingState; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.ValueState; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; @@ -58,7 +58,7 @@ public class FlinkBroadcastStateInternalsTest { private static final StateTag> STRING_VALUE_ADDR = StateTags.value("stringValue", StringUtf8Coder.of()); - private static final StateTag> + private static final StateTag> SUM_INTEGER_ADDR = StateTags.combiningValueFromInputInternal( "sumInteger", VarIntCoder.of(), Sum.ofIntegers()); private static final StateTag> STRING_BAG_ADDR = @@ -169,7 +169,7 @@ public void testMergeBagIntoNewNamespace() throws Exception { @Test public void testCombiningValue() throws Exception { - CombiningState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); + GroupingState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); // State instances are cached, but depend on the namespace. assertEquals(value, underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR)); @@ -189,7 +189,7 @@ public void testCombiningValue() throws Exception { @Test public void testCombiningIsEmpty() throws Exception { - CombiningState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); + GroupingState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); assertThat(value.isEmpty().read(), Matchers.is(true)); ReadableState readFuture = value.isEmpty(); @@ -202,9 +202,9 @@ public void testCombiningIsEmpty() throws Exception { @Test public void testMergeCombiningValueIntoSource() throws Exception { - AccumulatorCombiningState value1 = + CombiningState value1 = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); - AccumulatorCombiningState value2 = + CombiningState value2 = underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR); value1.add(5); @@ -223,11 +223,11 @@ public void testMergeCombiningValueIntoSource() throws Exception { @Test public void testMergeCombiningValueIntoNewNamespace() throws Exception { - AccumulatorCombiningState value1 = + CombiningState value1 = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); - AccumulatorCombiningState value2 = + CombiningState value2 = underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR); - AccumulatorCombiningState value3 = + CombiningState value3 = underTest.state(NAMESPACE_3, SUM_INTEGER_ADDR); value1.add(5); diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java index 7839cf335264d..d140271cad585 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java @@ -37,9 +37,9 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; import org.apache.beam.sdk.util.state.CombiningState; +import org.apache.beam.sdk.util.state.GroupingState; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.util.state.WatermarkHoldState; @@ -72,7 +72,7 @@ public class FlinkStateInternalsTest { private static final StateTag> STRING_VALUE_ADDR = StateTags.value("stringValue", StringUtf8Coder.of()); - private static final StateTag> + private static final StateTag> SUM_INTEGER_ADDR = StateTags.combiningValueFromInputInternal( "sumInteger", VarIntCoder.of(), Sum.ofIntegers()); private static final StateTag> STRING_BAG_ADDR = @@ -199,7 +199,7 @@ public void testMergeBagIntoNewNamespace() throws Exception { @Test public void testCombiningValue() throws Exception { - CombiningState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); + GroupingState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); // State instances are cached, but depend on the namespace. assertEquals(value, underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR)); @@ -219,7 +219,7 @@ public void testCombiningValue() throws Exception { @Test public void testCombiningIsEmpty() throws Exception { - CombiningState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); + GroupingState value = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); assertThat(value.isEmpty().read(), Matchers.is(true)); ReadableState readFuture = value.isEmpty(); @@ -232,9 +232,9 @@ public void testCombiningIsEmpty() throws Exception { @Test public void testMergeCombiningValueIntoSource() throws Exception { - AccumulatorCombiningState value1 = + CombiningState value1 = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); - AccumulatorCombiningState value2 = + CombiningState value2 = underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR); value1.add(5); @@ -253,11 +253,11 @@ public void testMergeCombiningValueIntoSource() throws Exception { @Test public void testMergeCombiningValueIntoNewNamespace() throws Exception { - AccumulatorCombiningState value1 = + CombiningState value1 = underTest.state(NAMESPACE_1, SUM_INTEGER_ADDR); - AccumulatorCombiningState value2 = + CombiningState value2 = underTest.state(NAMESPACE_2, SUM_INTEGER_ADDR); - AccumulatorCombiningState value3 = + CombiningState value3 = underTest.state(NAMESPACE_3, SUM_INTEGER_ADDR); value1.add(5); diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index fe531c73652b4..2e3dc8a0cf367 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -33,7 +33,7 @@ jar - beam-master-20170329 + beam-master-20170405 1 6 diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java index 5ad6f9f9131c1..7cb0f0ed203cc 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java @@ -369,7 +369,19 @@ public State call() throws Exception { } catch (IOException e) { State state = getState(); if (state.isTerminal()) { - LOG.warn("Job is already terminated. State is {}", state); + LOG.warn("Cancel failed because job is already terminated. State is {}", state); + return state; + } else if (e.getMessage().contains("has terminated")) { + // This handles the case where the getState() call above returns RUNNING but the cancel + // was rejected because the job is in fact done. Hopefully, someday we can delete this + // code if there is better consistency between the State and whether Cancel succeeds. + // + // Example message: + // Workflow modification failed. Causes: (7603adc9e9bff51e): Cannot perform + // operation 'cancel' on Job: 2017-04-01_22_50_59-9269855660514862348. Job has + // terminated in state SUCCESS: Workflow job: 2017-04-01_22_50_59-9269855660514862348 + // succeeded. + LOG.warn("Cancel failed because job is already terminated.", e); return state; } else { String errorMsg = String.format( diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java index 108baddc910d5..e3d2e4ecf201b 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java @@ -618,7 +618,6 @@ public void testGetAggregatorValuesWithUnusedAggregatorThrowsException() thrown.expect(IllegalArgumentException.class); thrown.expectMessage("not used in this pipeline"); - job.getAggregatorValues(aggregator); } @@ -656,7 +655,6 @@ public void testGetAggregatorValuesWhenClientThrowsExceptionThrowsAggregatorRetr thrown.expectCause(is(cause)); thrown.expectMessage(aggregator.toString()); thrown.expectMessage("when retrieving Aggregator values for"); - job.getAggregatorValues(aggregator); } @@ -750,7 +748,7 @@ public void testCancelUnterminatedJobThatFails() throws IOException { Dataflow.Projects.Locations.Jobs.Update.class); when(mockJobs.update(eq(PROJECT_ID), eq(REGION_ID), eq(JOB_ID), any(Job.class))) .thenReturn(update); - when(update.execute()).thenThrow(new IOException()); + when(update.execute()).thenThrow(new IOException("Some random IOException")); DataflowPipelineJob job = new DataflowPipelineJob(JOB_ID, options, null); @@ -758,13 +756,34 @@ public void testCancelUnterminatedJobThatFails() throws IOException { thrown.expectMessage("Failed to cancel job in state RUNNING, " + "please go to the Developers Console to cancel it manually:"); job.cancel(); + } - Job content = new Job(); - content.setProjectId(PROJECT_ID); - content.setId(JOB_ID); - content.setRequestedState("JOB_STATE_CANCELLED"); - verify(mockJobs).update(eq(PROJECT_ID), eq(REGION_ID), eq(JOB_ID), eq(content)); - verify(mockJobs).get(PROJECT_ID, REGION_ID, JOB_ID); + /** + * Test that {@link DataflowPipelineJob#cancel} doesn't throw if the Dataflow service returns + * non-terminal state even though the cancel API call failed, which can happen in practice. + * + *

TODO: delete this code if the API calls become consistent. + */ + @Test + public void testCancelTerminatedJobWithStaleState() throws IOException { + Dataflow.Projects.Locations.Jobs.Get statusRequest = + mock(Dataflow.Projects.Locations.Jobs.Get.class); + + Job statusResponse = new Job(); + statusResponse.setCurrentState("JOB_STATE_RUNNING"); + when(mockJobs.get(PROJECT_ID, REGION_ID, JOB_ID)).thenReturn(statusRequest); + when(statusRequest.execute()).thenReturn(statusResponse); + + Dataflow.Projects.Locations.Jobs.Update update = mock( + Dataflow.Projects.Locations.Jobs.Update.class); + when(mockJobs.update(eq(PROJECT_ID), eq(REGION_ID), eq(JOB_ID), any(Job.class))) + .thenReturn(update); + when(update.execute()).thenThrow(new IOException("Job has terminated in state SUCCESS")); + + DataflowPipelineJob job = new DataflowPipelineJob(JOB_ID, options, null); + State returned = job.cancel(); + assertThat(returned, equalTo(State.RUNNING)); + expectedLogs.verifyWarn("Cancel failed because job is already terminated."); } @Test diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java index be9ff2ea5d672..988a82bdad0a7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java @@ -27,6 +27,7 @@ import com.google.common.util.concurrent.Uninterruptibles; import java.io.File; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -217,10 +218,11 @@ private static void awaitWatermarksOrTimeout( @VisibleForTesting void adaptBoundedReads(Pipeline pipeline) { - pipeline.replace( - PTransformOverride.of( - PTransformMatchers.classEqualTo(BoundedReadFromUnboundedSource.class), - new AdaptedBoundedAsUnbounded.Factory())); + pipeline.replaceAll( + Collections.singletonList( + PTransformOverride.of( + PTransformMatchers.classEqualTo(BoundedReadFromUnboundedSource.class), + new AdaptedBoundedAsUnbounded.Factory()))); } private static class AdaptedBoundedAsUnbounded extends PTransform> { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java index 43fb383a54867..725e9d31f5272 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java @@ -36,8 +36,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.MapState; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.SetState; @@ -137,31 +137,31 @@ public MapState bindMap( } @Override - public AccumulatorCombiningState + public CombiningState bindCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, CombineFn combineFn) { - return new SparkAccumulatorCombiningState<>(namespace, address, accumCoder, key, + return new SparkCombiningState<>(namespace, address, accumCoder, key, combineFn.asKeyedFn()); } @Override - public AccumulatorCombiningState + public CombiningState bindKeyedCombiningValue( - StateTag> address, + StateTag> address, Coder accumCoder, KeyedCombineFn combineFn) { - return new SparkAccumulatorCombiningState<>(namespace, address, accumCoder, key, combineFn); + return new SparkCombiningState<>(namespace, address, accumCoder, key, combineFn); } @Override - public AccumulatorCombiningState + public CombiningState bindKeyedCombiningValueWithContext( - StateTag> address, + StateTag> address, Coder accumCoder, KeyedCombineFnWithContext combineFn) { - return new SparkAccumulatorCombiningState<>(namespace, address, accumCoder, key, + return new SparkCombiningState<>(namespace, address, accumCoder, key, CombineFnUtil.bindContext(combineFn, c)); } @@ -300,16 +300,16 @@ public OutputTimeFn getOutputTimeFn() { } } - private class SparkAccumulatorCombiningState + private class SparkCombiningState extends AbstractState - implements AccumulatorCombiningState { + implements CombiningState { private final K key; private final KeyedCombineFn combineFn; - private SparkAccumulatorCombiningState( + private SparkCombiningState( StateNamespace namespace, - StateTag> address, + StateTag> address, Coder coder, K key, KeyedCombineFn combineFn) { @@ -319,7 +319,7 @@ private SparkAccumulatorCombiningState( } @Override - public SparkAccumulatorCombiningState readLater() { + public SparkCombiningState readLater() { return this; } diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto index 3de03e7307632..3b7c5bd7d7649 100644 --- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto +++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto @@ -65,7 +65,7 @@ message MessageWithComponents { oneof root { Coder coder = 2; CombinePayload combine_payload = 3; - FunctionSpec function_spec = 4; + SdkFunctionSpec sdk_function_spec = 4; ParDoPayload par_do_payload = 6; PTransform ptransform = 7; PCollection pcollection = 8; @@ -73,7 +73,7 @@ message MessageWithComponents { SideInput side_input = 11; WindowIntoPayload window_into_payload = 12; WindowingStrategy windowing_strategy = 13; - UrnWithParameter urn_with_parameter = 14; + FunctionSpec function_spec = 14; } } @@ -133,7 +133,7 @@ message PTransform { // // - when the URN is "urn:beam:transforms:combine" it is a CombinePayload // - UrnWithParameter spec = 1; + FunctionSpec spec = 1; // (Optional) if this node is a composite, a list of the ids of // transforms that it contains. @@ -203,8 +203,8 @@ message PCollection { // The payload for the primitive ParDo transform. message ParDoPayload { - // (Required) The FunctionSpec of the DoFn. - FunctionSpec do_fn = 1; + // (Required) The SdkFunctionSpec of the DoFn. + SdkFunctionSpec do_fn = 1; // (Required) Additional pieces of context the DoFn may require that // are not otherwise represented in the payload. @@ -262,8 +262,8 @@ enum IsBounded { // The payload for the primitive Read transform. message ReadPayload { - // (Required) The FunctionSpec of the source for this Read. - FunctionSpec source = 1; + // (Required) The SdkFunctionSpec of the source for this Read. + SdkFunctionSpec source = 1; // (Required) Whether the source is bounded or unbounded IsBounded is_bounded = 2; @@ -274,15 +274,15 @@ message ReadPayload { // The payload for the WindowInto transform. message WindowIntoPayload { - // (Required) The FunctionSpec of the WindowFn. - FunctionSpec window_fn = 1; + // (Required) The SdkFunctionSpec of the WindowFn. + SdkFunctionSpec window_fn = 1; } // The payload for the special-but-not-primitive Combine transform. message CombinePayload { - // (Required) The FunctionSpec of the CombineFn. - FunctionSpec combine_fn = 1; + // (Required) The SdkFunctionSpec of the CombineFn. + SdkFunctionSpec combine_fn = 1; // (Required) A reference to the Coder to use for accumulators of the CombineFn string accumulator_coder_id = 2; @@ -306,11 +306,11 @@ message Coder { // that can only be used by a particular SDK. It does not include component // coders, as it is beneficial for these to be comprehensible to a runner // regardless of whether the binary format is agree-upon. - FunctionSpec spec = 1; + SdkFunctionSpec spec = 1; // (Optional) If this coder is parametric, such as ListCoder(VarIntCoder), // this is a list of the components. In order for encodings to be identical, - // the FunctionSpec and all components must be identical, recursively. + // the SdkFunctionSpec and all components must be identical, recursively. repeated string component_coder_ids = 2; } @@ -320,10 +320,10 @@ message Coder { // TODO: consider inlining field on PCollection message WindowingStrategy { - // (Required) The FunctionSpec of the UDF that assigns windows, + // (Required) The SdkFunctionSpec of the UDF that assigns windows, // merges windows, and shifts timestamps before they are // combined according to the OutputTime. - FunctionSpec window_fn = 1; + SdkFunctionSpec window_fn = 1; // (Required) Whether or not the window fn is merging. // @@ -577,22 +577,22 @@ message SideInput { // The only access pattern intended for Beam, because of its superior // performance possibilities, is "urn:beam:sideinput:multimap" (or some such // URN) - UrnWithParameter access_pattern = 1; + FunctionSpec access_pattern = 1; - // (Required) The FunctionSpec of the UDF that adapts a particular + // (Required) The SdkFunctionSpec of the UDF that adapts a particular // access_pattern to a user-facing view type. // // For example, View.asSingleton() may include a `view_fn` that adapts a // specially-designed multimap to a single value per window. - FunctionSpec view_fn = 2; + SdkFunctionSpec view_fn = 2; - // (Required) The FunctionSpec of the UDF that maps a main input window + // (Required) The SdkFunctionSpec of the UDF that maps a main input window // to a side input window. // // For example, when the main input is in fixed windows of one hour, this // can specify that the side input should be accessed according to the day // in which that hour falls. - FunctionSpec window_mapping_fn = 3; + SdkFunctionSpec window_mapping_fn = 3; } // An environment for executing UDFs. Generally an SDK container URL, but @@ -609,10 +609,10 @@ message Environment { // A specification of a user defined function. // -message FunctionSpec { +message SdkFunctionSpec { // (Required) A full specification of this function. - UrnWithParameter spec = 1; + FunctionSpec spec = 1; // (Required) Reference to an execution environment capable of // invoking this function. @@ -646,7 +646,7 @@ message FunctionSpec { // version of the Beam Java SDK. The payload will often // then be an opaque message such as bytes in a // language-specific serialization format. -message UrnWithParameter { +message FunctionSpec { // (Required) A URN that describes the accompanying payload. // For any URN that is not recognized (by whomever is inspecting diff --git a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml index 2799b00dfdbd8..1c0f301b8b7b8 100644 --- a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml +++ b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml @@ -380,7 +380,7 @@ - + diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java index fa8277f871513..11d781d613095 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java @@ -228,7 +228,7 @@ public void visitPrimitiveTransform(Node node) { }); } - public void replace(final PTransformOverride override) { + private void replace(final PTransformOverride override) { final Collection matches = new ArrayList<>(); transforms.visit( new PipelineVisitor.Defaults() { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java index 8538e7fc0f179..cd389e80dc632 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java @@ -61,6 +61,9 @@ public abstract List> splitIntoBundles( * An estimate of the total size (in bytes) of the data that would be read from this source. * This estimate is in terms of external storage size, before any decompression or other * processing done by the reader. + * + *

If there is no way to estimate the size of the source + * implementations MAY return 0L. */ public abstract long getEstimatedSizeBytes(PipelineOptions options) throws Exception; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 01c639a08feff..01f0291e527b6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -650,8 +650,7 @@ public T sideInput(PCollectionView view) { Map viewValues = sideInputs.get(view); if (viewValues != null) { BoundedWindow sideInputWindow = - view.getWindowingStrategyInternal() - .getWindowFn() + view.getWindowMappingFn() .getSideInputWindow(element.getWindow()); @SuppressWarnings("unchecked") T windowValue = (T) viewValues.get(sideInputWindow); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupIntoBatches.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupIntoBatches.java index 095ca2a49bb36..2462b1cd263ab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupIntoBatches.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupIntoBatches.java @@ -29,8 +29,8 @@ import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.TimerSpecs; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.StateSpec; import org.apache.beam.sdk.util.state.StateSpecs; import org.apache.beam.sdk.util.state.ValueState; @@ -115,7 +115,7 @@ static class GroupIntoBatchesDoFn private final StateSpec> batchSpec; @StateId(NUM_ELEMENTS_IN_BATCH_ID) - private final StateSpec> + private final StateSpec> numElementsInBatchSpec; @StateId(KEY_ID) @@ -132,7 +132,7 @@ static class GroupIntoBatchesDoFn this.allowedLateness = allowedLateness; this.batchSpec = StateSpecs.bag(inputValueCoder); this.numElementsInBatchSpec = - StateSpecs.combiningValue( + StateSpecs.combining( VarLongCoder.of(), new Combine.CombineFn() { @@ -171,7 +171,7 @@ public void processElement( @TimerId(END_OF_WINDOW_ID) Timer timer, @StateId(BATCH_ID) BagState batch, @StateId(NUM_ELEMENTS_IN_BATCH_ID) - AccumulatorCombiningState numElementsInBatch, + CombiningState numElementsInBatch, @StateId(KEY_ID) ValueState key, ProcessContext c, BoundedWindow window) { @@ -203,7 +203,7 @@ public void onTimerCallback( @StateId(KEY_ID) ValueState key, @StateId(BATCH_ID) BagState batch, @StateId(NUM_ELEMENTS_IN_BATCH_ID) - AccumulatorCombiningState numElementsInBatch, + CombiningState numElementsInBatch, BoundedWindow window) { LOGGER.debug( "*** END OF WINDOW *** for timer timestamp {} in windows {}", @@ -215,7 +215,7 @@ private void flushBatch( Context c, ValueState key, BagState batch, - AccumulatorCombiningState numElementsInBatch) { + CombiningState numElementsInBatch) { Iterable values = batch.read(); // when the timer fires, batch state might be empty if (Iterables.size(values) > 0) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java new file mode 100644 index 0000000000000..7cd6256d9f00b --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java @@ -0,0 +1,36 @@ +/* + * 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.transforms; + +import org.apache.beam.sdk.runners.PipelineRunner; + +/** + * How a view should be physically materialized by a {@link PipelineRunner}. + * + *

A {@link PipelineRunner} will support some set of materializations, and will reject + * {@link ViewFn ViewFns} that require materializations it does not support. See + * {@link Materializations} for known implementations. + */ +public interface Materialization { + /** + * Gets the URN describing this {@link Materialization}. This is a stable, SDK-independent URN + * understood by a {@link PipelineRunner}. + */ + String getUrn(); +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materializations.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materializations.java new file mode 100644 index 0000000000000..35925fae5f7ae --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materializations.java @@ -0,0 +1,53 @@ +/* + * 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.transforms; + +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.util.WindowedValue; + +/** + * Utility methods for constructing known {@link Materialization materializations} for a + * {@link ViewFn}. + */ +public class Materializations { + /** + * The URN for a {@link Materialization} where the primitive view type is an iterable of fully + * specified windowed values. + */ + @Experimental(Kind.CORE_RUNNERS_ONLY) + public static final String ITERABLE_MATERIALIZATION_URN = + "urn:beam:sideinput:materialization:iterable:0.1"; + + /** + * A {@link Materialization} where the primitive view type is an iterable of fully specified + * windowed values. + */ + public static Materialization>> iterable() { + return new IterableMaterialization<>(); + } + + private static class IterableMaterialization + implements Materialization>> { + @Override + public String getUrn() { + return ITERABLE_MATERIALIZATION_URN; + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java index 981d047d15a9f..cdfcb88652dcb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java @@ -37,6 +37,11 @@ * {@code } the type of the value(s) accessible via this {@link PCollectionView} */ public abstract class ViewFn implements Serializable { + /** + * Gets the materialization of this {@link ViewFn}. + */ + public abstract Materialization getMaterialization(); + /** * A function to adapt a primitive view type to a desired view type. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java index 37b73a6047d42..7cd2601b5cc3f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java @@ -55,7 +55,7 @@ * *

Additionaly firings before or after the watermark can be requested by calling * {@code AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)} or - * {@code AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)}. + * {@code AfterWatermark.pastEndOfWindow.withLateFirings(OnceTrigger)}. */ @Experimental(Experimental.Kind.TRIGGER) public class AfterWatermark { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java index a9830579f6a25..31d1f645fd7fc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java @@ -83,7 +83,7 @@ public T sideInput(PCollectionView view) { } BoundedWindow sideInputWindow = - view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow); + view.getWindowMappingFn().getSideInputWindow(mainInputWindow); return sideInputReader.get(view, sideInputWindow); } }; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java deleted file mode 100644 index 30e371656396c..0000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.util; - -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindows; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; - -/** - * Basic side input reader wrapping a {@link PTuple} of side input iterables. Encapsulates - * conversion according to the {@link PCollectionView} and projection to a particular - * window. - */ -public class DirectSideInputReader implements SideInputReader { - - private PTuple sideInputValues; - - private DirectSideInputReader(PTuple sideInputValues) { - this.sideInputValues = sideInputValues; - } - - public static DirectSideInputReader of(PTuple sideInputValues) { - return new DirectSideInputReader(sideInputValues); - } - - @Override - public boolean contains(PCollectionView view) { - return sideInputValues.has(view.getTagInternal()); - } - - @Override - public boolean isEmpty() { - return sideInputValues.isEmpty(); - } - - @Override - public T get(PCollectionView view, final BoundedWindow window) { - final TupleTag>> tag = view.getTagInternal(); - if (!sideInputValues.has(tag)) { - throw new IllegalArgumentException("calling getSideInput() with unknown view"); - } - - if (view.getWindowingStrategyInternal().getWindowFn() instanceof GlobalWindows) { - return view.getViewFn().apply(sideInputValues.get(tag)); - } else { - return view.getViewFn().apply( - Iterables.filter(sideInputValues.get(tag), - new Predicate>() { - @Override - public boolean apply(WindowedValue element) { - return element.getWindows().contains(window); - } - })); - } - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java index f9f2d6d5dc9bb..740aa46522820 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java @@ -64,43 +64,43 @@ public class NumberedShardedFile implements ShardedFile { private static final Pattern DEFAULT_SHARD_TEMPLATE = Pattern.compile("(?x) \\S* (? \\d+) -of- (? \\d+)"); - private final String filePath; + private final String filePattern; private final Pattern shardTemplate; /** * Constructor that uses default shard template. * - * @param filePath path or glob of files to include + * @param filePattern path or glob of files to include */ - public NumberedShardedFile(String filePath) { - this(filePath, DEFAULT_SHARD_TEMPLATE); + public NumberedShardedFile(String filePattern) { + this(filePattern, DEFAULT_SHARD_TEMPLATE); } /** * Constructor. * - * @param filePath path or glob of files to include + * @param filePattern path or glob of files to include * @param shardTemplate template of shard name to parse out the total number of shards * which is used in I/O retry to avoid inconsistency of filesystem. * Customized template should assign name "numshards" to capturing * group - total shard number. */ - public NumberedShardedFile(String filePath, Pattern shardTemplate) { + public NumberedShardedFile(String filePattern, Pattern shardTemplate) { checkArgument( - !Strings.isNullOrEmpty(filePath), - "Expected valid file path, but received %s", filePath); + !Strings.isNullOrEmpty(filePattern), + "Expected valid file path, but received %s", filePattern); checkNotNull( shardTemplate, "Expected non-null shard pattern. " + "Please call the other constructor to use default pattern: %s", DEFAULT_SHARD_TEMPLATE); - this.filePath = filePath; + this.filePattern = filePattern; this.shardTemplate = shardTemplate; } - public String getFilePath() { - return filePath; + public String getFilePattern() { + return filePattern; } /** @@ -112,14 +112,14 @@ public String getFilePath() { @Override public List readFilesWithRetries(Sleeper sleeper, BackOff backOff) throws IOException, InterruptedException { - IOChannelFactory factory = IOChannelUtils.getFactory(filePath); + IOChannelFactory factory = IOChannelUtils.getFactory(filePattern); IOException lastException = null; do { try { // Match inputPath which may contains glob - Collection files = factory.match(filePath); - LOG.debug("Found {} file(s) by matching the path: {}", files.size(), filePath); + Collection files = factory.match(filePattern); + LOG.debug("Found {} file(s) by matching the path: {}", files.size(), filePattern); if (files.isEmpty() || !checkTotalNumOfFiles(files)) { continue; @@ -152,7 +152,7 @@ public List readFilesWithRetries() @Override public String toString() { - return String.format("%s with shard template '%s'", filePath, shardTemplate); + return String.format("%s with shard template '%s'", filePattern, shardTemplate); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java index 7617253c97acf..14b36fd7568d8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java @@ -35,6 +35,8 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.transforms.Materialization; +import org.apache.beam.sdk.transforms.Materializations; import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.InvalidWindows; @@ -192,6 +194,11 @@ public T getDefaultValue() { } } + @Override + public Materialization>> getMaterialization() { + return Materializations.iterable(); + } + @Override public T apply(Iterable> contents) { try { @@ -220,6 +227,10 @@ public T apply(Iterable> contents) { @Experimental(Kind.CORE_RUNNERS_ONLY) public static class IterableViewFn extends ViewFn>, Iterable> { + @Override + public Materialization>> getMaterialization() { + return Materializations.iterable(); + } @Override public Iterable apply(Iterable> contents) { @@ -247,6 +258,11 @@ public T apply(WindowedValue input) { @Deprecated @Experimental(Kind.CORE_RUNNERS_ONLY) public static class ListViewFn extends ViewFn>, List> { + @Override + public Materialization>> getMaterialization() { + return Materializations.iterable(); + } + @Override public List apply(Iterable> contents) { return ImmutableList.copyOf( @@ -271,6 +287,10 @@ public T apply(WindowedValue input) { @Experimental(Kind.CORE_RUNNERS_ONLY) public static class MultimapViewFn extends ViewFn>>, Map>> { + @Override + public Materialization>>> getMaterialization() { + return Materializations.iterable(); + } @Override public Map> apply(Iterable>> elements) { @@ -296,6 +316,11 @@ public Map> apply(Iterable>> elements) { @Deprecated @Experimental(Kind.CORE_RUNNERS_ONLY) public static class MapViewFn extends ViewFn>>, Map> { + @Override + public Materialization>>> getMaterialization() { + return Materializations.iterable(); + } + /** * Input iterable must actually be {@code Iterable>>}. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java deleted file mode 100644 index d03803fd6d490..0000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java +++ /dev/null @@ -1,160 +0,0 @@ -/* - * 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.util; - -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.Map; -import org.apache.beam.sdk.values.TupleTag; - -/** - * A {@code PTuple} is an immutable tuple of - * heterogeneously-typed values, "keyed" by {@link TupleTag}s. - * - *

PTuples can be created and accessed like follows: - *

 {@code
- * String v1 = ...;
- * Integer v2 = ...;
- * Iterable v3 = ...;
- *
- * // Create TupleTags for each of the values to put in the
- * // PTuple (the type of the TupleTag enables tracking the
- * // static type of each of the values in the PTuple):
- * TupleTag tag1 = new TupleTag<>();
- * TupleTag tag2 = new TupleTag<>();
- * TupleTag> tag3 = new TupleTag<>();
- *
- * // Create a PTuple with three values:
- * PTuple povs =
- *     PTuple.of(tag1, v1)
- *         .and(tag2, v2)
- *         .and(tag3, v3);
- *
- * // Create an empty PTuple:
- * Pipeline p = ...;
- * PTuple povs2 = PTuple.empty(p);
- *
- * // Get values out of a PTuple, using the same tags
- * // that were used to put them in:
- * Integer vX = povs.get(tag2);
- * String vY = povs.get(tag1);
- * Iterable vZ = povs.get(tag3);
- *
- * // Get a map of all values in a PTuple:
- * Map, ?> allVs = povs.getAll();
- * } 
- */ -public class PTuple { - /** - * Returns an empty PTuple. - * - *

Longer PTuples can be created by calling - * {@link #and} on the result. - */ - public static PTuple empty() { - return new PTuple(); - } - - /** - * Returns a singleton PTuple containing the given - * value keyed by the given TupleTag. - * - *

Longer PTuples can be created by calling - * {@link #and} on the result. - */ - public static PTuple of(TupleTag tag, V value) { - return empty().and(tag, value); - } - - /** - * Returns a new PTuple that has all the values and - * tags of this PTuple plus the given value and tag. - * - *

The given TupleTag should not already be mapped to a - * value in this PTuple. - */ - public PTuple and(TupleTag tag, V value) { - Map, Object> newMap = new LinkedHashMap, Object>(); - newMap.putAll(valueMap); - newMap.put(tag, value); - return new PTuple(newMap); - } - - /** - * Returns whether this PTuple contains a value with - * the given tag. - */ - public boolean has(TupleTag tag) { - return valueMap.containsKey(tag); - } - - /** - * Returns true if this {@code PTuple} is empty. - */ - public boolean isEmpty() { - return valueMap.isEmpty(); - } - - /** - * Returns the value with the given tag in this - * PTuple. Throws IllegalArgumentException if there is no - * such value, i.e., {@code !has(tag)}. - */ - public V get(TupleTag tag) { - if (!has(tag)) { - throw new IllegalArgumentException( - "TupleTag not found in this PTuple"); - } - @SuppressWarnings("unchecked") - V value = (V) valueMap.get(tag); - return value; - } - - /** - * Returns an immutable Map from TupleTag to corresponding - * value, for all the members of this PTuple. - */ - public Map, ?> getAll() { - return valueMap; - } - - - ///////////////////////////////////////////////////////////////////////////// - // Internal details below here. - - private final Map, ?> valueMap; - - @SuppressWarnings("rawtypes") - private PTuple() { - this(new LinkedHashMap()); - } - - private PTuple(Map, ?> valueMap) { - this.valueMap = Collections.unmodifiableMap(valueMap); - } - - /** - * Returns a PTuple with each of the given tags mapping - * to the corresponding value. - * - *

For internal use only. - */ - public static PTuple ofInternal(Map, ?> valueMap) { - return new PTuple(valueMap); - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategies.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategies.java index 86bf42545af99..9595362c54e58 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategies.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategies.java @@ -31,7 +31,7 @@ import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi.Components; import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; -import org.apache.beam.sdk.common.runner.v1.RunnerApi.UrnWithParameter; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.SdkFunctionSpec; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; import org.apache.beam.sdk.transforms.windowing.Trigger; @@ -148,10 +148,10 @@ public static RunnerApi.MessageWithComponents toProto(WindowFn windowFn) // TODO: re-use components String windowCoderId = UUID.randomUUID().toString(); - RunnerApi.FunctionSpec windowFnSpec = - RunnerApi.FunctionSpec.newBuilder() + RunnerApi.SdkFunctionSpec windowFnSpec = + RunnerApi.SdkFunctionSpec.newBuilder() .setSpec( - UrnWithParameter.newBuilder() + FunctionSpec.newBuilder() .setUrn(CUSTOM_WINDOWFN_URN) .setParameter( Any.pack( @@ -165,9 +165,9 @@ public static RunnerApi.MessageWithComponents toProto(WindowFn windowFn) RunnerApi.Coder windowCoderProto = RunnerApi.Coder.newBuilder() .setSpec( - FunctionSpec.newBuilder() + SdkFunctionSpec.newBuilder() .setSpec( - UrnWithParameter.newBuilder() + FunctionSpec.newBuilder() .setUrn(CUSTOM_CODER_URN) .setParameter( Any.pack( @@ -180,7 +180,7 @@ public static RunnerApi.MessageWithComponents toProto(WindowFn windowFn) .build(); return RunnerApi.MessageWithComponents.newBuilder() - .setFunctionSpec(windowFnSpec) + .setSdkFunctionSpec(windowFnSpec) .setComponents(Components.newBuilder().putCoders(windowCoderId, windowCoderProto)) .build(); } @@ -204,7 +204,7 @@ public static RunnerApi.MessageWithComponents toProto(WindowingStrategy wi .setClosingBehavior(toProto(windowingStrategy.getClosingBehavior())) .setAllowedLateness(windowingStrategy.getAllowedLateness().getMillis()) .setTrigger(Triggers.toProto(windowingStrategy.getTrigger())) - .setWindowFn(windowFnWithComponents.getFunctionSpec()); + .setWindowFn(windowFnWithComponents.getSdkFunctionSpec()); return RunnerApi.MessageWithComponents.newBuilder() .setWindowingStrategy(windowingStrategyProto) @@ -236,7 +236,7 @@ public static RunnerApi.MessageWithComponents toProto(WindowingStrategy wi RunnerApi.WindowingStrategy proto, RunnerApi.Components components) throws InvalidProtocolBufferException { - FunctionSpec windowFnSpec = proto.getWindowFn(); + SdkFunctionSpec windowFnSpec = proto.getWindowFn(); checkArgument( windowFnSpec.getSpec().getUrn().equals(CUSTOM_WINDOWFN_URN), diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/BagState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/BagState.java index c7e6d1300941c..e0eebe598dffc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/BagState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/BagState.java @@ -22,7 +22,7 @@ * * @param The type of elements in the bag. */ -public interface BagState extends CombiningState> { +public interface BagState extends GroupingState> { @Override BagState readLater(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CombiningState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CombiningState.java index 11552628a10c9..80e4dc9478f30 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CombiningState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CombiningState.java @@ -20,23 +20,34 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn; /** - * State that combines multiple {@code InputT} values using a {@link CombineFn} to produce a single - * {@code OutputT} value. + * State for a single value that is managed by a {@link CombineFn}. This is an internal extension + * to {@link GroupingState} that includes the {@code AccumT} type. * * @param the type of values added to the state + * @param the type of accumulator * @param the type of value extracted from the state */ -public interface CombiningState extends ReadableState, State { +public interface CombiningState + extends GroupingState { + + /** + * Read the merged accumulator for this combining value. It is implied that reading the + * state involes reading the accumulator, so {@link #readLater} is sufficient to prefetch for + * this. + */ + AccumT getAccum(); + /** - * Add a value to the buffer. + * Add an accumulator to this combining value. Depending on implementation this may immediately + * merge it with the previous accumulator, or may buffer this accumulator for a future merge. */ - void add(InputT value); + void addAccum(AccumT accum); /** - * Return true if this state is empty. + * Merge the given accumulators according to the underlying combiner. */ - ReadableState isEmpty(); + AccumT mergeAccumulators(Iterable accumulators); @Override - CombiningState readLater(); + CombiningState readLater(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/AccumulatorCombiningState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/GroupingState.java similarity index 52% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/AccumulatorCombiningState.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/GroupingState.java index 8dd1678836123..bd7a8d9cf9d68 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/AccumulatorCombiningState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/GroupingState.java @@ -20,34 +20,23 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn; /** - * State for a single value that is managed by a {@link CombineFn}. This is an internal extension - * to {@link CombiningState} that includes the {@code AccumT} type. + * State that combines multiple {@code InputT} values using a {@link CombineFn} to produce a single + * {@code OutputT} value. * * @param the type of values added to the state - * @param the type of accumulator * @param the type of value extracted from the state */ -public interface AccumulatorCombiningState - extends CombiningState { - - /** - * Read the merged accumulator for this combining value. It is implied that reading the - * state involes reading the accumulator, so {@link #readLater} is sufficient to prefetch for - * this. - */ - AccumT getAccum(); - +public interface GroupingState extends ReadableState, State { /** - * Add an accumulator to this combining value. Depending on implementation this may immediately - * merge it with the previous accumulator, or may buffer this accumulator for a future merge. + * Add a value to the buffer. */ - void addAccum(AccumT accum); + void add(InputT value); /** - * Merge the given accumulators according to the underlying combiner. + * Return true if this state is empty. */ - AccumT mergeAccumulators(Iterable accumulators); + ReadableState isEmpty(); @Override - AccumulatorCombiningState readLater(); + GroupingState readLater(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MapState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MapState.java index 85d99d612d7a1..fb7e807804d06 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MapState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MapState.java @@ -29,22 +29,21 @@ */ public interface MapState extends State { - /** - * Returns the value to which the specified key is mapped in the state. - */ - V get(K key); - /** * Associates the specified value with the specified key in this state. */ void put(K key, V value); /** - * If the specified key is not already associated with a value (or is mapped - * to {@code null}) associates it with the given value and returns - * {@code null}, else returns the current value. + * A deferred read-followed-by-write. + * + *

When {@code read()} is called on the result or state is committed, it forces a read of the + * map and reconciliation with any pending modifications. + * + *

If the specified key is not already associated with a value (or is mapped to {@code null}) + * associates it with the given value and returns {@code null}, else returns the current value. */ - V putIfAbsent(K key, V value); + ReadableState putIfAbsent(K key, V value); /** * Removes the mapping for a key from this map if it is present. @@ -52,42 +51,29 @@ public interface MapState extends State { void remove(K key); /** - * A bulk get. - * @param keys the keys to search for - * @return a iterable view of values, maybe some values is null. - * The order of values corresponds to the order of the keys. + * A deferred lookup. + * + *

A user is encouraged to call {@code get} for all relevant keys and call {@code readLater()} + * on the results. + * + *

When {@code read()} is called, a particular state implementation is encouraged to perform + * all pending reads in a single batch. */ - Iterable get(Iterable keys); - - /** - * Indicate that specified key will be read later. - */ - MapState getLater(K k); - - /** - * Indicate that specified batch keys will be read later. - */ - MapState getLater(Iterable keys); + ReadableState get(K key); /** * Returns a iterable view of the keys contained in this map. */ - Iterable keys(); + ReadableState> keys(); /** * Returns a iterable view of the values contained in this map. */ - Iterable values(); - - /** - * Indicate that all key-values will be read later. - */ - MapState iterateLater(); + ReadableState> values(); /** * Returns a iterable view of all key-values. */ - Iterable> iterate(); - + ReadableState>> entries(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableStates.java similarity index 54% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableStates.java index abead0409c2ad..819eda694a0ce 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableStates.java @@ -15,26 +15,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.util.state; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.nullValue; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; -import org.apache.beam.sdk.values.TupleTag; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; +/** + * Utilities for constructing and manipulating {@link ReadableState} instances. + */ +@Experimental(Kind.STATE) +public class ReadableStates { + + /** + * A {@link ReadableState} constructed from a constant value, hence immediately available. + */ + public static ReadableState immediate(final T value) { + return new ReadableState() { + @Override + public T read() { + return value; + } -/** Unit tests for {@link PTuple}. */ -@RunWith(JUnit4.class) -public final class PTupleTest { - @Test - public void accessingNullVoidValuesShouldNotCauseExceptions() { - TupleTag tag = new TupleTag() {}; - PTuple tuple = PTuple.of(tag, null); - assertTrue(tuple.has(tag)); - assertThat(tuple.get(tag), is(nullValue())); + @Override + public ReadableState readLater() { + return this; + } + }; } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/SetState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/SetState.java index 93058b2cae061..56ea510411be5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/SetState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/SetState.java @@ -23,49 +23,23 @@ * * @param The type of elements in the set. */ -public interface SetState extends CombiningState> { +public interface SetState extends GroupingState> { /** * Returns true if this set contains the specified element. */ - boolean contains(T t); + ReadableState contains(T t); /** - * Add a value to the buffer if it is not already present. - * If this set already contains the element, the call leaves the set - * unchanged and returns false. + * Ensures a value is a member of the set, returning {@code true} if it was added and {@code + * false} otherwise. */ - boolean addIfAbsent(T t); + ReadableState addIfAbsent(T t); /** * Removes the specified element from this set if it is present. */ void remove(T t); - /** - * Indicate that elements will be read later. - * @param elements to be read later - * @return this for convenient chaining - */ - SetState readLater(Iterable elements); - - /** - *

Checks if SetState contains any given elements.

- * - * @param elements the elements to search for - * @return the {@code true} if any of the elements are found, - * {@code false} if no match - */ - boolean containsAny(Iterable elements); - - /** - *

Checks if SetState contains all given elements.

- * - * @param elements the elements to find - * @return true if the SetState contains all elements, - * false if not - */ - boolean containsAll(Iterable elements); - @Override SetState readLater(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/State.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/State.java index 973cb9c2f5253..3a49f017bec7c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/State.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/State.java @@ -21,7 +21,7 @@ * Base interface for all state locations. * *

Specific types of state add appropriate accessors for reading and writing values, see - * {@link ValueState}, {@link BagState}, and {@link CombiningState}. + * {@link ValueState}, {@link BagState}, and {@link GroupingState}. */ public interface State { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateBinder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateBinder.java index fbfb475c42c03..64841fbf805f4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateBinder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateBinder.java @@ -39,23 +39,23 @@ MapState bindMap( String id, StateSpec> spec, Coder mapKeyCoder, Coder mapValueCoder); - AccumulatorCombiningState bindCombiningValue( + CombiningState bindCombining( String id, - StateSpec> spec, + StateSpec> spec, Coder accumCoder, Combine.CombineFn combineFn); - AccumulatorCombiningState bindKeyedCombiningValue( + CombiningState bindKeyedCombining( String id, - StateSpec> spec, + StateSpec> spec, Coder accumCoder, Combine.KeyedCombineFn combineFn); - AccumulatorCombiningState bindKeyedCombiningValueWithContext( + CombiningState bindKeyedCombiningWithContext( String id, - StateSpec> spec, + StateSpec> spec, Coder accumCoder, CombineWithContext.KeyedCombineFnWithContext combineFn); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpecs.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpecs.java index 6a8c80b102a23..30a7a6d0ccdc2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpecs.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpecs.java @@ -62,9 +62,9 @@ public static StateSpec> value(Coder valueCoder) { * {@code InputT}s into a single {@code OutputT}. */ public static - StateSpec> combiningValue( + StateSpec> combining( CombineFn combineFn) { - return new CombiningValueStateSpec(null, combineFn); + return new CombiningStateSpec(null, combineFn); } /** @@ -72,12 +72,12 @@ StateSpec> combiningV * {@code InputT}s into a single {@code OutputT}. */ public static - StateSpec> combiningValue( + StateSpec> combining( Coder accumCoder, CombineFn combineFn) { checkArgument(accumCoder != null, "accumCoder should not be null. " - + "Consider using combiningValue(CombineFn<> combineFn) instead."); - return combiningValueInternal(accumCoder, combineFn); + + "Consider using combining(CombineFn<> combineFn) instead."); + return combiningInternal(accumCoder, combineFn); } /** @@ -85,9 +85,9 @@ StateSpec> combiningV * multiple {@code InputT}s into a single {@code OutputT}. */ public static - StateSpec> keyedCombiningValue( + StateSpec> keyedCombining( KeyedCombineFn combineFn) { - return new KeyedCombiningValueStateSpec(null, combineFn); + return new KeyedCombiningStateSpec(null, combineFn); } /** @@ -95,12 +95,12 @@ StateSpec> keyedCombiningV * multiple {@code InputT}s into a single {@code OutputT}. */ public static - StateSpec> keyedCombiningValue( + StateSpec> keyedCombining( Coder accumCoder, KeyedCombineFn combineFn) { checkArgument(accumCoder != null, "accumCoder should not be null. " - + "Consider using keyedCombiningValue(KeyedCombineFn<> combineFn) instead."); - return keyedCombiningValueInternal(accumCoder, combineFn); + + "Consider using keyedCombining(KeyedCombineFn<> combineFn) instead."); + return keyedCombiningInternal(accumCoder, combineFn); } /** @@ -108,9 +108,9 @@ StateSpec> keyedCombiningV * merge multiple {@code InputT}s into a single {@code OutputT}. */ public static - StateSpec> - keyedCombiningValueWithContext(KeyedCombineFnWithContext combineFn) { - return new KeyedCombiningValueWithContextStateSpec(null, combineFn); + StateSpec> + keyedCombiningWithContext(KeyedCombineFnWithContext combineFn) { + return new KeyedCombiningWithContextStateSpec(null, combineFn); } /** @@ -118,14 +118,14 @@ StateSpec> keyedCombiningV * merge multiple {@code InputT}s into a single {@code OutputT}. */ public static - StateSpec> - keyedCombiningValueWithContext( + StateSpec> + keyedCombiningWithContext( Coder accumCoder, KeyedCombineFnWithContext combineFn) { checkArgument(accumCoder != null, "accumCoder should not be null. Consider using " - + "keyedCombiningValueWithContext(KeyedCombineFnWithContext<> combineFn) instead."); - return new KeyedCombiningValueWithContextStateSpec( + + "keyedCombiningWithContext(KeyedCombineFnWithContext<> combineFn) instead."); + return new KeyedCombiningWithContextStateSpec( accumCoder, combineFn); } @@ -137,12 +137,12 @@ StateSpec> keyedCombiningV * only be used to initialize static values. */ public static - StateSpec> - combiningValueFromInputInternal( + StateSpec> + combiningFromInputInternal( Coder inputCoder, CombineFn combineFn) { try { Coder accumCoder = combineFn.getAccumulatorCoder(STANDARD_REGISTRY, inputCoder); - return combiningValueInternal(accumCoder, combineFn); + return combiningInternal(accumCoder, combineFn); } catch (CannotProvideCoderException e) { throw new IllegalArgumentException( "Unable to determine accumulator coder for " @@ -154,15 +154,15 @@ StateSpec> keyedCombiningV } private static - StateSpec> combiningValueInternal( + StateSpec> combiningInternal( Coder accumCoder, CombineFn combineFn) { - return new CombiningValueStateSpec(accumCoder, combineFn); + return new CombiningStateSpec(accumCoder, combineFn); } private static - StateSpec> keyedCombiningValueInternal( + StateSpec> keyedCombiningInternal( Coder accumCoder, KeyedCombineFn combineFn) { - return new KeyedCombiningValueStateSpec(accumCoder, combineFn); + return new KeyedCombiningStateSpec(accumCoder, combineFn); } /** @@ -219,18 +219,18 @@ StateSpec> watermarkStateInternal( public static StateSpec> convertToBagSpecInternal( - StateSpec> combiningSpec) { - if (combiningSpec instanceof KeyedCombiningValueStateSpec) { + StateSpec> combiningSpec) { + if (combiningSpec instanceof KeyedCombiningStateSpec) { // Checked above; conversion to a bag spec depends on the provided spec being one of those // created via the factory methods in this class. @SuppressWarnings("unchecked") - KeyedCombiningValueStateSpec typedSpec = - (KeyedCombiningValueStateSpec) combiningSpec; + KeyedCombiningStateSpec typedSpec = + (KeyedCombiningStateSpec) combiningSpec; return typedSpec.asBagSpec(); - } else if (combiningSpec instanceof KeyedCombiningValueWithContextStateSpec) { + } else if (combiningSpec instanceof KeyedCombiningWithContextStateSpec) { @SuppressWarnings("unchecked") - KeyedCombiningValueWithContextStateSpec typedSpec = - (KeyedCombiningValueWithContextStateSpec) combiningSpec; + KeyedCombiningWithContextStateSpec typedSpec = + (KeyedCombiningWithContextStateSpec) combiningSpec; return typedSpec.asBagSpec(); } else { throw new IllegalArgumentException("Unexpected StateSpec " + combiningSpec); @@ -300,15 +300,15 @@ public int hashCode() { * *

Includes the {@link CombineFn} and the coder for the accumulator type. */ - private static class CombiningValueStateSpec - extends KeyedCombiningValueStateSpec - implements StateSpec> { + private static class CombiningStateSpec + extends KeyedCombiningStateSpec + implements StateSpec> { @Nullable private Coder accumCoder; private final CombineFn combineFn; - private CombiningValueStateSpec( + private CombiningStateSpec( @Nullable Coder accumCoder, CombineFn combineFn) { super(accumCoder, combineFn.asKeyedFn()); @@ -338,14 +338,14 @@ public void offerCoders(Coder[] coders) { * *

Includes the {@link KeyedCombineFnWithContext} and the coder for the accumulator type. */ - private static class KeyedCombiningValueWithContextStateSpec - implements StateSpec> { + private static class KeyedCombiningWithContextStateSpec + implements StateSpec> { @Nullable private Coder accumCoder; private final KeyedCombineFnWithContext combineFn; - protected KeyedCombiningValueWithContextStateSpec( + protected KeyedCombiningWithContextStateSpec( @Nullable Coder accumCoder, KeyedCombineFnWithContext combineFn) { this.combineFn = combineFn; @@ -353,9 +353,9 @@ protected KeyedCombiningValueWithContextStateSpec( } @Override - public AccumulatorCombiningState bind( + public CombiningState bind( String id, StateBinder visitor) { - return visitor.bindKeyedCombiningValueWithContext(id, this, accumCoder, combineFn); + return visitor.bindKeyedCombiningWithContext(id, this, accumCoder, combineFn); } @SuppressWarnings("unchecked") @@ -371,9 +371,9 @@ public void offerCoders(Coder[] coders) { @Override public void finishSpecifying() { if (accumCoder == null) { throw new IllegalStateException("Unable to infer a coder for" - + " KeyedCombiningValueWithContextState and no Coder was specified." + + " KeyedCombiningWithContextState and no Coder was specified." + " Please set a coder by either invoking" - + " StateSpecs.keyedCombiningValue(Coder accumCoder," + + " StateSpecs.keyedCombining(Coder accumCoder," + " KeyedCombineFn combineFn)" + " or by registering the coder in the Pipeline's CoderRegistry."); } @@ -385,12 +385,12 @@ public boolean equals(Object obj) { return true; } - if (!(obj instanceof KeyedCombiningValueWithContextStateSpec)) { + if (!(obj instanceof KeyedCombiningWithContextStateSpec)) { return false; } - KeyedCombiningValueWithContextStateSpec that = - (KeyedCombiningValueWithContextStateSpec) obj; + KeyedCombiningWithContextStateSpec that = + (KeyedCombiningWithContextStateSpec) obj; return Objects.equals(this.accumCoder, that.accumCoder); } @@ -409,14 +409,14 @@ private StateSpec> asBagSpec() { * *

Includes the {@link KeyedCombineFn} and the coder for the accumulator type. */ - private static class KeyedCombiningValueStateSpec - implements StateSpec> { + private static class KeyedCombiningStateSpec + implements StateSpec> { @Nullable private Coder accumCoder; private final KeyedCombineFn keyedCombineFn; - protected KeyedCombiningValueStateSpec( + protected KeyedCombiningStateSpec( @Nullable Coder accumCoder, KeyedCombineFn keyedCombineFn) { this.keyedCombineFn = keyedCombineFn; @@ -428,9 +428,9 @@ protected Coder getAccumCoder() { } @Override - public AccumulatorCombiningState bind( + public CombiningState bind( String id, StateBinder visitor) { - return visitor.bindKeyedCombiningValue(id, this, getAccumCoder(), keyedCombineFn); + return visitor.bindKeyedCombining(id, this, getAccumCoder(), keyedCombineFn); } @SuppressWarnings("unchecked") @@ -445,9 +445,9 @@ public void offerCoders(Coder[] coders) { @Override public void finishSpecifying() { if (getAccumCoder() == null) { - throw new IllegalStateException("Unable to infer a coder for CombiningState and no" + throw new IllegalStateException("Unable to infer a coder for GroupingState and no" + " Coder was specified. Please set a coder by either invoking" - + " StateSpecs.combiningValue(Coder accumCoder," + + " StateSpecs.combining(Coder accumCoder," + " CombineFn combineFn)" + " or by registering the coder in the Pipeline's CoderRegistry."); } @@ -459,12 +459,12 @@ public boolean equals(Object obj) { return true; } - if (!(obj instanceof CombiningValueStateSpec)) { + if (!(obj instanceof CombiningStateSpec)) { return false; } - KeyedCombiningValueStateSpec that = - (KeyedCombiningValueStateSpec) obj; + KeyedCombiningStateSpec that = + (KeyedCombiningStateSpec) obj; return Objects.equals(this.accumCoder, that.accumCoder); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java index 415cc6e143e5a..20fa05ff45812 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java @@ -31,7 +31,7 @@ */ @Experimental(Kind.STATE) public interface WatermarkHoldState - extends CombiningState { + extends GroupingState { /** * Return the {@link OutputTimeFn} which will be used to determine a watermark hold time given * an element timestamp, and to combine watermarks from windows which are about to be merged. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java index e6b13c15efd8e..5f71cabb6148b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.beam.sdk.testing; import com.google.common.base.Function; @@ -25,6 +26,8 @@ import java.util.Objects; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.transforms.Materialization; +import org.apache.beam.sdk.transforms.Materializations; import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; @@ -85,6 +88,11 @@ private PCollectionViewTesting() { } * A {@link ViewFn} that returns the provided contents as a fully lazy iterable. */ public static class IdentityViewFn extends ViewFn>, Iterable> { + @Override + public Materialization>> getMaterialization() { + return Materializations.iterable(); + } + @Override public Iterable apply(Iterable> contents) { return Iterables.transform(contents, new Function, T>() { @@ -103,6 +111,11 @@ public T apply(WindowedValue windowedValue) { * provided than {@code Integer.MAX_VALUE} then behavior is unpredictable. */ public static class LengthViewFn extends ViewFn>, Long> { + @Override + public Materialization>> getMaterialization() { + return Materializations.iterable(); + } + @Override public Long apply(Iterable> contents) { return (long) Iterables.size(contents); @@ -120,6 +133,11 @@ public ConstantViewFn(ViewT value) { this.value = value; } + @Override + public Materialization>> getMaterialization() { + return Materializations.iterable(); + } + @Override public ViewT apply(Iterable> contents) { return value; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index 4249a779c8d33..b429eab17288e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -77,6 +77,7 @@ import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.DoFn.OnTimer; import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.Mean.CountSum; import org.apache.beam.sdk.transforms.ParDo.SingleOutput; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; @@ -92,8 +93,8 @@ import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; -import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; +import org.apache.beam.sdk.util.state.CombiningState; import org.apache.beam.sdk.util.state.MapState; import org.apache.beam.sdk.util.state.SetState; import org.apache.beam.sdk.util.state.StateSpec; @@ -2085,15 +2086,15 @@ public void testSetState() { private final StateSpec> setState = StateSpecs.set(VarIntCoder.of()); @StateId(countStateId) - private final StateSpec> - countState = StateSpecs.combiningValueFromInputInternal(VarIntCoder.of(), + private final StateSpec> + countState = StateSpecs.combiningFromInputInternal(VarIntCoder.of(), Sum.ofIntegers()); @ProcessElement public void processElement( ProcessContext c, @StateId(stateId) SetState state, - @StateId(countStateId) AccumulatorCombiningState + @StateId(countStateId) CombiningState count) { state.add(c.element().getValue()); count.add(1); @@ -2129,15 +2130,15 @@ public void testSetStateCoderInference() { private final StateSpec> setState = StateSpecs.set(); @StateId(countStateId) - private final StateSpec> - countState = StateSpecs.combiningValueFromInputInternal(VarIntCoder.of(), + private final StateSpec> + countState = StateSpecs.combiningFromInputInternal(VarIntCoder.of(), Sum.ofIntegers()); @ProcessElement public void processElement( ProcessContext c, @StateId(stateId) SetState state, - @StateId(countStateId) AccumulatorCombiningState count) { + @StateId(countStateId) CombiningState count) { state.add(new MyInteger(c.element().getValue())); count.add(1); if (count.read() >= 4) { @@ -2172,15 +2173,15 @@ public void testSetStateCoderInferenceFailure() throws Exception { private final StateSpec> setState = StateSpecs.set(); @StateId(countStateId) - private final StateSpec> - countState = StateSpecs.combiningValueFromInputInternal(VarIntCoder.of(), + private final StateSpec> + countState = StateSpecs.combiningFromInputInternal(VarIntCoder.of(), Sum.ofIntegers()); @ProcessElement public void processElement( ProcessContext c, @StateId(stateId) SetState state, - @StateId(countStateId) AccumulatorCombiningState count) { + @StateId(countStateId) CombiningState count) { state.add(new MyInteger(c.element().getValue())); count.add(1); if (count.read() >= 4) { @@ -2214,20 +2215,20 @@ public void testMapState() { private final StateSpec> mapState = StateSpecs.map(StringUtf8Coder.of(), VarIntCoder.of()); @StateId(countStateId) - private final StateSpec> - countState = StateSpecs.combiningValueFromInputInternal(VarIntCoder.of(), + private final StateSpec> + countState = StateSpecs.combiningFromInputInternal(VarIntCoder.of(), Sum.ofIntegers()); @ProcessElement public void processElement( ProcessContext c, @StateId(stateId) MapState state, - @StateId(countStateId) AccumulatorCombiningState + @StateId(countStateId) CombiningState count) { KV value = c.element().getValue(); state.put(value.getKey(), value.getValue()); count.add(1); if (count.read() >= 4) { - Iterable> iterate = state.iterate(); + Iterable> iterate = state.entries().read(); for (Map.Entry entry : iterate) { c.output(KV.of(entry.getKey(), entry.getValue())); } @@ -2260,20 +2261,20 @@ public void testMapStateCoderInference() { @StateId(stateId) private final StateSpec> mapState = StateSpecs.map(); @StateId(countStateId) - private final StateSpec> - countState = StateSpecs.combiningValueFromInputInternal(VarIntCoder.of(), + private final StateSpec> + countState = StateSpecs.combiningFromInputInternal(VarIntCoder.of(), Sum.ofIntegers()); @ProcessElement public void processElement( ProcessContext c, @StateId(stateId) MapState state, - @StateId(countStateId) AccumulatorCombiningState + @StateId(countStateId) CombiningState count) { KV value = c.element().getValue(); state.put(value.getKey(), new MyInteger(value.getValue())); count.add(1); if (count.read() >= 4) { - Iterable> iterate = state.iterate(); + Iterable> iterate = state.entries().read(); for (Map.Entry entry : iterate) { c.output(KV.of(entry.getKey(), entry.getValue())); } @@ -2306,20 +2307,20 @@ public void testMapStateCoderInferenceFailure() throws Exception { @StateId(stateId) private final StateSpec> mapState = StateSpecs.map(); @StateId(countStateId) - private final StateSpec> - countState = StateSpecs.combiningValueFromInputInternal(VarIntCoder.of(), + private final StateSpec> + countState = StateSpecs.combiningFromInputInternal(VarIntCoder.of(), Sum.ofIntegers()); @ProcessElement public void processElement( ProcessContext c, @StateId(stateId) MapState state, - @StateId(countStateId) AccumulatorCombiningState + @StateId(countStateId) CombiningState count) { KV value = c.element().getValue(); state.put(value.getKey(), new MyInteger(value.getValue())); count.add(1); if (count.read() >= 4) { - Iterable> iterate = state.iterate(); + Iterable> iterate = state.entries().read(); for (Map.Entry entry : iterate) { c.output(KV.of(entry.getKey(), entry.getValue())); } @@ -2351,15 +2352,15 @@ public void testCombiningState() { @StateId(stateId) private final StateSpec< - Object, AccumulatorCombiningState, Double>> + Object, CombiningState, Double>> combiningState = - StateSpecs.combiningValue(new Mean.CountSumCoder(), Mean.of()); + StateSpecs.combining(new Mean.CountSumCoder(), Mean.of()); @ProcessElement public void processElement( ProcessContext c, @StateId(stateId) - AccumulatorCombiningState, Double> state) { + CombiningState, Double> state) { state.add(c.element().getValue()); Double currentValue = state.read(); if (Math.abs(currentValue - 0.5) < EPSILON) { @@ -2391,9 +2392,9 @@ public void testCombiningStateCoderInference() { @StateId(stateId) private final StateSpec< - Object, AccumulatorCombiningState> + Object, CombiningState> combiningState = - StateSpecs.combiningValue(new Combine.CombineFn() { + StateSpecs.combining(new Combine.CombineFn() { @Override public MyInteger createAccumulator() { return new MyInteger(0); @@ -2423,7 +2424,7 @@ public Integer extractOutput(MyInteger accumulator) { public void processElement( ProcessContext c, @StateId(stateId) - AccumulatorCombiningState state) { + CombiningState state) { state.add(c.element().getValue()); Integer currentValue = state.read(); if (currentValue == EXPECTED_SUM) { @@ -2453,9 +2454,9 @@ public void testCombiningStateCoderInferenceFailure() throws Exception { @StateId(stateId) private final StateSpec< - Object, AccumulatorCombiningState> + Object, CombiningState> combiningState = - StateSpecs.combiningValue(new Combine.CombineFn() { + StateSpecs.combining(new Combine.CombineFn() { @Override public MyInteger createAccumulator() { return new MyInteger(0); @@ -2485,7 +2486,7 @@ public Integer extractOutput(MyInteger accumulator) { public void processElement( ProcessContext c, @StateId(stateId) - AccumulatorCombiningState state) { + CombiningState state) { state.add(c.element().getValue()); Integer currentValue = state.read(); if (currentValue == EXPECTED_SUM) { @@ -2495,7 +2496,7 @@ public void processElement( }; thrown.expect(RuntimeException.class); - thrown.expectMessage("Unable to infer a coder for CombiningState and no Coder was specified."); + thrown.expectMessage("Unable to infer a coder for GroupingState and no Coder was specified."); pipeline .apply(Create.of(KV.of("hello", 3), KV.of("hello", 6), KV.of("hello", 7))) diff --git a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java b/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java index a14f480fd998e..bca0fe8d050cc 100644 --- a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java +++ b/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java @@ -34,7 +34,6 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFnTester; @@ -52,7 +51,6 @@ import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; -import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; @@ -135,7 +133,6 @@ public void testSizes() throws Exception { } @Test - @Category(ValidatesRunner.class) public void testRead() throws Exception { ElasticSearchIOTestUtils.insertTestDocuments(ES_INDEX, ES_TYPE, NUM_DOCS, node.client()); @@ -152,7 +149,6 @@ public void testRead() throws Exception { } @Test - @Category(ValidatesRunner.class) public void testReadWithQuery() throws Exception { ElasticSearchIOTestUtils.insertTestDocuments(ES_INDEX, ES_TYPE, NUM_DOCS, node.client()); @@ -179,7 +175,6 @@ public void testReadWithQuery() throws Exception { } @Test - @Category(ValidatesRunner.class) public void testWrite() throws Exception { List data = ElasticSearchIOTestUtils.createDocuments( diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index f784e27b6ccc7..a5b2e5692202b 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -32,7 +32,7 @@ jar - 0.9.5.1 + 0.9.6 @@ -81,26 +81,6 @@ - - - org.apache.maven.plugins - maven-dependency-plugin - - - analyze-only - - - - com.google.api.grpc:grpc-google-common-protos - - - - - @@ -223,7 +203,7 @@ io.netty netty-tcnative-boringssl-static - 1.1.33.Fork18 + 1.1.33.Fork26 runtime diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 2cdd11d8999e2..89c67a4592945 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -212,9 +212,8 @@ public Read withBigtableOptions(BigtableOptions.Builder optionsBuilder) { // TODO: is there a better way to clone a Builder? Want it to be immune from user changes. BigtableOptions options = optionsBuilder.build(); - // Set data channel count to one because there is only 1 scanner in this session BigtableOptions.Builder clonedBuilder = options.toBuilder() - .setDataChannelCount(1); + .setUseCachedDataPool(true); BigtableOptions optionsWithAgent = clonedBuilder.setUserAgent(getBeamSdkPartOfUserAgent()).build(); @@ -449,7 +448,8 @@ public Write withBigtableOptions(BigtableOptions.Builder optionsBuilder) { .setBulkOptions( options.getBulkOptions().toBuilder() .setUseBulkApi(true) - .build()); + .build()) + .setUseCachedDataPool(true); BigtableOptions optionsWithAgent = clonedBuilder.setUserAgent(getBeamSdkPartOfUserAgent()).build(); return new Write(optionsWithAgent, tableId, bigtableService); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index 5ef8b2cd582d9..92ab204f5ca51 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -118,12 +118,10 @@ import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.testing.ExpectedLogs; -import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.testing.SourceTestUtils.ExpectedSplitOutcome; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnTester; @@ -163,7 +161,6 @@ import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; -import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; @@ -803,7 +800,6 @@ public void testValidateReadSetsDefaultProject() throws Exception { } @Test - @Category(NeedsRunner.class) public void testBuildSourceWithTableAndFlatten() { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); bqOptions.setProject("defaultProject"); @@ -822,7 +818,6 @@ public void testBuildSourceWithTableAndFlatten() { } @Test - @Category(NeedsRunner.class) public void testBuildSourceWithTableAndFlattenWithoutValidation() { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); bqOptions.setProject("defaultProject"); @@ -842,7 +837,6 @@ public void testBuildSourceWithTableAndFlattenWithoutValidation() { } @Test - @Category(NeedsRunner.class) public void testBuildSourceWithTableAndSqlDialect() { BigQueryOptions bqOptions = PipelineOptionsFactory.as(BigQueryOptions.class); bqOptions.setProject("defaultProject"); @@ -861,7 +855,6 @@ public void testBuildSourceWithTableAndSqlDialect() { } @Test - @Category(NeedsRunner.class) public void testReadFromTable() throws IOException, InterruptedException { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); bqOptions.setProject("defaultProject"); @@ -944,7 +937,6 @@ public void processElement(ProcessContext c) throws Exception { } @Test - @Category(NeedsRunner.class) public void testWrite() throws Exception { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); bqOptions.setProject("defaultProject"); @@ -976,7 +968,6 @@ public void testWrite() throws Exception { } @Test - @Category(NeedsRunner.class) public void testStreamingWrite() throws Exception { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); bqOptions.setProject("defaultProject"); @@ -1105,7 +1096,6 @@ public PartitionedGlobalWindow decode(InputStream inStream, Context context) } @Test - @Category(NeedsRunner.class) public void testStreamingWriteWithWindowFn() throws Exception { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); bqOptions.setProject("defaultProject"); @@ -1188,7 +1178,6 @@ public TableRow apply(Integer i) { } @Test - @Category(NeedsRunner.class) public void testWriteUnknown() throws Exception { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); bqOptions.setProject("defaultProject"); @@ -1221,7 +1210,6 @@ public void testWriteUnknown() throws Exception { } @Test - @Category(NeedsRunner.class) public void testWriteFailedJobs() throws Exception { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); bqOptions.setProject("defaultProject"); @@ -1291,8 +1279,7 @@ public void testBuildSourceDisplayDataQuery() { } @Test - @Category(ValidatesRunner.class) - @Ignore("[BEAM-436] DirectRunner ValidatesRunner tempLocation configuration insufficient") + @Ignore("[BEAM-436] DirectRunner tempLocation configuration insufficient") public void testTableSourcePrimitiveDisplayData() throws IOException, InterruptedException { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); BigQueryIO.Read read = BigQueryIO.read() @@ -1308,8 +1295,7 @@ public void testTableSourcePrimitiveDisplayData() throws IOException, Interrupte } @Test - @Category(ValidatesRunner.class) - @Ignore("[BEAM-436] DirectRunner ValidatesRunner tempLocation configuration insufficient") + @Ignore("[BEAM-436] DirectRunner tempLocation configuration insufficient") public void testQuerySourcePrimitiveDisplayData() throws IOException, InterruptedException { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); BigQueryIO.Read read = BigQueryIO.read() @@ -1335,15 +1321,13 @@ public void testBuildWrite() { } @Test - @Category(ValidatesRunner.class) - @Ignore("[BEAM-436] DirectRunner ValidatesRunner tempLocation configuration insufficient") + @Ignore("[BEAM-436] DirectRunner tempLocation configuration insufficient") public void testBatchWritePrimitiveDisplayData() throws IOException, InterruptedException { testWritePrimitiveDisplayData(/* streaming: */ false); } @Test - @Category(ValidatesRunner.class) - @Ignore("[BEAM-436] DirectRunner ValidatesRunner tempLocation configuration insufficient") + @Ignore("[BEAM-436] DirectRunner tempLocation configuration insufficient") public void testStreamingWritePrimitiveDisplayData() throws IOException, InterruptedException { testWritePrimitiveDisplayData(/* streaming: */ true); } @@ -2014,7 +1998,6 @@ public String apply(Long input) { } @Test - @Category(NeedsRunner.class) public void testPassThroughThenCleanup() throws Exception { PCollection output = p @@ -2031,7 +2014,6 @@ void cleanup(PipelineOptions options) throws Exception { } @Test - @Category(NeedsRunner.class) public void testPassThroughThenCleanupExecuted() throws Exception { p.apply(Create.empty(VarIntCoder.of())) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index 5d3b27869462e..c09a1fa5c2284 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -88,7 +88,6 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.transforms.DoFnTester.CloningBehavior; import org.apache.beam.sdk.transforms.PTransform; @@ -100,7 +99,6 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -425,7 +423,6 @@ public void testDeleteKeyDisplayData() { } @Test - @Category(ValidatesRunner.class) public void testWritePrimitiveDisplayData() { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); PTransform, ?> write = @@ -439,7 +436,6 @@ public void testWritePrimitiveDisplayData() { } @Test - @Category(ValidatesRunner.class) public void testDeleteEntityPrimitiveDisplayData() { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); PTransform, ?> write = @@ -453,7 +449,6 @@ public void testDeleteEntityPrimitiveDisplayData() { } @Test - @Category(ValidatesRunner.class) public void testDeleteKeyPrimitiveDisplayData() { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); PTransform, ?> write = diff --git a/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java b/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java index 10b4ed27fe084..8101f4b7f9336 100644 --- a/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java +++ b/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java @@ -74,7 +74,8 @@ public static Job newJob(@Nullable SerializableConfiguration conf) throws IOExce if (conf == null) { return Job.getInstance(); } else { - Job job = Job.getInstance(); + // Don't reading configuration from slave thread, but only from master thread. + Job job = Job.getInstance(new Configuration(false)); for (Map.Entry entry : conf.get()) { job.getConfiguration().set(entry.getKey(), entry.getValue()); } diff --git a/sdks/java/io/hadoop-input-format/README.md b/sdks/java/io/hadoop/README.md similarity index 100% rename from sdks/java/io/hadoop-input-format/README.md rename to sdks/java/io/hadoop/README.md diff --git a/sdks/java/io/hadoop-input-format/pom.xml b/sdks/java/io/hadoop/input-format/pom.xml similarity index 67% rename from sdks/java/io/hadoop-input-format/pom.xml rename to sdks/java/io/hadoop/input-format/pom.xml index 6680087eb12b3..9558ecd266537 100644 --- a/sdks/java/io/hadoop-input-format/pom.xml +++ b/sdks/java/io/hadoop/input-format/pom.xml @@ -15,51 +15,25 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.beam - beam-sdks-java-io-parent + beam-sdks-java-io-hadoop-parent 0.7.0-SNAPSHOT ../pom.xml - beam-sdks-java-io-hadoop-input-format - Apache Beam :: SDKs :: Java :: IO :: Hadoop Input Format - IO library to read data from data sources which implement Hadoop Input Format from Beam. - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - org.apache.maven.plugins - maven-surefire-plugin - - - org.apache.maven.plugins - maven-jar-plugin - - - + Apache Beam :: SDKs :: Java :: IO :: Hadoop :: input-format + IO to read data from data sources which implement Hadoop Input Format. 2.6.2 2.7.0 - 3.0.1 - 1.7.14 19.0 - - org.apache.beam - beam-sdks-java-core - com.google.guava guava @@ -68,16 +42,10 @@ org.slf4j slf4j-api - ${slf4j.api.version} com.google.code.findbugs jsr305 - ${findbugs.jsr305.version} - - - org.apache.beam - beam-sdks-java-io-hadoop-common @@ -86,6 +54,10 @@ auto-value provided + + org.apache.beam + beam-sdks-java-io-hadoop-common + org.apache.hadoop hadoop-common @@ -122,15 +94,5 @@ hamcrest-all test - - junit - junit - test - - - org.mockito - mockito-all - test - - + \ No newline at end of file diff --git a/sdks/java/io/hadoop-input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java b/sdks/java/io/hadoop/input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java similarity index 84% rename from sdks/java/io/hadoop-input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java rename to sdks/java/io/hadoop/input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java index 3b786fbdaba50..675f4bffa4782 100644 --- a/sdks/java/io/hadoop-input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java +++ b/sdks/java/io/hadoop/input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java @@ -31,8 +31,6 @@ import java.io.ObjectOutput; import java.io.ObjectOutputStream; import java.io.Serializable; -import java.lang.reflect.ParameterizedType; -import java.lang.reflect.Type; import java.math.BigDecimal; import java.math.BigInteger; import java.util.ArrayList; @@ -42,6 +40,7 @@ import java.util.List; import java.util.Map.Entry; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; import javax.annotation.Nullable; @@ -115,6 +114,12 @@ * for both key and value. If you specify a translation, you will need to make sure the K or V of * the read transform match the output type of the translation. * + *

You will need to set appropriate InputFormat key and value class (i.e. "key.class" and + * "value.class") in Hadoop {@link Configuration}. If you set different InputFormat key or + * value class than InputFormat's actual key or value class then, it may result in an error like + * "unexpected extra bytes after decoding" while the decoding process of key/value object happens. + * Hence, it is important to set appropriate InputFormat key and value class. + * *

Reading using {@link HadoopInputFormatIO}

* *
@@ -343,7 +348,6 @@ private void validateTranslationFunction(TypeDescriptor inputType,
      * coder, if not found in Coder Registry, then check if the type descriptor provided is of type
      * Writable, then WritableCoder is returned, else exception is thrown "Cannot find coder".
      */
-    @VisibleForTesting
     public  Coder getDefaultCoder(TypeDescriptor typeDesc, CoderRegistry coderRegistry) {
       Class classType = typeDesc.getRawType();
       try {
@@ -389,6 +393,19 @@ public static class HadoopInputFormatBoundedSource extends BoundedSource inputFormatObj;
     private transient TaskAttemptContext taskAttemptContext;
+    private static final Set> immutableTypes = new HashSet>(
+        Arrays.asList(
+            String.class,
+            Byte.class,
+            Short.class,
+            Integer.class,
+            Long.class,
+            Float.class,
+            Double.class,
+            Boolean.class,
+            BigInteger.class,
+            BigDecimal.class));
+
     HadoopInputFormatBoundedSource(
         SerializableConfiguration conf,
         Coder keyCoder,
@@ -493,7 +510,6 @@ void computeSplitsIfNecessary() throws IOException, InterruptedException {
         boundedSourceEstimatedSize += inputSplit.getLength();
         inputSplits.add(new SerializableSplit(inputSplit));
       }
-      validateUserInputForKeyAndValue();
     }
 
     /**
@@ -527,113 +543,6 @@ protected void createInputFormatInstance() throws IOException {
       }
     }
 
-    /**
-     * Throws exception if you set different InputFormat key or value class than InputFormat's
-     * actual key or value class. If you set incorrect classes then, it may result in an error like
-     * "unexpected extra bytes after decoding" while the decoding process happens. Hence this
-     * validation is required.
-     */
-    private void validateUserInputForKeyAndValue() throws IOException, InterruptedException {
-      ParameterizedType genericClassType = determineGenericType();
-      RecordReader reader = fetchFirstRecordReader();
-      boolean isCorrectKeyClassSet =
-          validateClass(genericClassType.getActualTypeArguments()[0].getTypeName(), keyCoder,
-              reader.getCurrentKey(), "key.class");
-      boolean isCorrectValueClassSet =
-          validateClass(genericClassType.getActualTypeArguments()[1].getTypeName(), valueCoder,
-              reader.getCurrentValue(), "value.class");
-      if (!isCorrectKeyClassSet) {
-        Class actualClass = conf.getHadoopConfiguration().getClass("key.class", Object.class);
-        throw new IllegalArgumentException(String.format(
-            "Wrong InputFormat key class in configuration : Expected key.class is %s but was %s.",
-            reader.getCurrentKey().getClass().getName(), actualClass.getName()));
-      }
-      if (!isCorrectValueClassSet) {
-        Class actualClass = conf.getHadoopConfiguration().getClass("value.class", Object.class);
-        throw new IllegalArgumentException(String.format("Wrong InputFormat value class in "
-            + "configuration : Expected value.class is %s but was %s.", reader.getCurrentValue()
-            .getClass().getName(), actualClass.getName()));
-      }
-    }
-
-    /**
-     * Returns true if key/value class set by the user is compatible with the key/value class of a
-     * pair returned by RecordReader. User provided key/value class is validated against the
-     * parameterized type's type arguments of InputFormat. If parameterized type has any type
-     * arguments such as T, K, V, etc then validation is done by encoding and decoding key/value
-     * object of first pair returned by RecordReader.
-     */
-    private  boolean validateClass(String inputFormatGenericClassName, Coder coder,
-        Object object, String property) {
-      try {
-        Class inputClass = Class.forName(inputFormatGenericClassName);
-        /*
-         * Validates key/value class with InputFormat's parameterized type.
-         */
-        if (property.equals("key.class")) {
-          return (conf.getHadoopConfiguration().getClass("key.class",
-              Object.class)).isAssignableFrom(inputClass);
-        }
-        return (conf.getHadoopConfiguration().getClass("value.class",
-            Object.class)).isAssignableFrom(inputClass);
-      } catch (ClassNotFoundException e) {
-        /*
-         * Given inputFormatGenericClassName is a type parameter i.e. T, K, V, etc. In such cases
-         * class validation for user provided input key/value will not work correctly. Therefore
-         * the need to validate key/value class by encoding and decoding key/value object with
-         * the given coder.
-         */
-        return checkEncodingAndDecoding((Coder) coder, (T) object);
-      }
-    }
-
-    /**
-     * Validates whether the input gets encoded or decoded correctly using the provided coder.
-     */
-    private  boolean checkEncodingAndDecoding(Coder coder, T input) {
-      try {
-        CoderUtils.clone(coder, input);
-      } catch (CoderException e) {
-        return false;
-      }
-      return true;
-    }
-
-    /**
-     * Returns parameterized type of the InputFormat class.
-     */
-    private ParameterizedType determineGenericType() {
-      // Any InputFormatClass always inherits from InputFormat which is a ParameterizedType.
-      // Hence, we can fetch generic super class of inputFormatClass which is a ParameterizedType.
-      Class inputFormatClass = inputFormatObj.getClass();
-      Type genericSuperclass = null;
-      for (;;) {
-        genericSuperclass = inputFormatClass.getGenericSuperclass();
-        if (genericSuperclass instanceof ParameterizedType) {
-          break;
-        }
-        inputFormatClass = inputFormatClass.getSuperclass();
-      }
-      return (ParameterizedType) genericSuperclass;
-    }
-
-    /**
-     * Returns RecordReader object of the first split to read first record for validating key/value
-     * classes.
-     */
-    private RecordReader fetchFirstRecordReader() throws IOException, InterruptedException {
-      RecordReader reader =
-          inputFormatObj.createRecordReader(inputSplits.get(0).getSplit(), taskAttemptContext);
-      if (reader == null) {
-        throw new IOException(String.format("Null RecordReader object returned by %s",
-            inputFormatObj.getClass()));
-      }
-      reader.initialize(inputSplits.get(0).getSplit(), taskAttemptContext);
-      // First record is read to get the InputFormat's key and value classes.
-      reader.nextKeyValue();
-      return reader;
-    }
-
     @VisibleForTesting
     InputFormat getInputFormat(){
       return inputFormatObj;
@@ -680,7 +589,7 @@ class HadoopInputFormatReader extends BoundedSource.BoundedReader recordReader;
       private volatile boolean doneReading = false;
-      private volatile long recordsReturned = 0L;
+      private AtomicLong recordsReturned = new AtomicLong();
       // Tracks the progress of the RecordReader.
       private AtomicDouble progressValue = new AtomicDouble();
       private transient InputFormat inputFormatObj;
@@ -708,7 +617,7 @@ public HadoopInputFormatBoundedSource getCurrentSource() {
       @Override
       public boolean start() throws IOException {
         try {
-          recordsReturned = 0;
+          recordsReturned.set(0L);
           recordReader =
               (RecordReader) inputFormatObj.createRecordReader(split.getSplit(),
                   taskAttemptContext);
@@ -716,7 +625,7 @@ public boolean start() throws IOException {
             recordReader.initialize(split.getSplit(), taskAttemptContext);
             progressValue.set(getProgress());
             if (recordReader.nextKeyValue()) {
-              recordsReturned++;
+              recordsReturned.incrementAndGet();
               doneReading = false;
               return true;
             }
@@ -740,7 +649,7 @@ public boolean advance() throws IOException {
         try {
           progressValue.set(getProgress());
           if (recordReader.nextKeyValue()) {
-            recordsReturned++;
+            recordsReturned.incrementAndGet();
             return true;
           }
           doneReading = true;
@@ -804,18 +713,6 @@ private  T cloneIfPossiblyMutable(T input, Coder coder) throws CoderExcept
        * Utility method to check if the passed object is of a known immutable type.
        */
       private boolean isKnownImmutable(Object o) {
-        Set> immutableTypes = new HashSet>(
-            Arrays.asList(
-                String.class,
-                Byte.class,
-                Short.class,
-                Integer.class,
-                Long.class,
-                Float.class,
-                Double.class,
-                Boolean.class,
-                BigInteger.class,
-                BigDecimal.class));
         return immutableTypes.contains(o.getClass());
       }
 
@@ -831,9 +728,12 @@ public void close() throws IOException {
       @Override
       public Double getFractionConsumed() {
         if (doneReading) {
-          progressValue.set(1.0);
-        } else if (recordReader == null || recordsReturned == 0) {
-          progressValue.set(0.0);
+          return 1.0;
+        } else if (recordReader == null || recordsReturned.get() == 0L) {
+          return 0.0;
+        }
+        if (progressValue.get() == 0.0) {
+          return null;
         }
         return progressValue.doubleValue();
       }
@@ -845,7 +745,8 @@ public Double getFractionConsumed() {
        */
       private Double getProgress() throws IOException, InterruptedException {
         try {
-          return (double) recordReader.getProgress();
+          float progress = recordReader.getProgress();
+          return (double) progress < 0 || progress > 1 ? 0.0 : progress;
         } catch (IOException e) {
           LOG.error(
               "Error in computing the fractions consumed as RecordReader.getProgress() throws an "
diff --git a/sdks/java/io/hadoop-input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/package-info.java b/sdks/java/io/hadoop/input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/package-info.java
similarity index 100%
rename from sdks/java/io/hadoop-input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/package-info.java
rename to sdks/java/io/hadoop/input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/package-info.java
diff --git a/sdks/java/io/hadoop-input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ConfigurableEmployeeInputFormat.java b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ConfigurableEmployeeInputFormat.java
similarity index 100%
rename from sdks/java/io/hadoop-input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ConfigurableEmployeeInputFormat.java
rename to sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ConfigurableEmployeeInputFormat.java
diff --git a/sdks/java/io/hadoop-input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/Employee.java b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/Employee.java
similarity index 100%
rename from sdks/java/io/hadoop-input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/Employee.java
rename to sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/Employee.java
diff --git a/sdks/java/io/hadoop-input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/EmployeeInputFormat.java b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/EmployeeInputFormat.java
similarity index 100%
rename from sdks/java/io/hadoop-input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/EmployeeInputFormat.java
rename to sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/EmployeeInputFormat.java
diff --git a/sdks/java/io/hadoop-input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIOTest.java b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIOTest.java
similarity index 94%
rename from sdks/java/io/hadoop-input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIOTest.java
rename to sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIOTest.java
index c25cf51321fa5..2f2857b335a71 100644
--- a/sdks/java/io/hadoop-input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIOTest.java
+++ b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIOTest.java
@@ -25,7 +25,6 @@
 import java.util.List;
 import java.util.Map.Entry;
 
-import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.io.BoundedSource;
@@ -46,7 +45,6 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.MapWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputFormat;
@@ -406,51 +404,6 @@ public String apply(LongWritable input) {
     read.validate(input);
   }
 
-  /**
-   * This test validates reading from Hadoop InputFormat if wrong key class is set in
-   * configuration.
-   */
-  @Test
-  public void testReadFailsWithWrongKeyClass() {
-    SerializableConfiguration wrongConf = loadTestConfiguration(
-       EmployeeInputFormat.class,
-       MapWritable.class, // Actual key class is Text.class.
-       Employee.class);
-    HadoopInputFormatIO.Read read = HadoopInputFormatIO.read()
-        .withConfiguration(wrongConf.getHadoopConfiguration());
-    String expectedMessage =
-        String.format("java.lang.IllegalArgumentException: " + "Wrong InputFormat key class in "
-            + "configuration : Expected key.class is %s but was %s.", Text.class.getName(),
-            MapWritable.class.getName());
-    thrown.expect(PipelineExecutionException.class);
-    thrown.expectMessage(expectedMessage);
-    p.apply("ReadTest", read);
-    p.run();
-  }
-
-  /**
-   * This test validates reading from Hadoop InputFormat if wrong value class is set in
-   * configuration.
-   */
-  @Test
-  public void testReadFailsWithWrongValueClass() {
-    SerializableConfiguration wrongConf = loadTestConfiguration(
-       EmployeeInputFormat.class,
-       Text.class,
-       MapWritable.class); // Actual value class is Employee.class.
-    HadoopInputFormatIO.Read read = HadoopInputFormatIO.read()
-        .withConfiguration(wrongConf.getHadoopConfiguration());
-    String expectedMessage =
-        String.format("java.lang.IllegalArgumentException: "
-            + "Wrong InputFormat value class in configuration : "
-            + "Expected value.class is %s but was %s.", Employee.class.getName(),
-            MapWritable.class.getName());
-    thrown.expect(PipelineExecutionException.class);
-    thrown.expectMessage(expectedMessage);
-    p.apply("ReadTest", read);
-    p.run();
-  }
-
   @Test
   public void testReadingData() throws Exception {
     HadoopInputFormatIO.Read read = HadoopInputFormatIO.read()
diff --git a/sdks/java/io/hadoop-input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ReuseObjectsEmployeeInputFormat.java b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ReuseObjectsEmployeeInputFormat.java
similarity index 100%
rename from sdks/java/io/hadoop-input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ReuseObjectsEmployeeInputFormat.java
rename to sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ReuseObjectsEmployeeInputFormat.java
diff --git a/sdks/java/io/hadoop-input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/TestEmployeeDataSet.java b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/TestEmployeeDataSet.java
similarity index 100%
rename from sdks/java/io/hadoop-input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/TestEmployeeDataSet.java
rename to sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/TestEmployeeDataSet.java
diff --git a/sdks/java/io/hadoop/jdk1.8-tests/pom.xml b/sdks/java/io/hadoop/jdk1.8-tests/pom.xml
new file mode 100644
index 0000000000000..4c510aeae2ac8
--- /dev/null
+++ b/sdks/java/io/hadoop/jdk1.8-tests/pom.xml
@@ -0,0 +1,278 @@
+
+
+
+
+
+
+  4.0.0
+  
+    org.apache.beam
+    beam-sdks-java-io-hadoop-parent
+    0.7.0-SNAPSHOT
+    ../pom.xml
+  
+  beam-sdks-java-io-hadoop-jdk1.8-tests
+  Apache Beam :: SDKs :: Java :: IO :: Hadoop :: jdk1.8-tests
+  Integration tests and junits which need JDK1.8.
+
+  
+    
+      
+       
+        org.apache.maven.plugins
+        maven-shade-plugin
+        
+          
+            package
+            
+              shade
+            
+            
+              
+                
+                  com.google.guava:guava:19.0
+                
+              
+              
+                
+                  com.google.common
+                  org.apache.beam.sdk.io.hadoop.jdk1.8-tests.repackaged.com.google.common
+                
+               
+                 com.google.thirdparty
+                 org.apache.beam.sdk.io.hadoop.jdk1.8-tests.repackaged.com.google.thirdparty
+                 
+               
+               
+                 
+               
+            
+          
+        
+      
+      
+      
+        org.apache.maven.plugins
+        maven-enforcer-plugin
+        1.4.1
+        
+          
+            enforce
+            
+              enforce
+            
+            
+              
+                
+                  1.8
+                  
+                    
+                    jdk.tools:jdk.tools
+                  
+                
+                
+                  [1.8,)
+                
+              
+            
+          
+        
+        
+          
+            org.codehaus.mojo
+            extra-enforcer-rules
+            1.0-beta-6
+          
+        
+      
+    
+  
+  
+
+  
+    
+    
+      spark-runner
+      
+        
+          org.apache.beam
+          beam-runners-spark
+          runtime
+        
+        
+          org.apache.spark
+          spark-streaming_2.10
+          ${spark.version}
+          runtime
+        
+        
+          org.apache.spark
+          spark-core_2.10
+          ${spark.version}
+          runtime
+          
+            
+              org.slf4j
+              jul-to-slf4j
+            
+          
+        
+      
+    
+
+    
+    
+      dataflow-runner
+      
+        
+          org.apache.beam
+          beam-runners-google-cloud-dataflow-java
+          runtime
+        
+      
+    
+  
+
+  
+    2.6.2
+    2.7.0
+    19.0
+    5.0.0
+    4.1.0.CR3
+    5.0.0
+    3.1.1
+    3.9
+    3.1.1
+    2.4
+  
+
+  
+    
+      org.apache.beam
+      beam-sdks-java-io-hadoop-input-format
+    
+    
+      com.google.guava
+      guava
+      ${guava.version}
+    
+    
+      org.slf4j
+      slf4j-api
+    
+
+    
+    
+      org.apache.hadoop
+      hadoop-common
+      ${hadoop.common.version}
+      provided
+    
+    
+      org.apache.hadoop
+      hadoop-mapreduce-client-core
+      ${hadoop.common.version}
+      provided
+    
+
+    
+    
+      org.apache.beam
+      beam-runners-direct-java
+      test
+    
+    
+      org.apache.logging.log4j
+      log4j-core
+      2.6.2
+      test
+    
+    
+      org.elasticsearch.plugin
+      transport-netty4-client
+      ${transport.netty4.client.version}
+      test
+    
+    
+      org.elasticsearch.client
+      transport
+      ${elasticsearch.version}
+      test
+    
+    
+      io.netty
+      netty-transport-native-epoll
+      ${netty.transport.native.epoll.version}
+      test
+    
+    
+      org.elasticsearch
+      elasticsearch
+      ${elasticsearch.version}
+      test
+    
+    
+      org.elasticsearch
+      elasticsearch-hadoop
+      ${elasticsearch.version}
+    
+    
+      com.datastax.cassandra
+      cassandra-driver-mapping
+      ${cassandra.driver.mapping.version}
+      test
+    
+    
+      org.apache.cassandra
+      cassandra-all
+      ${cassandra.all.verison}
+      test
+    
+    
+      org.hamcrest
+      hamcrest-all
+      test
+    
+    
+      com.datastax.cassandra
+      cassandra-driver-core
+      ${cassandra.driver.core.version}
+      test
+    
+
+    
+    
+      commons-io
+      commons-io
+      ${commons.io.version}
+      runtime
+    
+  
+
+
\ No newline at end of file
diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HIFIOWithElasticTest.java b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HIFIOWithElasticTest.java
new file mode 100644
index 0000000000000..599a4a10f5ac3
--- /dev/null
+++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HIFIOWithElasticTest.java
@@ -0,0 +1,277 @@
+/*
+ * 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.hadoop.inputformat;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.beam.sdk.io.hadoop.inputformat.hashing.HashingFn;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
+import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.hadoop.cfg.ConfigurationOptions;
+import org.elasticsearch.hadoop.mr.EsInputFormat;
+import org.elasticsearch.hadoop.mr.LinkedMapWritable;
+import org.elasticsearch.node.Node;
+import org.elasticsearch.node.NodeValidationException;
+import org.elasticsearch.node.internal.InternalSettingsPreparer;
+import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.transport.Netty4Plugin;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests to validate HadoopInputFormatIO for embedded Elasticsearch instance.
+ *
+ * {@link EsInputFormat} can be used to read data from Elasticsearch. EsInputFormat by default
+ * returns key class as Text and value class as LinkedMapWritable. You can also set MapWritable as
+ * value class, provided that you set the property "mapred.mapoutput.value.class" with
+ * MapWritable.class. If this property is not set then, using MapWritable as value class may give
+ * org.apache.beam.sdk.coders.CoderException due to unexpected extra bytes after decoding.
+ */
+
+@RunWith(JUnit4.class)
+public class HIFIOWithElasticTest implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOGGER = LoggerFactory.getLogger(HIFIOWithElasticTest.class);
+  private static final String ELASTIC_IN_MEM_HOSTNAME = "127.0.0.1";
+  private static final String ELASTIC_IN_MEM_PORT = "9200";
+  private static final String ELASTIC_INTERNAL_VERSION = "5.x";
+  private static final String TRUE = "true";
+  private static final String ELASTIC_INDEX_NAME = "beamdb";
+  private static final String ELASTIC_TYPE_NAME = "scientists";
+  private static final String ELASTIC_RESOURCE = "/" + ELASTIC_INDEX_NAME + "/" + ELASTIC_TYPE_NAME;
+  private static final int TEST_DATA_ROW_COUNT = 10;
+  private static final String ELASTIC_TYPE_ID_PREFIX = "s";
+
+  @ClassRule
+  public static TemporaryFolder elasticTempFolder = new TemporaryFolder();
+
+  @Rule
+  public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @BeforeClass
+  public static void startServer()
+      throws NodeValidationException, InterruptedException, IOException {
+    ElasticEmbeddedServer.startElasticEmbeddedServer();
+  }
+
+  /**
+   * Test to read data from embedded Elasticsearch instance and verify whether data is read
+   * successfully.
+   */
+  @Test
+  public void testHifIOWithElastic() {
+    // Expected hashcode is evaluated during insertion time one time and hardcoded here.
+    String expectedHashCode = "e2098f431f90193aa4545e033e6fd2217aafe7b6";
+    Configuration conf = getConfiguration();
+    PCollection> esData =
+        pipeline.apply(HadoopInputFormatIO.read().withConfiguration(conf));
+    PCollection count = esData.apply(Count.>globally());
+    // Verify that the count of objects fetched using HIFInputFormat IO is correct.
+    PAssert.thatSingleton(count).isEqualTo((long) TEST_DATA_ROW_COUNT);
+    PCollection values = esData.apply(Values.create());
+    PCollection textValues = values.apply(transformFunc);
+    // Verify the output values using checksum comparison.
+    PCollection consolidatedHashcode =
+        textValues.apply(Combine.globally(new HashingFn()).withoutDefaults());
+    PAssert.that(consolidatedHashcode).containsInAnyOrder(expectedHashCode);
+    pipeline.run().waitUntilFinish();
+  }
+
+  MapElements transformFunc =
+      MapElements.via(new SimpleFunction() {
+        @Override
+        public String apply(LinkedMapWritable mapw) {
+          return mapw.get(new Text("id")) + "|" + mapw.get(new Text("scientist"));
+        }
+      });
+  /**
+   * Test to read data from embedded Elasticsearch instance based on query and verify whether data
+   * is read successfully.
+   */
+  @Test
+  public void testHifIOWithElasticQuery() {
+    long expectedRowCount = 1L;
+    String expectedHashCode = "caa37dbd8258e3a7f98932958c819a57aab044ec";
+    Configuration conf = getConfiguration();
+    String fieldValue = ELASTIC_TYPE_ID_PREFIX + "2";
+    String query = "{"
+                  + "  \"query\": {"
+                  + "  \"match\" : {"
+                  + "    \"id\" : {"
+                  + "      \"query\" : \"" + fieldValue + "\","
+                  + "      \"type\" : \"boolean\""
+                  + "    }"
+                  + "  }"
+                  + "  }"
+                  + "}";
+    conf.set(ConfigurationOptions.ES_QUERY, query);
+    PCollection> esData =
+        pipeline.apply(HadoopInputFormatIO.read().withConfiguration(conf));
+    PCollection count = esData.apply(Count.>globally());
+    // Verify that the count of objects fetched using HIFInputFormat IO is correct.
+    PAssert.thatSingleton(count).isEqualTo(expectedRowCount);
+    PCollection values = esData.apply(Values.create());
+    PCollection textValues = values.apply(transformFunc);
+    // Verify the output values using checksum comparison.
+    PCollection consolidatedHashcode =
+        textValues.apply(Combine.globally(new HashingFn()).withoutDefaults());
+    PAssert.that(consolidatedHashcode).containsInAnyOrder(expectedHashCode);
+    pipeline.run().waitUntilFinish();
+  }
+
+  /**
+   * Set the Elasticsearch configuration parameters in the Hadoop configuration object.
+   * Configuration object should have InputFormat class, key class and value class set. Mandatory
+   * fields for ESInputFormat to be set are es.resource, es.nodes, es.port, es.internal.es.version.
+   * Please refer to
+   * Elasticsearch Configuration for more details.
+   */
+  public Configuration getConfiguration() {
+    Configuration conf = new Configuration();
+    conf.set(ConfigurationOptions.ES_NODES, ELASTIC_IN_MEM_HOSTNAME);
+    conf.set(ConfigurationOptions.ES_PORT, String.format("%s", ELASTIC_IN_MEM_PORT));
+    conf.set(ConfigurationOptions.ES_RESOURCE, ELASTIC_RESOURCE);
+    conf.set("es.internal.es.version", ELASTIC_INTERNAL_VERSION);
+    conf.set(ConfigurationOptions.ES_NODES_DISCOVERY, TRUE);
+    conf.set(ConfigurationOptions.ES_INDEX_AUTO_CREATE, TRUE);
+    conf.setClass("mapreduce.job.inputformat.class",
+        org.elasticsearch.hadoop.mr.EsInputFormat.class, InputFormat.class);
+    conf.setClass("key.class", Text.class, Object.class);
+    conf.setClass("value.class", LinkedMapWritable.class, Object.class);
+    return conf;
+ }
+
+  private static Map createElasticRow(String id, String name) {
+    Map data = new HashMap();
+    data.put("id", id);
+    data.put("scientist", name);
+    return data;
+  }
+
+  @AfterClass
+  public static void shutdownServer() throws IOException {
+    ElasticEmbeddedServer.shutdown();
+  }
+
+  /**
+   * Class for in memory Elasticsearch server.
+   */
+  static class ElasticEmbeddedServer implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private static Node node;
+
+    public static void startElasticEmbeddedServer()
+        throws NodeValidationException, InterruptedException {
+      Settings settings = Settings.builder()
+          .put("node.data", TRUE)
+          .put("network.host", ELASTIC_IN_MEM_HOSTNAME)
+          .put("http.port", ELASTIC_IN_MEM_PORT)
+          .put("path.data", elasticTempFolder.getRoot().getPath())
+          .put("path.home", elasticTempFolder.getRoot().getPath())
+          .put("transport.type", "local")
+          .put("http.enabled", TRUE)
+          .put("node.ingest", TRUE).build();
+      node = new PluginNode(settings);
+      node.start();
+      LOGGER.info("Elastic in memory server started.");
+      prepareElasticIndex();
+      LOGGER.info("Prepared index " + ELASTIC_INDEX_NAME
+          + "and populated data on elastic in memory server.");
+    }
+
+    /**
+     * Prepares Elastic index, by adding rows.
+     */
+    private static void prepareElasticIndex() throws InterruptedException {
+      CreateIndexRequest indexRequest = new CreateIndexRequest(ELASTIC_INDEX_NAME);
+      node.client().admin().indices().create(indexRequest).actionGet();
+      for (int i = 0; i < TEST_DATA_ROW_COUNT; i++) {
+        node.client().prepareIndex(ELASTIC_INDEX_NAME, ELASTIC_TYPE_NAME, String.valueOf(i))
+            .setSource(createElasticRow(ELASTIC_TYPE_ID_PREFIX + i, "Faraday" + i)).execute()
+            .actionGet();
+      }
+      node.client().admin().indices().prepareRefresh(ELASTIC_INDEX_NAME).get();
+    }
+    /**
+     * Shutdown the embedded instance.
+     * @throws IOException
+     */
+    public static void shutdown() throws IOException {
+      DeleteIndexRequest indexRequest = new DeleteIndexRequest(ELASTIC_INDEX_NAME);
+      node.client().admin().indices().delete(indexRequest).actionGet();
+      LOGGER.info("Deleted index " + ELASTIC_INDEX_NAME + " from elastic in memory server");
+      node.close();
+      LOGGER.info("Closed elastic in memory server node.");
+      deleteElasticDataDirectory();
+    }
+
+    private static void deleteElasticDataDirectory() {
+      try {
+        FileUtils.deleteDirectory(new File(elasticTempFolder.getRoot().getPath()));
+      } catch (IOException e) {
+        throw new RuntimeException("Could not delete elastic data directory: " + e.getMessage(), e);
+      }
+    }
+  }
+
+  /**
+   * Class created for handling "http.enabled" property as "true" for Elasticsearch node.
+   */
+  static class PluginNode extends Node implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+    static Collection> list = new ArrayList>();
+    static {
+      list.add(Netty4Plugin.class);
+    }
+
+    public PluginNode(final Settings settings) {
+      super(InternalSettingsPreparer.prepareEnvironment(settings, null), list);
+    }
+  }
+}
diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/custom/options/HIFTestOptions.java b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/custom/options/HIFTestOptions.java
new file mode 100644
index 0000000000000..2e89ed179e0b1
--- /dev/null
+++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/custom/options/HIFTestOptions.java
@@ -0,0 +1,64 @@
+/*
+ * 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.hadoop.inputformat.custom.options;
+
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+
+/**
+ * Properties needed when using HadoopInputFormatIO with the Beam SDK.
+ */
+public interface HIFTestOptions extends TestPipelineOptions {
+
+  //Cassandra test options
+  @Description("Cassandra Server IP")
+  @Default.String("cassandraServerIp")
+  String getCassandraServerIp();
+  void setCassandraServerIp(String cassandraServerIp);
+  @Description("Cassandra Server port")
+  @Default.Integer(0)
+  Integer getCassandraServerPort();
+  void setCassandraServerPort(Integer cassandraServerPort);
+  @Description("Cassandra User name")
+  @Default.String("cassandraUserName")
+  String getCassandraUserName();
+  void setCassandraUserName(String cassandraUserName);
+  @Description("Cassandra Password")
+  @Default.String("cassandraPassword")
+  String getCassandraPassword();
+  void setCassandraPassword(String cassandraPassword);
+
+  //Elasticsearch test options
+  @Description("Elasticsearch Server IP")
+  @Default.String("elasticServerIp")
+  String getElasticServerIp();
+  void setElasticServerIp(String elasticServerIp);
+  @Description("Elasticsearch Server port")
+  @Default.Integer(0)
+  Integer getElasticServerPort();
+  void setElasticServerPort(Integer elasticServerPort);
+  @Description("Elasticsearch User name")
+  @Default.String("elasticUserName")
+  String getElasticUserName();
+  void setElasticUserName(String elasticUserName);
+  @Description("Elastic Password")
+  @Default.String("elasticPassword")
+  String getElasticPassword();
+  void setElasticPassword(String elasticPassword);
+}
diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/hashing/HashingFn.java b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/hashing/HashingFn.java
new file mode 100644
index 0000000000000..fe370483000b8
--- /dev/null
+++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/hashing/HashingFn.java
@@ -0,0 +1,109 @@
+/*
+ * 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.hadoop.inputformat.hashing;
+
+import com.google.common.collect.Lists;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.Hashing;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+
+/**
+ * Custom Function for Hashing. The combiner is combineUnordered, and accumulator is a
+ * HashCode.
+ */
+public class HashingFn extends CombineFn {
+
+  /**
+   * Serializable Class to store the HashCode of input String.
+   */
+  public static class Accum implements Serializable {
+    HashCode hashCode = null;
+
+    public Accum(HashCode value) {
+      this.hashCode = value;
+    }
+
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+      in.defaultReadObject();
+    }
+
+    private void writeObject(ObjectOutputStream out) throws IOException {
+      out.defaultWriteObject();
+    }
+  }
+
+  @Override
+  public Accum addInput(Accum accum, String input) {
+    List elementHashes = Lists.newArrayList();
+     if (accum.hashCode != null) {
+      elementHashes.add(accum.hashCode);
+    }
+    HashCode inputHashCode = Hashing.sha1().hashString(input, StandardCharsets.UTF_8);
+    elementHashes.add(inputHashCode);
+    accum.hashCode = Hashing.combineUnordered(elementHashes);
+    return accum;
+  }
+
+  @Override
+  public Accum mergeAccumulators(Iterable accums) {
+    Accum merged = createAccumulator();
+    List elementHashes = Lists.newArrayList();
+    for (Accum accum : accums) {
+      if (accum.hashCode != null) {
+        elementHashes.add(accum.hashCode);
+      }
+    }
+    merged.hashCode = Hashing.combineUnordered(elementHashes);
+    return merged;
+  }
+
+  @Override
+  public String extractOutput(Accum accum) {
+    // Return the combined hash code of list of elements in the Pcollection.
+    String consolidatedHash = "";
+    if (accum.hashCode != null) {
+      consolidatedHash = accum.hashCode.toString();
+    }
+    return consolidatedHash;
+  }
+
+  @Override
+  public Coder getAccumulatorCoder(CoderRegistry registry, Coder inputCoder)
+      throws CannotProvideCoderException {
+    return SerializableCoder.of(Accum.class);
+  }
+
+  @Override
+  public Coder getDefaultOutputCoder(CoderRegistry registry, Coder inputCoder) {
+    return inputCoder;
+  }
+
+  @Override
+  public Accum createAccumulator() {
+    return new Accum(null);
+  }
+}
diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOCassandraIT.java b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOCassandraIT.java
new file mode 100644
index 0000000000000..bf9a5fd265043
--- /dev/null
+++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOCassandraIT.java
@@ -0,0 +1,173 @@
+/*
+ * 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.hadoop.inputformat.integration.tests;
+
+import com.datastax.driver.core.Row;
+
+import java.io.Serializable;
+
+import org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIO;
+import org.apache.beam.sdk.io.hadoop.inputformat.custom.options.HIFTestOptions;
+import org.apache.beam.sdk.io.hadoop.inputformat.hashing.HashingFn;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * A test of {@link org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIO} on an
+ * independent Cassandra instance.
+ *
+ * 

This test requires a running instance of Cassandra, and the test dataset must exist in + * the database. + * + *

You can run this test by doing the following: + *

+ *  mvn -e -Pio-it verify -pl sdks/java/io/hadoop/jdk1.8-tests/HIFIOCassandraIT
+ *  -DintegrationTestPipelineOptions='[
+ *  "--cassandraServerIp=1.2.3.4",
+ *  "--cassandraServerPort=port",
+ *  "--cassandraUserName=user",
+ *  "--cassandraPassword=mypass" ]'
+ * 
+ * + *

If you want to run this with a runner besides directrunner, there are profiles for dataflow + * and spark in the jdk1.8-tests pom. You'll want to activate those in addition to the normal test + * runner invocation pipeline options. + */ + +@RunWith(JUnit4.class) +public class HIFIOCassandraIT implements Serializable { + + private static final String CASSANDRA_KEYSPACE = "ycsb"; + private static final String CASSANDRA_TABLE = "usertable"; + private static final String CASSANDRA_THRIFT_PORT_PROPERTY = "cassandra.input.thrift.port"; + private static final String CASSANDRA_THRIFT_ADDRESS_PROPERTY = "cassandra.input.thrift.address"; + private static final String CASSANDRA_PARTITIONER_CLASS_PROPERTY = + "cassandra.input.partitioner.class"; + private static final String CASSANDRA_KEYSPACE_PROPERTY = "cassandra.input.keyspace"; + private static final String CASSANDRA_COLUMNFAMILY_PROPERTY = "cassandra.input.columnfamily"; + private static final String CASSANDRA_PARTITIONER_CLASS_VALUE = "Murmur3Partitioner"; + private static final String USERNAME = "cassandra.username"; + private static final String PASSWORD = "cassandra.password"; + private static final String INPUT_KEYSPACE_USERNAME_CONFIG = "cassandra.input.keyspace.username"; + private static final String INPUT_KEYSPACE_PASSWD_CONFIG = "cassandra.input.keyspace.passwd"; + private static HIFTestOptions options; + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + + @BeforeClass + public static void setUp() { + PipelineOptionsFactory.register(HIFTestOptions.class); + options = TestPipeline.testingPipelineOptions().as(HIFTestOptions.class); + } + + /** + * This test reads data from the Cassandra instance and verifies if data is read successfully. + */ + @Test + public void testHIFReadForCassandra() { + // Expected hashcode is evaluated during insertion time one time and hardcoded here. + String expectedHashCode = "5ea121d90d95c84076f7556605080f4b2c3081a7"; + Long expectedRecordsCount = 1000L; + Configuration conf = getConfiguration(options); + PCollection> cassandraData = pipeline.apply(HadoopInputFormatIO + .read().withConfiguration(conf).withValueTranslation(myValueTranslate)); + PAssert.thatSingleton(cassandraData.apply("Count", Count.>globally())) + .isEqualTo(expectedRecordsCount); + PCollection textValues = cassandraData.apply(Values.create()); + // Verify the output values using checksum comparison. + PCollection consolidatedHashcode = + textValues.apply(Combine.globally(new HashingFn()).withoutDefaults()); + PAssert.that(consolidatedHashcode).containsInAnyOrder(expectedHashCode); + pipeline.run().waitUntilFinish(); + } + + SimpleFunction myValueTranslate = new SimpleFunction() { + @Override + public String apply(Row input) { + return input.getString("y_id") + "|" + input.getString("field0") + "|" + + input.getString("field1") + "|" + input.getString("field2") + "|" + + input.getString("field3") + "|" + input.getString("field4") + "|" + + input.getString("field5") + "|" + input.getString("field6") + "|" + + input.getString("field7") + "|" + input.getString("field8") + "|" + + input.getString("field9"); + } + }; + /** + * This test reads data from the Cassandra instance based on query and verifies if data is read + * successfully. + */ + @Test + public void testHIFReadForCassandraQuery() { + String expectedHashCode = "a19593e4c72a67e26cb470130864daabf5a99d62"; + Long expectedNumRows = 1L; + Configuration conf = getConfiguration(options); + conf.set("cassandra.input.cql", "select * from " + CASSANDRA_KEYSPACE + "." + CASSANDRA_TABLE + + " where token(y_id) > ? and token(y_id) <= ? " + + "and field0 = 'user48:field0:431531' allow filtering"); + PCollection> cassandraData = + pipeline.apply(HadoopInputFormatIO.read().withConfiguration(conf) + .withValueTranslation(myValueTranslate)); + PAssert.thatSingleton(cassandraData.apply("Count", Count.>globally())) + .isEqualTo(expectedNumRows); + PCollection textValues = cassandraData.apply(Values.create()); + // Verify the output values using checksum comparison. + PCollection consolidatedHashcode = + textValues.apply(Combine.globally(new HashingFn()).withoutDefaults()); + PAssert.that(consolidatedHashcode).containsInAnyOrder(expectedHashCode); + pipeline.run().waitUntilFinish(); + } + + /** + * Returns Hadoop configuration for reading data from Cassandra. To read data from Cassandra using + * HadoopInputFormatIO, following properties must be set: InputFormat class, InputFormat key + * class, InputFormat value class, Thrift address, Thrift port, partitioner class, keyspace and + * columnfamily name. + */ + private static Configuration getConfiguration(HIFTestOptions options) { + Configuration conf = new Configuration(); + conf.set(CASSANDRA_THRIFT_PORT_PROPERTY, options.getCassandraServerPort().toString()); + conf.set(CASSANDRA_THRIFT_ADDRESS_PROPERTY, options.getCassandraServerIp()); + conf.set(CASSANDRA_PARTITIONER_CLASS_PROPERTY, CASSANDRA_PARTITIONER_CLASS_VALUE); + conf.set(CASSANDRA_KEYSPACE_PROPERTY, CASSANDRA_KEYSPACE); + conf.set(CASSANDRA_COLUMNFAMILY_PROPERTY, CASSANDRA_TABLE); + // Set user name and password if Cassandra instance has security configured. + conf.set(USERNAME, options.getCassandraUserName()); + conf.set(PASSWORD, options.getCassandraPassword()); + conf.set(INPUT_KEYSPACE_USERNAME_CONFIG, options.getCassandraUserName()); + conf.set(INPUT_KEYSPACE_PASSWD_CONFIG, options.getCassandraPassword()); + conf.setClass("mapreduce.job.inputformat.class", + org.apache.cassandra.hadoop.cql3.CqlInputFormat.class, InputFormat.class); + conf.setClass("key.class", java.lang.Long.class, Object.class); + conf.setClass("value.class", com.datastax.driver.core.Row.class, Object.class); + return conf; + } +} diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOElasticIT.java b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOElasticIT.java new file mode 100644 index 0000000000000..13c0cbc841cb2 --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOElasticIT.java @@ -0,0 +1,215 @@ +/* + * 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.hadoop.inputformat.integration.tests; + +import java.io.IOException; +import java.io.Serializable; + +import org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIO; +import org.apache.beam.sdk.io.hadoop.inputformat.custom.options.HIFTestOptions; +import org.apache.beam.sdk.io.hadoop.inputformat.hashing.HashingFn; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.MapWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.InputFormat; +import org.elasticsearch.hadoop.cfg.ConfigurationOptions; +import org.elasticsearch.hadoop.mr.LinkedMapWritable; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * A test of {@link org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIO} on an + * independent Elasticsearch instance. + * + *

This test requires a running instance of Elasticsearch, and the test dataset must exist in + * the database. + * + *

You can run this test by doing the following: + *

+ *  mvn -e -Pio-it verify -pl sdks/java/io/hadoop/jdk1.8-tests/HIFIOElasticIT
+ *  -DintegrationTestPipelineOptions='[
+ *  "--elasticServerIp=1.2.3.4",
+ *  "--elasticServerPort=port",
+ *  "--elasticUserName=user",
+ *  "--elasticPassword=mypass" ]'
+ * 
+ * + *

If you want to run this with a runner besides directrunner, there are profiles for dataflow + * and spark in the jdk1.8-tests pom. You'll want to activate those in addition to the normal test + * runner invocation pipeline options. + */ + +@RunWith(JUnit4.class) +public class HIFIOElasticIT implements Serializable { + + private static final String ELASTIC_INTERNAL_VERSION = "5.x"; + private static final String TRUE = "true"; + private static final String ELASTIC_INDEX_NAME = "test_data"; + private static final String ELASTIC_TYPE_NAME = "test_type"; + private static final String ELASTIC_RESOURCE = "/" + ELASTIC_INDEX_NAME + "/" + ELASTIC_TYPE_NAME; + private static HIFTestOptions options; + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + + @BeforeClass + public static void setUp() { + PipelineOptionsFactory.register(HIFTestOptions.class); + options = TestPipeline.testingPipelineOptions().as(HIFTestOptions.class); + } + + /** + * This test reads data from the Elasticsearch instance and verifies whether data is read + * successfully. + */ + @Test + public void testHifIOWithElastic() throws SecurityException, IOException { + // Expected hashcode is evaluated during insertion time one time and hardcoded here. + final long expectedRowCount = 1000L; + String expectedHashCode = "ed36c09b5e24a95fd8d3cc711a043a85320bb47d"; + Configuration conf = getConfiguration(options); + PCollection> esData = + pipeline.apply(HadoopInputFormatIO.read().withConfiguration(conf)); + // Verify that the count of objects fetched using HIFInputFormat IO is correct. + PCollection count = esData.apply(Count.>globally()); + PAssert.thatSingleton(count).isEqualTo(expectedRowCount); + PCollection values = esData.apply(Values.create()); + PCollection textValues = values.apply(transformFunc); + // Verify the output values using checksum comparison. + PCollection consolidatedHashcode = + textValues.apply(Combine.globally(new HashingFn()).withoutDefaults()); + PAssert.that(consolidatedHashcode).containsInAnyOrder(expectedHashCode); + pipeline.run().waitUntilFinish(); + } + + MapElements transformFunc = + MapElements.via(new SimpleFunction() { + @Override + public String apply(LinkedMapWritable mapw) { + String rowValue = ""; + rowValue = convertMapWRowToString(mapw); + return rowValue; + } + }); + /* + * Function to create a toString implementation of a MapWritable row by writing all field values + * in a string row. + */ + private String convertMapWRowToString(LinkedMapWritable mapw) { + String rowValue = ""; + rowValue = addFieldValuesToRow(rowValue, mapw, "User_Name"); + rowValue = addFieldValuesToRow(rowValue, mapw, "Item_Code"); + rowValue = addFieldValuesToRow(rowValue, mapw, "Txn_ID"); + rowValue = addFieldValuesToRow(rowValue, mapw, "Item_ID"); + rowValue = addFieldValuesToRow(rowValue, mapw, "last_updated"); + rowValue = addFieldValuesToRow(rowValue, mapw, "Price"); + rowValue = addFieldValuesToRow(rowValue, mapw, "Title"); + rowValue = addFieldValuesToRow(rowValue, mapw, "Description"); + rowValue = addFieldValuesToRow(rowValue, mapw, "Age"); + rowValue = addFieldValuesToRow(rowValue, mapw, "Item_Name"); + rowValue = addFieldValuesToRow(rowValue, mapw, "Item_Price"); + rowValue = addFieldValuesToRow(rowValue, mapw, "Availability"); + rowValue = addFieldValuesToRow(rowValue, mapw, "Batch_Num"); + rowValue = addFieldValuesToRow(rowValue, mapw, "Last_Ordered"); + rowValue = addFieldValuesToRow(rowValue, mapw, "City"); + return rowValue; + } + + /* + * Convert a MapWritable row field into a string, and append it to the row string with a + * separator. + */ + private String addFieldValuesToRow(String row, MapWritable mapw, String columnName) { + Object valueObj = (Object) mapw.get(new Text(columnName)); + row += valueObj.toString() + "|"; + return row; + } + + /** + * This test reads data from the Elasticsearch instance based on a query and verifies if data is + * read successfully. + */ + @Test + public void testHifIOWithElasticQuery() { + String expectedHashCode = "83c108ff81e87b6f3807c638e6bb9a9e3d430dc7"; + Long expectedRecordsCount = 1L; + Configuration conf = getConfiguration(options); + String query = "{" + + " \"query\": {" + + " \"match\" : {" + + " \"Title\" : {" + + " \"query\" : \"Title9\"," + + " \"type\" : \"boolean\"" + + " }" + + " }" + + " }" + + "}"; + conf.set(ConfigurationOptions.ES_QUERY, query); + PCollection> esData = + pipeline.apply(HadoopInputFormatIO.read().withConfiguration(conf)); + PCollection count = esData.apply(Count.>globally()); + // Verify that the count of objects fetched using HIFInputFormat IO is correct. + PAssert.thatSingleton(count).isEqualTo(expectedRecordsCount); + PCollection values = esData.apply(Values.create()); + PCollection textValues = values.apply(transformFunc); + // Verify the output values using checksum comparison. + PCollection consolidatedHashcode = + textValues.apply(Combine.globally(new HashingFn()).withoutDefaults()); + PAssert.that(consolidatedHashcode).containsInAnyOrder(expectedHashCode); + pipeline.run().waitUntilFinish(); + } + + /** + * Returns Hadoop configuration for reading data from Elasticsearch. Configuration object should + * have InputFormat class, key class and value class to be set. Mandatory fields for ESInputFormat + * to be set are es.resource, es.nodes, es.port, es.internal.es.version, es.nodes.wan.only. Please + * refer Elasticsearch Configuration for more details. + */ + private static Configuration getConfiguration(HIFTestOptions options) { + Configuration conf = new Configuration(); + conf.set(ConfigurationOptions.ES_NODES, options.getElasticServerIp()); + conf.set(ConfigurationOptions.ES_PORT, options.getElasticServerPort().toString()); + conf.set(ConfigurationOptions.ES_NODES_WAN_ONLY, TRUE); + // Set username and password if Elasticsearch is configured with security. + conf.set(ConfigurationOptions.ES_NET_HTTP_AUTH_USER, options.getElasticUserName()); + conf.set(ConfigurationOptions.ES_NET_HTTP_AUTH_PASS, options.getElasticPassword()); + conf.set(ConfigurationOptions.ES_RESOURCE, ELASTIC_RESOURCE); + conf.set("es.internal.es.version", ELASTIC_INTERNAL_VERSION); + conf.set(ConfigurationOptions.ES_INDEX_AUTO_CREATE, TRUE); + conf.setClass("mapreduce.job.inputformat.class", + org.elasticsearch.hadoop.mr.EsInputFormat.class, InputFormat.class); + conf.setClass("key.class", Text.class, Object.class); + conf.setClass("value.class", LinkedMapWritable.class, Object.class); + // Optimizations added to change the max docs per partition, scroll size and batch size of + // bytes to improve the test time for large data + conf.set("es.input.max.docs.per.partition", "50000"); + conf.set("es.scroll.size", "400"); + conf.set("es.batch.size.bytes", "8mb"); + return conf; + } +} diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/cassandra.yaml b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/cassandra.yaml new file mode 100644 index 0000000000000..ca1e48fd8eed2 --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/cassandra.yaml @@ -0,0 +1,1074 @@ +# +# 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. +# + +# Cassandra storage config YAML required for Embedded Cassandra server test + +# NOTE: +# See http://wiki.apache.org/cassandra/StorageConfiguration for +# full explanations of configuration directives +# /NOTE + +# The name of the cluster. This is mainly used to prevent machines in +# one logical cluster from joining another. +cluster_name: 'beam' + +# This defines the number of tokens randomly assigned to this node on the ring +# The more tokens, relative to other nodes, the larger the proportion of data +# that this node will store. You probably want all nodes to have the same number +# of tokens assuming they have equal hardware capability. +# +# If you leave this unspecified, Cassandra will use the default of 1 token for legacy compatibility, +# and will use the initial_token as described below. +# +# Specifying initial_token will override this setting on the node's initial start, +# on subsequent starts, this setting will apply even if initial token is set. +# +# If you already have a cluster with 1 token per node, and wish to migrate to +# multiple tokens per node, see http://wiki.apache.org/cassandra/Operations +num_tokens: 1 + +# Triggers automatic allocation of num_tokens tokens for this node. The allocation +# algorithm attempts to choose tokens in a way that optimizes replicated load over +# the nodes in the datacenter for the replication strategy used by the specified +# keyspace. +# +# The load assigned to each node will be close to proportional to its number of +# vnodes. +# +# Only supported with the Murmur3Partitioner. +# allocate_tokens_for_keyspace: KEYSPACE + +# initial_token allows you to specify tokens manually. While you can use # it with +# vnodes (num_tokens > 1, above) -- in which case you should provide a +# comma-separated list -- it's primarily used when adding nodes # to legacy clusters +# that do not have vnodes enabled. +# initial_token: + +# See http://wiki.apache.org/cassandra/HintedHandoff +# May either be "true" or "false" to enable globally +hinted_handoff_enabled: true +# When hinted_handoff_enabled is true, a black list of data centers that will not +# perform hinted handoff +#hinted_handoff_disabled_datacenters: +# - DC1 +# - DC2 +# this defines the maximum amount of time a dead host will have hints +# generated. After it has been dead this long, new hints for it will not be +# created until it has been seen alive and gone down again. +max_hint_window_in_ms: 10800000 # 3 hours + +# Maximum throttle in KBs per second, per delivery thread. This will be +# reduced proportionally to the number of nodes in the cluster. (If there +# are two nodes in the cluster, each delivery thread will use the maximum +# rate; if there are three, each will throttle to half of the maximum, +# since we expect two nodes to be delivering hints simultaneously.) +hinted_handoff_throttle_in_kb: 1024 + +# Number of threads with which to deliver hints; +# Consider increasing this number when you have multi-dc deployments, since +# cross-dc handoff tends to be slower +max_hints_delivery_threads: 2 + +# Directory where Cassandra should store hints. +# If not set, the default directory is $CASSANDRA_HOME/data/hints. +# hints_directory: /var/lib/cassandra/hints +hints_directory: target/cassandra/hints + +# How often hints should be flushed from the internal buffers to disk. +# Will *not* trigger fsync. +hints_flush_period_in_ms: 10000 + +# Maximum size for a single hints file, in megabytes. +max_hints_file_size_in_mb: 128 + +# Compression to apply to the hint files. If omitted, hints files +# will be written uncompressed. LZ4, Snappy, and Deflate compressors +# are supported. +#hints_compression: +# - class_name: LZ4Compressor +# parameters: +# - + +# Maximum throttle in KBs per second, total. This will be +# reduced proportionally to the number of nodes in the cluster. +batchlog_replay_throttle_in_kb: 1024 + +# Authentication backend, implementing IAuthenticator; used to identify users +# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthenticator, +# PasswordAuthenticator}. +# +# - AllowAllAuthenticator performs no checks - set it to disable authentication. +# - PasswordAuthenticator relies on username/password pairs to authenticate +# users. It keeps usernames and hashed passwords in system_auth.credentials table. +# Please increase system_auth keyspace replication factor if you use this authenticator. +# If using PasswordAuthenticator, CassandraRoleManager must also be used (see below) +authenticator: AllowAllAuthenticator + +# Authorization backend, implementing IAuthorizer; used to limit access/provide permissions +# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthorizer, +# CassandraAuthorizer}. +# +# - AllowAllAuthorizer allows any action to any user - set it to disable authorization. +# - CassandraAuthorizer stores permissions in system_auth.permissions table. Please +# increase system_auth keyspace replication factor if you use this authorizer. +authorizer: AllowAllAuthorizer + +# Part of the Authentication & Authorization backend, implementing IRoleManager; used +# to maintain grants and memberships between roles. +# Out of the box, Cassandra provides org.apache.cassandra.auth.CassandraRoleManager, +# which stores role information in the system_auth keyspace. Most functions of the +# IRoleManager require an authenticated login, so unless the configured IAuthenticator +# actually implements authentication, most of this functionality will be unavailable. +# +# - CassandraRoleManager stores role data in the system_auth keyspace. Please +# increase system_auth keyspace replication factor if you use this role manager. +role_manager: CassandraRoleManager + +# Validity period for roles cache (fetching granted roles can be an expensive +# operation depending on the role manager, CassandraRoleManager is one example) +# Granted roles are cached for authenticated sessions in AuthenticatedUser and +# after the period specified here, become eligible for (async) reload. +# Defaults to 2000, set to 0 to disable caching entirely. +# Will be disabled automatically for AllowAllAuthenticator. +roles_validity_in_ms: 2000 + +# Refresh interval for roles cache (if enabled). +# After this interval, cache entries become eligible for refresh. Upon next +# access, an async reload is scheduled and the old value returned until it +# completes. If roles_validity_in_ms is non-zero, then this must be +# also. +# Defaults to the same value as roles_validity_in_ms. +# roles_update_interval_in_ms: 2000 + +# Validity period for permissions cache (fetching permissions can be an +# expensive operation depending on the authorizer, CassandraAuthorizer is +# one example). Defaults to 2000, set to 0 to disable. +# Will be disabled automatically for AllowAllAuthorizer. +permissions_validity_in_ms: 2000 + +# Refresh interval for permissions cache (if enabled). +# After this interval, cache entries become eligible for refresh. Upon next +# access, an async reload is scheduled and the old value returned until it +# completes. If permissions_validity_in_ms is non-zero, then this must be +# also. +# Defaults to the same value as permissions_validity_in_ms. +# permissions_update_interval_in_ms: 2000 + +# Validity period for credentials cache. This cache is tightly coupled to +# the provided PasswordAuthenticator implementation of IAuthenticator. If +# another IAuthenticator implementation is configured, this cache will not +# be automatically used and so the following settings will have no effect. +# Please note, credentials are cached in their encrypted form, so while +# activating this cache may reduce the number of queries made to the +# underlying table, it may not bring a significant reduction in the +# latency of individual authentication attempts. +# Defaults to 2000, set to 0 to disable credentials caching. +credentials_validity_in_ms: 2000 + +# Refresh interval for credentials cache (if enabled). +# After this interval, cache entries become eligible for refresh. Upon next +# access, an async reload is scheduled and the old value returned until it +# completes. If credentials_validity_in_ms is non-zero, then this must be +# also. +# Defaults to the same value as credentials_validity_in_ms. +# credentials_update_interval_in_ms: 2000 + +# The partitioner is responsible for distributing groups of rows (by +# partition key) across nodes in the cluster. You should leave this +# alone for new clusters. The partitioner can NOT be changed without +# reloading all data, so when upgrading you should set this to the +# same partitioner you were already using. +# +# Besides Murmur3Partitioner, partitioners included for backwards +# compatibility include RandomPartitioner, ByteOrderedPartitioner, and +# OrderPreservingPartitioner. +# +partitioner: org.apache.cassandra.dht.Murmur3Partitioner + +# Directories where Cassandra should store data on disk. Cassandra +# will spread data evenly across them, subject to the granularity of +# the configured compaction strategy. +# If not set, the default directory is $CASSANDRA_HOME/data/data. +# data_file_directories: +# - /var/lib/cassandra/data +data_file_directories: + - target/cassandra/data + +# commit log. when running on magnetic HDD, this should be a +# separate spindle than the data directories. +# If not set, the default directory is $CASSANDRA_HOME/data/commitlog. +# commitlog_directory: /var/lib/cassandra/commitlog +commitlog_directory: target/cassandra/commitlog +cdc_raw_directory: target/cassandra/cdc_raw +# policy for data disk failures: +# die: shut down gossip and client transports and kill the JVM for any fs errors or +# single-sstable errors, so the node can be replaced. +# stop_paranoid: shut down gossip and client transports even for single-sstable errors, +# kill the JVM for errors during startup. +# stop: shut down gossip and client transports, leaving the node effectively dead, but +# can still be inspected via JMX, kill the JVM for errors during startup. +# best_effort: stop using the failed disk and respond to requests based on +# remaining available sstables. This means you WILL see obsolete +# data at CL.ONE! +# ignore: ignore fatal errors and let requests fail, as in pre-1.2 Cassandra +disk_failure_policy: stop + +# policy for commit disk failures: +# die: shut down gossip and Thrift and kill the JVM, so the node can be replaced. +# stop: shut down gossip and Thrift, leaving the node effectively dead, but +# can still be inspected via JMX. +# stop_commit: shutdown the commit log, letting writes collect but +# continuing to service reads, as in pre-2.0.5 Cassandra +# ignore: ignore fatal errors and let the batches fail +commit_failure_policy: stop + +# Maximum size of the native protocol prepared statement cache +# +# Valid values are either "auto" (omitting the value) or a value greater 0. +# +# Note that specifying a too large value will result in long running GCs and possbily +# out-of-memory errors. Keep the value at a small fraction of the heap. +# +# If you constantly see "prepared statements discarded in the last minute because +# cache limit reached" messages, the first step is to investigate the root cause +# of these messages and check whether prepared statements are used correctly - +# i.e. use bind markers for variable parts. +# +# Do only change the default value, if you really have more prepared statements than +# fit in the cache. In most cases it is not neccessary to change this value. +# Constantly re-preparing statements is a performance penalty. +# +# Default value ("auto") is 1/256th of the heap or 10MB, whichever is greater +prepared_statements_cache_size_mb: + +# Maximum size of the Thrift prepared statement cache +# +# If you do not use Thrift at all, it is safe to leave this value at "auto". +# +# See description of 'prepared_statements_cache_size_mb' above for more information. +# +# Default value ("auto") is 1/256th of the heap or 10MB, whichever is greater +thrift_prepared_statements_cache_size_mb: + +# Maximum size of the key cache in memory. +# +# Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the +# minimum, sometimes more. The key cache is fairly tiny for the amount of +# time it saves, so it's worthwhile to use it at large numbers. +# The row cache saves even more time, but must contain the entire row, +# so it is extremely space-intensive. It's best to only use the +# row cache if you have hot rows or static rows. +# +# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup. +# +# Default value is empty to make it "auto" (min(5% of Heap (in MB), 100MB)). Set to 0 to disable key cache. +key_cache_size_in_mb: + +# Duration in seconds after which Cassandra should +# save the key cache. Caches are saved to saved_caches_directory as +# specified in this configuration file. +# +# Saved caches greatly improve cold-start speeds, and is relatively cheap in +# terms of I/O for the key cache. Row cache saving is much more expensive and +# has limited use. +# +# Default is 14400 or 4 hours. +key_cache_save_period: 14400 + +# Number of keys from the key cache to save +# Disabled by default, meaning all keys are going to be saved +# key_cache_keys_to_save: 100 + +# Row cache implementation class name. +# Available implementations: +# org.apache.cassandra.cache.OHCProvider Fully off-heap row cache implementation (default). +# org.apache.cassandra.cache.SerializingCacheProvider This is the row cache implementation availabile +# in previous releases of Cassandra. +# row_cache_class_name: org.apache.cassandra.cache.OHCProvider + +# Maximum size of the row cache in memory. +# Please note that OHC cache implementation requires some additional off-heap memory to manage +# the map structures and some in-flight memory during operations before/after cache entries can be +# accounted against the cache capacity. This overhead is usually small compared to the whole capacity. +# Do not specify more memory that the system can afford in the worst usual situation and leave some +# headroom for OS block level cache. Do never allow your system to swap. +# +# Default value is 0, to disable row caching. +row_cache_size_in_mb: 0 + +# Duration in seconds after which Cassandra should save the row cache. +# Caches are saved to saved_caches_directory as specified in this configuration file. +# +# Saved caches greatly improve cold-start speeds, and is relatively cheap in +# terms of I/O for the key cache. Row cache saving is much more expensive and +# has limited use. +# +# Default is 0 to disable saving the row cache. +row_cache_save_period: 0 + +# Number of keys from the row cache to save. +# Specify 0 (which is the default), meaning all keys are going to be saved +# row_cache_keys_to_save: 100 + +# Maximum size of the counter cache in memory. +# +# Counter cache helps to reduce counter locks' contention for hot counter cells. +# In case of RF = 1 a counter cache hit will cause Cassandra to skip the read before +# write entirely. With RF > 1 a counter cache hit will still help to reduce the duration +# of the lock hold, helping with hot counter cell updates, but will not allow skipping +# the read entirely. Only the local (clock, count) tuple of a counter cell is kept +# in memory, not the whole counter, so it's relatively cheap. +# +# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup. +# +# Default value is empty to make it "auto" (min(2.5% of Heap (in MB), 50MB)). Set to 0 to disable counter cache. +# NOTE: if you perform counter deletes and rely on low gcgs, you should disable the counter cache. +counter_cache_size_in_mb: + +# Duration in seconds after which Cassandra should +# save the counter cache (keys only). Caches are saved to saved_caches_directory as +# specified in this configuration file. +# +# Default is 7200 or 2 hours. +counter_cache_save_period: 7200 + +# Number of keys from the counter cache to save +# Disabled by default, meaning all keys are going to be saved +# counter_cache_keys_to_save: 100 + +# saved caches +# If not set, the default directory is $CASSANDRA_HOME/data/saved_caches. +# saved_caches_directory: /var/lib/cassandra/saved_caches +saved_caches_directory: target/cassandra/saved_caches + +# commitlog_sync may be either "periodic" or "batch." +# +# When in batch mode, Cassandra won't ack writes until the commit log +# has been fsynced to disk. It will wait +# commitlog_sync_batch_window_in_ms milliseconds between fsyncs. +# This window should be kept short because the writer threads will +# be unable to do extra work while waiting. (You may need to increase +# concurrent_writes for the same reason.) +# +# commitlog_sync: batch +# commitlog_sync_batch_window_in_ms: 2 +# +# the other option is "periodic" where writes may be acked immediately +# and the CommitLog is simply synced every commitlog_sync_period_in_ms +# milliseconds. +commitlog_sync: periodic +commitlog_sync_period_in_ms: 1 + +# The size of the individual commitlog file segments. A commitlog +# segment may be archived, deleted, or recycled once all the data +# in it (potentially from each columnfamily in the system) has been +# flushed to sstables. +# +# The default size is 32, which is almost always fine, but if you are +# archiving commitlog segments (see commitlog_archiving.properties), +# then you probably want a finer granularity of archiving; 8 or 16 MB +# is reasonable. +# Max mutation size is also configurable via max_mutation_size_in_kb setting in +# cassandra.yaml. The default is half the size commitlog_segment_size_in_mb * 1024. +# +# NOTE: If max_mutation_size_in_kb is set explicitly then commitlog_segment_size_in_mb must +# be set to at least twice the size of max_mutation_size_in_kb / 1024 +# +commitlog_segment_size_in_mb: 32 + +# Compression to apply to the commit log. If omitted, the commit log +# will be written uncompressed. LZ4, Snappy, and Deflate compressors +# are supported. +#commitlog_compression: +# - class_name: LZ4Compressor +# parameters: +# - + +# any class that implements the SeedProvider interface and has a +# constructor that takes a Map of parameters will do. +seed_provider: + # Addresses of hosts that are deemed contact points. + # Cassandra nodes use this list of hosts to find each other and learn + # the topology of the ring. You must change this if you are running + # multiple nodes! + - class_name: org.apache.cassandra.locator.SimpleSeedProvider + parameters: + # seeds is actually a comma-delimited list of addresses. + # Ex: ",," + - seeds: "127.0.0.1" + +# For workloads with more data than can fit in memory, Cassandra's +# bottleneck will be reads that need to fetch data from +# disk. "concurrent_reads" should be set to (16 * number_of_drives) in +# order to allow the operations to enqueue low enough in the stack +# that the OS and drives can reorder them. Same applies to +# "concurrent_counter_writes", since counter writes read the current +# values before incrementing and writing them back. +# +# On the other hand, since writes are almost never IO bound, the ideal +# number of "concurrent_writes" is dependent on the number of cores in +# your system; (8 * number_of_cores) is a good rule of thumb. +concurrent_reads: 32 +concurrent_writes: 32 +concurrent_counter_writes: 32 + +# For materialized view writes, as there is a read involved, so this should +# be limited by the less of concurrent reads or concurrent writes. +concurrent_materialized_view_writes: 32 + +# Maximum memory to use for sstable chunk cache and buffer pooling. +# 32MB of this are reserved for pooling buffers, the rest is used as an +# cache that holds uncompressed sstable chunks. +# Defaults to the smaller of 1/4 of heap or 512MB. This pool is allocated off-heap, +# so is in addition to the memory allocated for heap. The cache also has on-heap +# overhead which is roughly 128 bytes per chunk (i.e. 0.2% of the reserved size +# if the default 64k chunk size is used). +# Memory is only allocated when needed. +# file_cache_size_in_mb: 512 + +# Flag indicating whether to allocate on or off heap when the sstable buffer +# pool is exhausted, that is when it has exceeded the maximum memory +# file_cache_size_in_mb, beyond which it will not cache buffers but allocate on request. + +# buffer_pool_use_heap_if_exhausted: true + +# The strategy for optimizing disk read +# Possible values are: +# ssd (for solid state disks, the default) +# spinning (for spinning disks) +# disk_optimization_strategy: ssd + +# Total permitted memory to use for memtables. Cassandra will stop +# accepting writes when the limit is exceeded until a flush completes, +# and will trigger a flush based on memtable_cleanup_threshold +# If omitted, Cassandra will set both to 1/4 the size of the heap. +# memtable_heap_space_in_mb: 2048 +# memtable_offheap_space_in_mb: 2048 + +# Ratio of occupied non-flushing memtable size to total permitted size +# that will trigger a flush of the largest memtable. Larger mct will +# mean larger flushes and hence less compaction, but also less concurrent +# flush activity which can make it difficult to keep your disks fed +# under heavy write load. +# +# memtable_cleanup_threshold defaults to 1 / (memtable_flush_writers + 1) +# memtable_cleanup_threshold: 0.11 + +# Specify the way Cassandra allocates and manages memtable memory. +# Options are: +# heap_buffers: on heap nio buffers +# offheap_buffers: off heap (direct) nio buffers +# offheap_objects: off heap objects +memtable_allocation_type: heap_buffers + +# Total space to use for commit logs on disk. +# +# If space gets above this value, Cassandra will flush every dirty CF +# in the oldest segment and remove it. So a small total commitlog space +# will tend to cause more flush activity on less-active columnfamilies. +# +# The default value is the smaller of 8192, and 1/4 of the total space +# of the commitlog volume. +# +# commitlog_total_space_in_mb: 8192 + +# This sets the amount of memtable flush writer threads. These will +# be blocked by disk io, and each one will hold a memtable in memory +# while blocked. +# +# memtable_flush_writers defaults to one per data_file_directory. +# +# If your data directories are backed by SSD, you can increase this, but +# avoid having memtable_flush_writers * data_file_directories > number of cores +#memtable_flush_writers: 1 + +# A fixed memory pool size in MB for for SSTable index summaries. If left +# empty, this will default to 5% of the heap size. If the memory usage of +# all index summaries exceeds this limit, SSTables with low read rates will +# shrink their index summaries in order to meet this limit. However, this +# is a best-effort process. In extreme conditions Cassandra may need to use +# more than this amount of memory. +index_summary_capacity_in_mb: + +# How frequently index summaries should be resampled. This is done +# periodically to redistribute memory from the fixed-size pool to sstables +# proportional their recent read rates. Setting to -1 will disable this +# process, leaving existing index summaries at their current sampling level. +index_summary_resize_interval_in_minutes: 60 + +# Whether to, when doing sequential writing, fsync() at intervals in +# order to force the operating system to flush the dirty +# buffers. Enable this to avoid sudden dirty buffer flushing from +# impacting read latencies. Almost always a good idea on SSDs; not +# necessarily on platters. +trickle_fsync: false +trickle_fsync_interval_in_kb: 10240 + +# TCP port, for commands and data +# For security reasons, you should not expose this port to the internet. Firewall it if needed. +storage_port: 7000 + +# SSL port, for encrypted communication. Unused unless enabled in +# encryption_options +# For security reasons, you should not expose this port to the internet. Firewall it if needed. +ssl_storage_port: 7001 + +# Address or interface to bind to and tell other Cassandra nodes to connect to. +# You _must_ change this if you want multiple nodes to be able to communicate! +# +# Set listen_address OR listen_interface, not both. Interfaces must correspond +# to a single address, IP aliasing is not supported. +# +# Leaving it blank leaves it up to InetAddress.getLocalHost(). This +# will always do the Right Thing _if_ the node is properly configured +# (hostname, name resolution, etc), and the Right Thing is to use the +# address associated with the hostname (it might not be). +# +# Setting listen_address to 0.0.0.0 is always wrong. +# +# If you choose to specify the interface by name and the interface has an ipv4 and an ipv6 address +# you can specify which should be chosen using listen_interface_prefer_ipv6. If false the first ipv4 +# address will be used. If true the first ipv6 address will be used. Defaults to false preferring +# ipv4. If there is only one address it will be selected regardless of ipv4/ipv6. +listen_address: localhost +# listen_interface: eth0 +# listen_interface_prefer_ipv6: false + +# Address to broadcast to other Cassandra nodes +# Leaving this blank will set it to the same value as listen_address +# broadcast_address: 1.2.3.4 + +# When using multiple physical network interfaces, set this +# to true to listen on broadcast_address in addition to +# the listen_address, allowing nodes to communicate in both +# interfaces. +# Ignore this property if the network configuration automatically +# routes between the public and private networks such as EC2. +# listen_on_broadcast_address: false + +# Internode authentication backend, implementing IInternodeAuthenticator; +# used to allow/disallow connections from peer nodes. +# internode_authenticator: org.apache.cassandra.auth.AllowAllInternodeAuthenticator + +# Whether to start the native transport server. +# Please note that the address on which the native transport is bound is the +# same as the rpc_address. The port however is different and specified below. +start_native_transport: true +# port for the CQL native transport to listen for clients on +# For security reasons, you should not expose this port to the internet. Firewall it if needed. +native_transport_port: 9042 +# Enabling native transport encryption in client_encryption_options allows you to either use +# encryption for the standard port or to use a dedicated, additional port along with the unencrypted +# standard native_transport_port. +# Enabling client encryption and keeping native_transport_port_ssl disabled will use encryption +# for native_transport_port. Setting native_transport_port_ssl to a different value +# from native_transport_port will use encryption for native_transport_port_ssl while +# keeping native_transport_port unencrypted. +# native_transport_port_ssl: 9142 +# The maximum threads for handling requests when the native transport is used. +# This is similar to rpc_max_threads though the default differs slightly (and +# there is no native_transport_min_threads, idle threads will always be stopped +# after 30 seconds). +# native_transport_max_threads: 128 +# +# The maximum size of allowed frame. Frame (requests) larger than this will +# be rejected as invalid. The default is 256MB. If you're changing this parameter, +# you may want to adjust max_value_size_in_mb accordingly. +# native_transport_max_frame_size_in_mb: 256 + +# The maximum number of concurrent client connections. +# The default is -1, which means unlimited. +# native_transport_max_concurrent_connections: -1 + +# The maximum number of concurrent client connections per source ip. +# The default is -1, which means unlimited. +# native_transport_max_concurrent_connections_per_ip: -1 + +# Whether to start the thrift rpc server. +start_rpc: true + +# The address or interface to bind the Thrift RPC service and native transport +# server to. +# +# Set rpc_address OR rpc_interface, not both. Interfaces must correspond +# to a single address, IP aliasing is not supported. +# +# Leaving rpc_address blank has the same effect as on listen_address +# (i.e. it will be based on the configured hostname of the node). +# +# Note that unlike listen_address, you can specify 0.0.0.0, but you must also +# set broadcast_rpc_address to a value other than 0.0.0.0. +# +# For security reasons, you should not expose this port to the internet. Firewall it if needed. +# +# If you choose to specify the interface by name and the interface has an ipv4 and an ipv6 address +# you can specify which should be chosen using rpc_interface_prefer_ipv6. If false the first ipv4 +# address will be used. If true the first ipv6 address will be used. Defaults to false preferring +# ipv4. If there is only one address it will be selected regardless of ipv4/ipv6. +rpc_address: 127.0.0.1 +# rpc_interface: eth1 +# rpc_interface_prefer_ipv6: false + +# port for Thrift to listen for clients on +rpc_port: 9160 + +# RPC address to broadcast to drivers and other Cassandra nodes. This cannot +# be set to 0.0.0.0. If left blank, this will be set to the value of +# rpc_address. If rpc_address is set to 0.0.0.0, broadcast_rpc_address must +# be set. +# broadcast_rpc_address: 1.2.3.4 + +# enable or disable keepalive on rpc/native connections +rpc_keepalive: true + +# Cassandra provides two out-of-the-box options for the RPC Server: +# +# sync -> One thread per thrift connection. For a very large number of clients, memory +# will be your limiting factor. On a 64 bit JVM, 180KB is the minimum stack size +# per thread, and that will correspond to your use of virtual memory (but physical memory +# may be limited depending on use of stack space). +# +# hsha -> Stands for "half synchronous, half asynchronous." All thrift clients are handled +# asynchronously using a small number of threads that does not vary with the amount +# of thrift clients (and thus scales well to many clients). The rpc requests are still +# synchronous (one thread per active request). If hsha is selected then it is essential +# that rpc_max_threads is changed from the default value of unlimited. +# +# The default is sync because on Windows hsha is about 30% slower. On Linux, +# sync/hsha performance is about the same, with hsha of course using less memory. +# +# Alternatively, can provide your own RPC server by providing the fully-qualified class name +# of an o.a.c.t.TServerFactory that can create an instance of it. +rpc_server_type: sync + +# Uncomment rpc_min|max_thread to set request pool size limits. +# +# Regardless of your choice of RPC server (see above), the number of maximum requests in the +# RPC thread pool dictates how many concurrent requests are possible (but if you are using the sync +# RPC server, it also dictates the number of clients that can be connected at all). +# +# The default is unlimited and thus provides no protection against clients overwhelming the server. You are +# encouraged to set a maximum that makes sense for you in production, but do keep in mind that +# rpc_max_threads represents the maximum number of client requests this server may execute concurrently. +# +# rpc_min_threads: 16 +# rpc_max_threads: 2048 + +# uncomment to set socket buffer sizes on rpc connections +# rpc_send_buff_size_in_bytes: +# rpc_recv_buff_size_in_bytes: + +# Uncomment to set socket buffer size for internode communication +# Note that when setting this, the buffer size is limited by net.core.wmem_max +# and when not setting it it is defined by net.ipv4.tcp_wmem +# See: +# /proc/sys/net/core/wmem_max +# /proc/sys/net/core/rmem_max +# /proc/sys/net/ipv4/tcp_wmem +# /proc/sys/net/ipv4/tcp_wmem +# and: man tcp +# internode_send_buff_size_in_bytes: +# internode_recv_buff_size_in_bytes: + +# Frame size for thrift (maximum message length). +thrift_framed_transport_size_in_mb: 15 + +# Set to true to have Cassandra create a hard link to each sstable +# flushed or streamed locally in a backups/ subdirectory of the +# keyspace data. Removing these links is the operator's +# responsibility. +incremental_backups: false + +# Whether or not to take a snapshot before each compaction. Be +# careful using this option, since Cassandra won't clean up the +# snapshots for you. Mostly useful if you're paranoid when there +# is a data format change. +snapshot_before_compaction: false + +# Whether or not a snapshot is taken of the data before keyspace truncation +# or dropping of column families. The STRONGLY advised default of true +# should be used to provide data safety. If you set this flag to false, you will +# lose data on truncation or drop. +auto_snapshot: true + +# Granularity of the collation index of rows within a partition. +# Increase if your rows are large, or if you have a very large +# number of rows per partition. The competing goals are these: +# 1) a smaller granularity means more index entries are generated +# and looking up rows withing the partition by collation column +# is faster +# 2) but, Cassandra will keep the collation index in memory for hot +# rows (as part of the key cache), so a larger granularity means +# you can cache more hot rows +column_index_size_in_kb: 64 +# Per sstable indexed key cache entries (the collation index in memory +# mentioned above) exceeding this size will not be held on heap. +# This means that only partition information is held on heap and the +# index entries are read from disk. +# +# Note that this size refers to the size of the +# serialized index information and not the size of the partition. +column_index_cache_size_in_kb: 2 + +# Number of simultaneous compactions to allow, NOT including +# validation "compactions" for anti-entropy repair. Simultaneous +# compactions can help preserve read performance in a mixed read/write +# workload, by mitigating the tendency of small sstables to accumulate +# during a single long running compactions. The default is usually +# fine and if you experience problems with compaction running too +# slowly or too fast, you should look at +# compaction_throughput_mb_per_sec first. +# +# concurrent_compactors defaults to the smaller of (number of disks, +# number of cores), with a minimum of 2 and a maximum of 8. +# +# If your data directories are backed by SSD, you should increase this +# to the number of cores. +#concurrent_compactors: 1 + +# Throttles compaction to the given total throughput across the entire +# system. The faster you insert data, the faster you need to compact in +# order to keep the sstable count down, but in general, setting this to +# 16 to 32 times the rate you are inserting data is more than sufficient. +# Setting this to 0 disables throttling. Note that this account for all types +# of compaction, including validation compaction. +compaction_throughput_mb_per_sec: 16 + +# When compacting, the replacement sstable(s) can be opened before they +# are completely written, and used in place of the prior sstables for +# any range that has been written. This helps to smoothly transfer reads +# between the sstables, reducing page cache churn and keeping hot rows hot +sstable_preemptive_open_interval_in_mb: 50 + +# Throttles all outbound streaming file transfers on this node to the +# given total throughput in Mbps. This is necessary because Cassandra does +# mostly sequential IO when streaming data during bootstrap or repair, which +# can lead to saturating the network connection and degrading rpc performance. +# When unset, the default is 200 Mbps or 25 MB/s. +# stream_throughput_outbound_megabits_per_sec: 200 + +# Throttles all streaming file transfer between the datacenters, +# this setting allows users to throttle inter dc stream throughput in addition +# to throttling all network stream traffic as configured with +# stream_throughput_outbound_megabits_per_sec +# When unset, the default is 200 Mbps or 25 MB/s +# inter_dc_stream_throughput_outbound_megabits_per_sec: 200 + +# How long the coordinator should wait for read operations to complete +read_request_timeout_in_ms: 5000 +# How long the coordinator should wait for seq or index scans to complete +range_request_timeout_in_ms: 10000 +# How long the coordinator should wait for writes to complete +write_request_timeout_in_ms: 2000 +# How long the coordinator should wait for counter writes to complete +counter_write_request_timeout_in_ms: 5000 +# How long a coordinator should continue to retry a CAS operation +# that contends with other proposals for the same row +cas_contention_timeout_in_ms: 1000 +# How long the coordinator should wait for truncates to complete +# (This can be much longer, because unless auto_snapshot is disabled +# we need to flush first so we can snapshot before removing the data.) +truncate_request_timeout_in_ms: 60000 +# The default timeout for other, miscellaneous operations +request_timeout_in_ms: 10000 + +# Enable operation timeout information exchange between nodes to accurately +# measure request timeouts. If disabled, replicas will assume that requests +# were forwarded to them instantly by the coordinator, which means that +# under overload conditions we will waste that much extra time processing +# already-timed-out requests. +# +# Warning: before enabling this property make sure to ntp is installed +# and the times are synchronized between the nodes. +cross_node_timeout: false + +# Set socket timeout for streaming operation. +# The stream session is failed if no data/ack is received by any of the participants +# within that period, which means this should also be sufficient to stream a large +# sstable or rebuild table indexes. +# Default value is 86400000ms, which means stale streams timeout after 24 hours. +# A value of zero means stream sockets should never time out. +# streaming_socket_timeout_in_ms: 86400000 + +# phi value that must be reached for a host to be marked down. +# most users should never need to adjust this. +# phi_convict_threshold: 8 + +# endpoint_snitch -- Set this to a class that implements +# IEndpointSnitch. The snitch has two functions: +# - it teaches Cassandra enough about your network topology to route +# requests efficiently +# - it allows Cassandra to spread replicas around your cluster to avoid +# correlated failures. It does this by grouping machines into +# "datacenters" and "racks." Cassandra will do its best not to have +# more than one replica on the same "rack" (which may not actually +# be a physical location) +# +# IF YOU CHANGE THE SNITCH AFTER DATA IS INSERTED INTO THE CLUSTER, +# YOU MUST RUN A FULL REPAIR, SINCE THE SNITCH AFFECTS WHERE REPLICAS +# ARE PLACED. +# +# IF THE RACK A REPLICA IS PLACED IN CHANGES AFTER THE REPLICA HAS BEEN +# ADDED TO A RING, THE NODE MUST BE DECOMMISSIONED AND REBOOTSTRAPPED. +# +# Out of the box, Cassandra provides +# - SimpleSnitch: +# Treats Strategy order as proximity. This can improve cache +# locality when disabling read repair. Only appropriate for +# single-datacenter deployments. +# - GossipingPropertyFileSnitch +# This should be your go-to snitch for production use. The rack +# and datacenter for the local node are defined in +# cassandra-rackdc.properties and propagated to other nodes via +# gossip. If cassandra-topology.properties exists, it is used as a +# fallback, allowing migration from the PropertyFileSnitch. +# - PropertyFileSnitch: +# Proximity is determined by rack and data center, which are +# explicitly configured in cassandra-topology.properties. +# - Ec2Snitch: +# Appropriate for EC2 deployments in a single Region. Loads Region +# and Availability Zone information from the EC2 API. The Region is +# treated as the datacenter, and the Availability Zone as the rack. +# Only private IPs are used, so this will not work across multiple +# Regions. +# - Ec2MultiRegionSnitch: +# Uses public IPs as broadcast_address to allow cross-region +# connectivity. (Thus, you should set seed addresses to the public +# IP as well.) You will need to open the storage_port or +# ssl_storage_port on the public IP firewall. (For intra-Region +# traffic, Cassandra will switch to the private IP after +# establishing a connection.) +# - RackInferringSnitch: +# Proximity is determined by rack and data center, which are +# assumed to correspond to the 3rd and 2nd octet of each node's IP +# address, respectively. Unless this happens to match your +# deployment conventions, this is best used as an example of +# writing a custom Snitch class and is provided in that spirit. +# +# You can use a custom Snitch by setting this to the full class name +# of the snitch, which will be assumed to be on your classpath. +endpoint_snitch: SimpleSnitch + +# controls how often to perform the more expensive part of host score +# calculation +dynamic_snitch_update_interval_in_ms: 100 +# controls how often to reset all host scores, allowing a bad host to +# possibly recover +dynamic_snitch_reset_interval_in_ms: 600000 +# if set greater than zero and read_repair_chance is < 1.0, this will allow +# 'pinning' of replicas to hosts in order to increase cache capacity. +# The badness threshold will control how much worse the pinned host has to be +# before the dynamic snitch will prefer other replicas over it. This is +# expressed as a double which represents a percentage. Thus, a value of +# 0.2 means Cassandra would continue to prefer the static snitch values +# until the pinned host was 20% worse than the fastest. +dynamic_snitch_badness_threshold: 0.1 + +# request_scheduler -- Set this to a class that implements +# RequestScheduler, which will schedule incoming client requests +# according to the specific policy. This is useful for multi-tenancy +# with a single Cassandra cluster. +# NOTE: This is specifically for requests from the client and does +# not affect inter node communication. +# org.apache.cassandra.scheduler.NoScheduler - No scheduling takes place +# org.apache.cassandra.scheduler.RoundRobinScheduler - Round robin of +# client requests to a node with a separate queue for each +# request_scheduler_id. The scheduler is further customized by +# request_scheduler_options as described below. +request_scheduler: org.apache.cassandra.scheduler.NoScheduler + +# Scheduler Options vary based on the type of scheduler +# NoScheduler - Has no options +# RoundRobin +# - throttle_limit -- The throttle_limit is the number of in-flight +# requests per client. Requests beyond +# that limit are queued up until +# running requests can complete. +# The value of 80 here is twice the number of +# concurrent_reads + concurrent_writes. +# - default_weight -- default_weight is optional and allows for +# overriding the default which is 1. +# - weights -- Weights are optional and will default to 1 or the +# overridden default_weight. The weight translates into how +# many requests are handled during each turn of the +# RoundRobin, based on the scheduler id. +# +# request_scheduler_options: +# throttle_limit: 80 +# default_weight: 5 +# weights: +# Keyspace1: 1 +# Keyspace2: 5 + +# request_scheduler_id -- An identifier based on which to perform +# the request scheduling. Currently the only valid option is keyspace. +# request_scheduler_id: keyspace + +# Enable or disable inter-node encryption +# JVM defaults for supported SSL socket protocols and cipher suites can +# be replaced using custom encryption options. This is not recommended +# unless you have policies in place that dictate certain settings, or +# need to disable vulnerable ciphers or protocols in case the JVM cannot +# be updated. +# FIPS compliant settings can be configured at JVM level and should not +# involve changing encryption settings here: +# https://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/FIPS.html +# NOTE: No custom encryption options are enabled at the moment +# The available internode options are : all, none, dc, rack +# +# If set to dc cassandra will encrypt the traffic between the DCs +# If set to rack cassandra will encrypt the traffic between the racks +# +# The passwords used in these options must match the passwords used when generating +# the keystore and truststore. For instructions on generating these files, see: +# http://download.oracle.com/javase/6/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore +# +server_encryption_options: + internode_encryption: none + keystore: conf/.keystore + keystore_password: cassandra + truststore: conf/.truststore + truststore_password: cassandra + # More advanced defaults below: + # protocol: TLS + # algorithm: SunX509 + # store_type: JKS + # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA,TLS_DHE_RSA_WITH_AES_128_CBC_SHA,TLS_DHE_RSA_WITH_AES_256_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA] + # require_client_auth: false + # require_endpoint_verification: false + +# enable or disable client/server encryption. +client_encryption_options: + enabled: false + # If enabled and optional is set to true encrypted and unencrypted connections are handled. + optional: false + keystore: conf/.keystore + keystore_password: cassandra + # require_client_auth: false + # Set trustore and truststore_password if require_client_auth is true + # truststore: conf/.truststore + # truststore_password: cassandra + # More advanced defaults below: + # protocol: TLS + # algorithm: SunX509 + # store_type: JKS + # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA,TLS_DHE_RSA_WITH_AES_128_CBC_SHA,TLS_DHE_RSA_WITH_AES_256_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA] + +# internode_compression controls whether traffic between nodes is +# compressed. +# can be: all - all traffic is compressed +# dc - traffic between different datacenters is compressed +# none - nothing is compressed. +internode_compression: dc + +# Enable or disable tcp_nodelay for inter-dc communication. +# Disabling it will result in larger (but fewer) network packets being sent, +# reducing overhead from the TCP protocol itself, at the cost of increasing +# latency if you block for cross-datacenter responses. +inter_dc_tcp_nodelay: false + +# TTL for different trace types used during logging of the repair process. +tracetype_query_ttl: 86400 +tracetype_repair_ttl: 604800 + +# UDFs (user defined functions) are disabled by default. +# As of Cassandra 3.0 there is a sandbox in place that should prevent execution of evil code. +enable_user_defined_functions: false + +# Enables scripted UDFs (JavaScript UDFs). +# Java UDFs are always enabled, if enable_user_defined_functions is true. +# Enable this option to be able to use UDFs with "language javascript" or any custom JSR-223 provider. +# This option has no effect, if enable_user_defined_functions is false. +enable_scripted_user_defined_functions: false + +# The default Windows kernel timer and scheduling resolution is 15.6ms for power conservation. +# Lowering this value on Windows can provide much tighter latency and better throughput, however +# some virtualized environments may see a negative performance impact from changing this setting +# below their system default. The sysinternals 'clockres' tool can confirm your system's default +# setting. +windows_timer_interval: 1 + + +# Enables encrypting data at-rest (on disk). Different key providers can be plugged in, but the default reads from +# a JCE-style keystore. A single keystore can hold multiple keys, but the one referenced by +# the "key_alias" is the only key that will be used for encrypt opertaions; previously used keys +# can still (and should!) be in the keystore and will be used on decrypt operations +# (to handle the case of key rotation). +# +# It is strongly recommended to download and install Java Cryptography Extension (JCE) +# Unlimited Strength Jurisdiction Policy Files for your version of the JDK. +# (current link: http://www.oracle.com/technetwork/java/javase/downloads/jce8-download-2133166.html) +# +# Currently, only the following file types are supported for transparent data encryption, although +# more are coming in future cassandra releases: commitlog, hints +transparent_data_encryption_options: + enabled: false + chunk_length_kb: 64 + cipher: AES/CBC/PKCS5Padding + key_alias: testing:1 + # CBC IV length for AES needs to be 16 bytes (which is also the default size) + # iv_length: 16 + key_provider: + - class_name: org.apache.cassandra.security.JKSKeyProvider + parameters: + - keystore: conf/.keystore + keystore_password: cassandra + store_type: JCEKS + key_password: cassandra + + +##################### +# SAFETY THRESHOLDS # +##################### + +# When executing a scan, within or across a partition, we need to keep the +# tombstones seen in memory so we can return them to the coordinator, which +# will use them to make sure other replicas also know about the deleted rows. +# With workloads that generate a lot of tombstones, this can cause performance +# problems and even exaust the server heap. +# (http://www.datastax.com/dev/blog/cassandra-anti-patterns-queues-and-queue-like-datasets) +# Adjust the thresholds here if you understand the dangers and want to +# scan more tombstones anyway. These thresholds may also be adjusted at runtime +# using the StorageService mbean. +tombstone_warn_threshold: 1000 +tombstone_failure_threshold: 100000 + +# Log WARN on any batch size exceeding this value. 5kb per batch by default. +# Caution should be taken on increasing the size of this threshold as it can lead to node instability. +batch_size_warn_threshold_in_kb: 5 + +# Fail any batch exceeding this value. 50kb (10x warn threshold) by default. +batch_size_fail_threshold_in_kb: 50 + +# Log WARN on any batches not of type LOGGED than span across more partitions than this limit +unlogged_batch_across_partitions_warn_threshold: 10 + +# Log a warning when compacting partitions larger than this value +compaction_large_partition_warning_threshold_mb: 100 + +# GC Pauses greater than gc_warn_threshold_in_ms will be logged at WARN level +# Adjust the threshold based on your application throughput requirement +# By default, Cassandra logs GC Pauses greater than 200 ms at INFO level +gc_warn_threshold_in_ms: 1000 + +# Maximum size of any value in SSTables. Safety measure to detect SSTable corruption +# early. Any value size larger than this threshold will result into marking an SSTable +# as corrupted. +# max_value_size_in_mb: 256 + diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/SmallITCluster/cassandra-svc-rc.yaml b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/SmallITCluster/cassandra-svc-rc.yaml new file mode 100644 index 0000000000000..7c36e34845cec --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/SmallITCluster/cassandra-svc-rc.yaml @@ -0,0 +1,88 @@ +# 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. + +# Headless service that allows us to get the IP addresses of our Cassandra nodes +apiVersion: v1 +kind: Service +metadata: + labels: + name: cassandra-peers + name: cassandra-peers +spec: + clusterIP: None + ports: + - port: 7000 + name: intra-node-communication + - port: 7001 + name: tls-intra-node-communication + selector: + name: cassandra +--- +# Kubernetes service file exposing Cassandra endpoint used by clients. +apiVersion: v1 +kind: Service +metadata: + labels: + name: cassandra + name: cassandra +spec: + ports: + - port: 9042 + name: cql + selector: + name: cassandra + type: LoadBalancer +--- +# Replication Controller for Cassandra which tracks the Cassandra pods. +apiVersion: v1 +kind: ReplicationController +metadata: + labels: + name: cassandra + name: cassandra +spec: + replicas: 1 + selector: + name: cassandra + template: + metadata: + labels: + name: cassandra + spec: + containers: + - image: cassandra + name: cassandra + env: + - name: PEER_DISCOVERY_SERVICE + value: cassandra-peers + - name: CASSANDRA_CLUSTER_NAME + value: Cassandra + - name: CASSANDRA_DC + value: DC1 + - name: CASSANDRA_RACK + value: Kubernetes Cluster +# Number of tokens currently configured to 1. If this is not configured, default value is 256. You can change it as per requirement. + - name: CASSANDRA_NUM_TOKENS + value: '1' + ports: + - containerPort: 9042 + name: cql + volumeMounts: + - mountPath: /var/lib/cassandra/data + name: data + volumes: + - name: data + emptyDir: {} \ No newline at end of file diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/SmallITCluster/start-up.sh b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/SmallITCluster/start-up.sh new file mode 100644 index 0000000000000..c05b771de43f3 --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/SmallITCluster/start-up.sh @@ -0,0 +1,21 @@ +# 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. + +#!/bin/bash +set -e + +# Create Cassandra services and Replication controller. +kubectl create -f cassandra-svc-rc.yaml diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/SmallITCluster/teardown.sh b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/SmallITCluster/teardown.sh new file mode 100644 index 0000000000000..f538a753eb0ae --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/SmallITCluster/teardown.sh @@ -0,0 +1,21 @@ +# +# 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. +# +#!/bin/bash +set -e + +# Delete Cassandra services and Replication controller. +kubectl delete -f cassandra-svc-rc.yaml diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/data-load-setup.sh b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/data-load-setup.sh new file mode 100644 index 0000000000000..4e12f89479bd7 --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/data-load-setup.sh @@ -0,0 +1,29 @@ +# 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. + +#!/bin/bash +set -e + +# Load YCSB tool +echo "Downloading YCSB tool" +echo "------------------------------" +curl -O --location https://github.com/brianfrankcooper/YCSB/releases/download/0.12.0/ycsb-0.12.0.tar.gz +tar xfz ycsb-0.12.0.tar.gz +wget https://www.slf4j.org/dist/slf4j-1.7.22.tar.gz +tar xfz slf4j-1.7.22.tar.gz +cp slf4j-1.7.22/slf4j-simple-*.jar ycsb-0.12.0/lib/ +cp slf4j-1.7.22/slf4j-api-*.jar ycsb-0.12.0/lib/ +echo "YCSB tool loaded" diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/data-load.sh b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/data-load.sh new file mode 100644 index 0000000000000..59d0e2245d09e --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/cassandra/data-load.sh @@ -0,0 +1,67 @@ +# 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. + +#!/bin/bash +set -e + +recordcount=1000 +# Identify the pod +cassandra_pods="kubectl get pods -l name=cassandra" +running_seed="$(kubectl get pods -o json -l name=cassandra -o jsonpath=\ +'{.items[0].metadata.name}')" +echo "Detected Running Pod $running_seed" + +# After starting the service, it takes couple of minutes to generate the external IP for the +# service. Hence, wait for sometime. + +# Identify external IP of the pod +external_ip="$(kubectl get svc cassandra -o jsonpath='{.status.loadBalancer.ingress[0].ip}')" +echo "Waiting for the Cassandra service to come up ........" +while [ -z "$external_ip" ] +do + sleep 10s + external_ip="$(kubectl get svc cassandra -o jsonpath='{.status.loadBalancer.ingress[0].ip}')" + echo "." +done +echo "External IP - $external_ip" + +# Create keyspace +keyspace_creation_command="drop keyspace if exists ycsb;create keyspace ycsb WITH REPLICATION = {\ +'class' : 'SimpleStrategy', 'replication_factor': 3 };" +kubectl exec -ti $running_seed -- cqlsh -e "$keyspace_creation_command" +echo "Keyspace creation............" +echo "-----------------------------" +echo "$keyspace_creation_command" +echo + +# Create table +table_creation_command="use ycsb;drop table if exists usertable;create table usertable (\ +y_id varchar primary key,field0 varchar,field1 varchar,field2 varchar,field3 varchar,\ +field4 varchar,field5 varchar,field6 varchar,field7 varchar,field8 varchar,field9 varchar);" +kubectl exec -ti $running_seed -- cqlsh -e "$table_creation_command" +echo "Table creation .............." +echo "-----------------------------" +echo "$table_creation_command" + +cd ycsb-0.12.0 + +echo "Starting to load data on ${external_ip}" +echo "-----------------------------" +# Record count set to 1000, change this value to load as per requirement. +# dataintegrity flag is set to true to load deterministic data +./bin/ycsb load cassandra-cql -p hosts=${external_ip} -p dataintegrity=true -p recordcount=\ +${recordcount} -p insertorder=ordered -p fieldlength=20 -P workloads/workloadd \ +-s > workloada_load_res.txt diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/LargeProductionCluster/es-services.yaml b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/LargeProductionCluster/es-services.yaml new file mode 100644 index 0000000000000..38c820e3209f7 --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/LargeProductionCluster/es-services.yaml @@ -0,0 +1,277 @@ +# 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. + +# Service file containing services for ES discovery, elasticsearch and master node deployment. + +# Kubernetes headless service for Elasticsearch discovery of nodes. +apiVersion: v1 +kind: Service +metadata: + name: elasticsearch-discovery + labels: + component: elasticsearch + role: master +spec: + selector: + component: elasticsearch + role: master + ports: + - name: transport + port: 9300 + protocol: TCP +--- +# To create Elasticsearch frontend cluster Kubernetes service. +# It sets up a load balancer on TCP port 9200 that distributes network traffic to the ES client nodes. +apiVersion: v1 +kind: Service +metadata: + name: elasticsearch + labels: + component: elasticsearch + role: client +spec: + type: LoadBalancer + selector: + component: elasticsearch + role: client + ports: + - name: http + port: 9200 + protocol: TCP +--- +# The Kubernetes deployment script for Elasticsearch master nodes. +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + name: es-master + labels: + component: elasticsearch + role: master +spec: + replicas: 3 + template: + metadata: + labels: + component: elasticsearch + role: master + annotations: + pod.beta.kubernetes.io/init-containers: '[ + { + "name": "sysctl", + "image": "busybox", + "imagePullPolicy": "IfNotPresent", + "command": ["sysctl", "-w", "vm.max_map_count=262144"], + "securityContext": { + "privileged": true + } + } + ]' + spec: + containers: + - name: es-master + securityContext: + privileged: false + capabilities: + add: +# IPC_LOCK capability is enabled to allow Elasticsearch to lock the heap in memory so it will not be swapped. + - IPC_LOCK +# SYS_RESOURCE is docker capability key to control and override the resource limits. +# This could be needed to increase base limits.(e.g. File descriptor limit for elasticsearch) + - SYS_RESOURCE + image: quay.io/pires/docker-elasticsearch-kubernetes:5.2.2 + env: + - name: NAMESPACE + valueFrom: + fieldRef: + fieldPath: metadata.namespace + - name: NODE_NAME + valueFrom: + fieldRef: + fieldPath: metadata.name + - name: "CLUSTER_NAME" + value: "myesdb" + - name: "NUMBER_OF_MASTERS" + value: "2" + - name: NODE_MASTER + value: "true" + - name: NODE_INGEST + value: "false" + - name: NODE_DATA + value: "false" + - name: HTTP_ENABLE + value: "false" + - name: "ES_JAVA_OPTS" + value: "-Xms256m -Xmx256m" + ports: + - containerPort: 9300 + name: transport + protocol: TCP + volumeMounts: + - name: storage + mountPath: /data + volumes: + - emptyDir: + medium: "" + name: "storage" +--- +# Kubernetes deployment script for Elasticsearch client nodes (aka load balancing proxies). +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + name: es-client + labels: + component: elasticsearch + role: client +spec: + # The no. of replicas can be incremented based on the client usage using HTTP API. + replicas: 1 + template: + metadata: + labels: + component: elasticsearch + role: client + annotations: + # Elasticsearch uses a hybrid mmapfs / niofs directory by default to store its indices. + # The default operating system limits on mmap counts is likely to be too low, which may result + # in out of memory exceptions. Therefore, the need to increase virtual memory + # vm.max_map_count for large amount of data in the pod initialization annotation. + pod.beta.kubernetes.io/init-containers: '[ + { + "name": "sysctl", + "image": "busybox", + "imagePullPolicy": "IfNotPresent", + "command": ["sysctl", "-w", "vm.max_map_count=262144"], + "securityContext": { + "privileged": true + } + } + ]' + spec: + containers: + - name: es-client + securityContext: + privileged: false + capabilities: + add: +# IPC_LOCK capability is enabled to allow Elasticsearch to lock the heap in memory so it will not be swapped. + - IPC_LOCK +# SYS_RESOURCE is docker capability key to control and override the resource limits. +# This could be needed to increase base limits.(e.g. File descriptor limit for elasticsearch) + - SYS_RESOURCE + image: quay.io/pires/docker-elasticsearch-kubernetes:5.2.2 + env: + - name: NAMESPACE + valueFrom: + fieldRef: + fieldPath: metadata.namespace + - name: NODE_NAME + valueFrom: + fieldRef: + fieldPath: metadata.name + - name: "CLUSTER_NAME" + value: "myesdb" + - name: NODE_MASTER + value: "false" + - name: NODE_DATA + value: "false" + - name: HTTP_ENABLE + value: "true" + - name: "ES_JAVA_OPTS" + value: "-Xms256m -Xmx256m" + ports: + - containerPort: 9200 + name: http + protocol: TCP + - containerPort: 9300 + name: transport + protocol: TCP + volumeMounts: + - name: storage + mountPath: /data + volumes: + - emptyDir: + medium: "" + name: "storage" +--- +# Kubernetes deployment script for Elasticsearch data nodes which store and index data. +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + name: es-data + labels: + component: elasticsearch + role: data +spec: + replicas: 2 + template: + metadata: + labels: + component: elasticsearch + role: data + annotations: + pod.beta.kubernetes.io/init-containers: '[ + { + "name": "sysctl", + "image": "busybox", + "imagePullPolicy": "IfNotPresent", + "command": ["sysctl", "-w", "vm.max_map_count=1048575"], + "securityContext": { + "privileged": true + } + } + ]' + spec: + containers: + - name: es-data + securityContext: + privileged: false + capabilities: + add: +# IPC_LOCK capability is enabled to allow Elasticsearch to lock the heap in memory so it will not be swapped. + - IPC_LOCK +# SYS_RESOURCE is docker capability key to control and override the resource limits. +# This could be needed to increase base limits.(e.g. File descriptor limit for elasticsearch) + - SYS_RESOURCE + image: quay.io/pires/docker-elasticsearch-kubernetes:5.2.2 + env: + - name: NAMESPACE + valueFrom: + fieldRef: + fieldPath: metadata.namespace + - name: NODE_NAME + valueFrom: + fieldRef: + fieldPath: metadata.name + - name: "CLUSTER_NAME" + value: "myesdb" + - name: NODE_MASTER + value: "false" + - name: NODE_INGEST + value: "false" + - name: HTTP_ENABLE + value: "false" + - name: "ES_JAVA_OPTS" + value: "-Xms256m -Xmx256m" + ports: + - containerPort: 9300 + name: transport + protocol: TCP + volumeMounts: + - name: storage + mountPath: /data + volumes: + - emptyDir: + medium: "" + name: "storage" \ No newline at end of file diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/LargeProductionCluster/start-up.sh b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/LargeProductionCluster/start-up.sh new file mode 100644 index 0000000000000..4d277c84017aa --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/LargeProductionCluster/start-up.sh @@ -0,0 +1,21 @@ +# 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. +# + +#!/bin/sh +set -e + +# Create Elasticsearch services and deployments. +kubectl create -f es-services.yaml diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/LargeProductionCluster/teardown.sh b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/LargeProductionCluster/teardown.sh new file mode 100644 index 0000000000000..a30793beb7cbe --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/LargeProductionCluster/teardown.sh @@ -0,0 +1,20 @@ +# 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. + +#!/bin/bash +set -e + +# Delete elasticsearch services and deployments. +kubectl delete -f es-services.yaml \ No newline at end of file diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/SmallITCluster/elasticsearch-svc-rc.yaml b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/SmallITCluster/elasticsearch-svc-rc.yaml new file mode 100644 index 0000000000000..9a7ac3d7a8d9e --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/SmallITCluster/elasticsearch-svc-rc.yaml @@ -0,0 +1,84 @@ +# 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. + +# To create Elasticsearch frontend cluster Kubernetes service. +# It sets up a load balancer on TCP port 9200 that distributes network traffic to the ES nodes. +apiVersion: v1 +kind: Service +metadata: + name: elasticsearch + labels: + component: elasticsearch +spec: + type: LoadBalancer + selector: + component: elasticsearch + ports: + - name: http + port: 9200 + protocol: TCP + - name: transport + port: 9300 + protocol: TCP +--- +# The Kubernetes deployment script for Elasticsearch replication nodes. It will create 1 node cluster. +# To scale the cluster as desired, you can create replicas of node use 'kubectl scale --replicas=3 rc es' command +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + name: es + labels: + component: elasticsearch +spec: + replicas: 1 + template: + metadata: + labels: + component: elasticsearch + spec: + containers: + - name: es + securityContext: + capabilities: + add: +# IPC_LOCK capability is enabled to allow Elasticsearch to lock the heap in memory so it will not be swapped. + - IPC_LOCK +# SYS_RESOURCE capability is set to control and override various resource limits. + - SYS_RESOURCE + image: quay.io/pires/docker-elasticsearch-kubernetes:5.2.2 + env: + - name: "CLUSTER_NAME" + value: "myesdb" + - name: "DISCOVERY_SERVICE" + value: "elasticsearch" + - name: NODE_MASTER + value: "true" + - name: NODE_DATA + value: "true" + - name: HTTP_ENABLE + value: "true" + ports: + - containerPort: 9200 + name: http + protocol: TCP + - containerPort: 9300 + name: transport + protocol: TCP + volumeMounts: + - mountPath: /data + name: storage + volumes: + - name: storage + emptyDir: {} \ No newline at end of file diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/SmallITCluster/start-up.sh b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/SmallITCluster/start-up.sh new file mode 100644 index 0000000000000..e8cf275d97295 --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/SmallITCluster/start-up.sh @@ -0,0 +1,22 @@ +# 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. +# + +#!/bin/sh +set -e + +# Create Elasticsearch services and deployments. +kubectl create -f elasticsearch-svc-rc.yaml + diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/SmallITCluster/teardown.sh b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/SmallITCluster/teardown.sh new file mode 100644 index 0000000000000..079141d8cd9b4 --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/SmallITCluster/teardown.sh @@ -0,0 +1,20 @@ +# 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. + +#!/bin/bash +set -e + +# Delete elasticsearch services and deployments. +kubectl delete -f elasticsearch-svc-rc.yaml diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/data-load-setup.sh b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/data-load-setup.sh new file mode 100644 index 0000000000000..00991bc6d3379 --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/data-load-setup.sh @@ -0,0 +1,26 @@ +# 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. + +#!/bin/bash +set -e + +# Install python +sudo apt-get update +sudo apt-get install python-pip +sudo pip install --upgrade pip +sudo apt-get install python-dev +sudo pip install tornado numpy +echo diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/data-load.sh b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/data-load.sh new file mode 100644 index 0000000000000..21150fbee43c3 --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/data-load.sh @@ -0,0 +1,33 @@ +# 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. + +#!/bin/bash +set -e + +# Identify external IP +external_ip="$(kubectl get svc elasticsearch -o jsonpath='{.status.loadBalancer.ingress[0].ip}')" +echo "Waiting for the Elasticsearch service to come up ........" +while [ -z "$external_ip" ] +do + sleep 10s + external_ip="$(kubectl get svc elasticsearch -o jsonpath='{.status.loadBalancer.ingress[0].ip}')" + echo "." +done +echo "External IP - $external_ip" +echo + +# Run the script +/usr/bin/python es_test_data.py --count=1000 --format=Txn_ID:int,Item_Code:int,Item_ID:int,User_Name:str,last_updated:ts,Price:int,Title:str,Description:str,Age:int,Item_Name:str,Item_Price:int,Availability:bool,Batch_Num:int,Last_Ordered:tstxt,City:text --es_url=http://$external_ip:9200 & \ No newline at end of file diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/es_test_data.py b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/es_test_data.py new file mode 100644 index 0000000000000..1658e2cc50886 --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/es_test_data.py @@ -0,0 +1,299 @@ +# 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. + +# Script to populate data on Elasticsearch +# Hashcode for 1000 records is ed36c09b5e24a95fd8d3cc711a043a85320bb47d, +# For test with query to select one record from 1000 docs, +# hashcode is 83c108ff81e87b6f3807c638e6bb9a9e3d430dc7 +# Hashcode for 50m records (~20 gigs) is aff7390ee25c4c330f0a58dfbfe335421b11e405 +#!/usr/bin/python + +import json +import time +import logging +import random +import string +import uuid +import datetime + +import tornado.gen +import tornado.httpclient +import tornado.ioloop +import tornado.options + +async_http_client = tornado.httpclient.AsyncHTTPClient() +id_counter = 0 +upload_data_count = 0 +_dict_data = None + + + +def delete_index(idx_name): + try: + url = "%s/%s?refresh=true" % (tornado.options.options.es_url, idx_name) + request = tornado.httpclient.HTTPRequest(url, method="DELETE", request_timeout=240, + auth_username=tornado.options.options.username, + auth_password=tornado.options.options.password) + response = tornado.httpclient.HTTPClient().fetch(request) + logging.info('Deleting index "%s" done %s' % (idx_name, response.body)) + except tornado.httpclient.HTTPError: + pass + + +def create_index(idx_name): + schema = { + "settings": { + "number_of_shards": tornado.options.options.num_of_shards, + "number_of_replicas": tornado.options.options.num_of_replicas + }, + "refresh": True + } + + body = json.dumps(schema) + url = "%s/%s" % (tornado.options.options.es_url, idx_name) + try: + logging.info('Trying to create index %s' % (url)) + request = tornado.httpclient.HTTPRequest(url, method="PUT", body=body, request_timeout=240, + auth_username=tornado.options.options.username, + auth_password=tornado.options.options.password) + response = tornado.httpclient.HTTPClient().fetch(request) + logging.info('Creating index "%s" done %s' % (idx_name, response.body)) + except tornado.httpclient.HTTPError: + logging.info('Looks like the index exists already') + pass + + +@tornado.gen.coroutine +def upload_batch(upload_data_txt): + try: + request = tornado.httpclient.HTTPRequest(tornado.options.options.es_url + "/_bulk", + method="POST", body=upload_data_txt, + request_timeout= + tornado.options.options.http_upload_timeout, + auth_username=tornado.options.options.username, + auth_password=tornado.options.options.password) + response = yield async_http_client.fetch(request) + except Exception as ex: + logging.error("upload failed, error: %s" % ex) + return + + result = json.loads(response.body.decode('utf-8')) + res_txt = "OK" if not result['errors'] else "FAILED" + took = int(result['took']) + logging.info("Upload: %s - upload took: %5dms, total docs uploaded: %7d" % (res_txt, took, + upload_data_count)) + + +def get_data_for_format(format,count): + split_f = format.split(":") + if not split_f: + return None, None + + field_name = split_f[0] + field_type = split_f[1] + + return_val = '' + + if field_type == "bool": + if count%2 == 0: + return_val = True + else: + return_val = False + + elif field_type == "str": + return_val = field_name + str(count) + + elif field_type == "int": + return_val = count + + elif field_type == "ipv4": + return_val = "{0}.{1}.{2}.{3}".format(1,2,3,count%255) + + elif field_type in ["ts", "tstxt"]: + return_val = int(count * 1000) if field_type == "ts" else\ + datetime.datetime.fromtimestamp(count)\ + .strftime("%Y-%m-%dT%H:%M:%S.000-0000") + + elif field_type == "words": + return_val = field_name + str(count) + + elif field_type == "dict": + mydict = dict(a=field_name + str(count), b=field_name + str(count), c=field_name + str(count), + d=field_name + str(count), e=field_name + str(count), f=field_name + str(count), + g=field_name + str(count), h=field_name + str(count), i=field_name + str(count), + j=field_name + str(count)) + return_val = ", ".join("=".join(_) for _ in mydict.items()) + + elif field_type == "text": + return_val = field_name + str(count) + + return field_name, return_val + + +def generate_count(min, max): + if min == max: + return max + elif min > max: + return random.randrange(max, min); + else: + return random.randrange(min, max); + + +def generate_random_doc(format,count): + global id_counter + + res = {} + + for f in format: + f_key, f_val = get_data_for_format(f,count) + if f_key: + res[f_key] = f_val + + if not tornado.options.options.id_type: + return res + + if tornado.options.options.id_type == 'int': + res['_id'] = id_counter + id_counter += 1 + elif tornado.options.options.id_type == 'uuid4': + res['_id'] = str(uuid.uuid4()) + + return res + + +def set_index_refresh(val): + + params = {"index": {"refresh_interval": val}} + body = json.dumps(params) + url = "%s/%s/_settings" % (tornado.options.options.es_url, tornado.options.options.index_name) + try: + request = tornado.httpclient.HTTPRequest(url, method="PUT", body=body, request_timeout=240, + auth_username=tornado.options.options.username, + auth_password=tornado.options.options.password) + http_client = tornado.httpclient.HTTPClient() + http_client.fetch(request) + logging.info('Set index refresh to %s' % val) + except Exception as ex: + logging.exception(ex) + + +@tornado.gen.coroutine +def generate_test_data(): + + global upload_data_count + + if tornado.options.options.force_init_index: + delete_index(tornado.options.options.index_name) + + create_index(tornado.options.options.index_name) + + # todo: query what refresh is set to, then restore later + if tornado.options.options.set_refresh: + set_index_refresh("-1") + + if tornado.options.options.out_file: + out_file = open(tornado.options.options.out_file, "w") + else: + out_file = None + + if tornado.options.options.dict_file: + global _dict_data + with open(tornado.options.options.dict_file, 'r') as f: + _dict_data = f.readlines() + logging.info("Loaded %d words from the %s" % (len(_dict_data), + tornado.options.options.dict_file)) + + format = tornado.options.options.format.split(',') + if not format: + logging.error('invalid format') + exit(1) + + ts_start = int(time.time()) + upload_data_txt = "" + total_uploaded = 0 + + logging.info("Generating %d docs, upload batch size is %d" % (tornado.options.options.count, + tornado.options + .options.batch_size)) + for num in range(0, tornado.options.options.count): + + item = generate_random_doc(format,num) + + if out_file: + out_file.write("%s\n" % json.dumps(item)) + + cmd = {'index': {'_index': tornado.options.options.index_name, + '_type': tornado.options.options.index_type}} + if '_id' in item: + cmd['index']['_id'] = item['_id'] + + upload_data_txt += json.dumps(cmd) + "\n" + upload_data_txt += json.dumps(item) + "\n" + upload_data_count += 1 + + if upload_data_count % tornado.options.options.batch_size == 0: + yield upload_batch(upload_data_txt) + upload_data_txt = "" + + # upload remaining items in `upload_data_txt` + if upload_data_txt: + yield upload_batch(upload_data_txt) + + if tornado.options.options.set_refresh: + set_index_refresh("1s") + + if out_file: + out_file.close() + + took_secs = int(time.time() - ts_start) + + logging.info("Done - total docs uploaded: %d, took %d seconds" % + (tornado.options.options.count, took_secs)) + + +if __name__ == '__main__': + tornado.options.define("es_url", type=str, default='http://localhost:9200/', + help="URL of your Elasticsearch node") + tornado.options.define("index_name", type=str, default='test_data', + help="Name of the index to store your messages") + tornado.options.define("index_type", type=str, default='test_type', help="Type") + tornado.options.define("batch_size", type=int, default=1000, + help="Elasticsearch bulk index batch size") + tornado.options.define("num_of_shards", type=int, default=2, + help="Number of shards for ES index") + tornado.options.define("http_upload_timeout", type=int, default=3, + help="Timeout in seconds when uploading data") + tornado.options.define("count", type=int, default=100000, help="Number of docs to generate") + tornado.options.define("format", type=str, default='name:str,age:int,last_updated:ts', + help="message format") + tornado.options.define("num_of_replicas", type=int, default=0, + help="Number of replicas for ES index") + tornado.options.define("force_init_index", type=bool, default=False, + help="Force deleting and re-initializing the Elasticsearch index") + tornado.options.define("set_refresh", type=bool, default=False, + help="Set refresh rate to -1 before starting the upload") + tornado.options.define("out_file", type=str, default=False, + help="If set, write test data to out_file as well.") + tornado.options.define("id_type", type=str, default=None, + help="Type of 'id' to use for the docs, \ + valid settings are int and uuid4, None is default") + tornado.options.define("dict_file", type=str, default=None, + help="Name of dictionary file to use") + tornado.options.define("username", type=str, default=None, help="Username for elasticsearch") + tornado.options.define("password", type=str, default=None, help="Password for elasticsearch") + tornado.options.parse_command_line() + + tornado.ioloop.IOLoop.instance().run_sync(generate_test_data) diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/show-health.sh b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/show-health.sh new file mode 100644 index 0000000000000..8fa912cb24dda --- /dev/null +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/resources/kubernetes/elasticsearch/show-health.sh @@ -0,0 +1,25 @@ +# 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. + +#!/bin/sh +set -e + +external_ip="$(kubectl get svc elasticsearch -o jsonpath='{.status.loadBalancer.ingress[0].ip}')" + +echo "Elasticsearch cluster health info" +echo "---------------------------------" +curl $external_ip:9200/_cluster/health +echo # empty line since curl doesn't output CRLF \ No newline at end of file diff --git a/sdks/java/io/hadoop/pom.xml b/sdks/java/io/hadoop/pom.xml new file mode 100644 index 0000000000000..1982c25551d64 --- /dev/null +++ b/sdks/java/io/hadoop/pom.xml @@ -0,0 +1,53 @@ + + + + 4.0.0 + + org.apache.beam + beam-sdks-java-io-parent + 0.7.0-SNAPSHOT + ../pom.xml + + pom + beam-sdks-java-io-hadoop-parent + Apache Beam :: SDKs :: Java :: IO :: Hadoop + Parent for Beam SDK Hadoop IO which reads data from any source which implements Hadoop Input Format. + + + jdk1.8-tests + input-format + + + + + org.apache.beam + beam-sdks-java-core + + + + junit + junit + test + + + org.mockito + mockito-all + test + + + \ No newline at end of file diff --git a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java index 7bfdd253708ca..ee3369ea6b9f3 100644 --- a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java +++ b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java @@ -32,7 +32,6 @@ import org.apache.beam.sdk.io.hbase.HBaseIO.HBaseSource; import org.apache.beam.sdk.io.range.ByteKey; import org.apache.beam.sdk.io.range.ByteKeyRange; -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.Count; @@ -68,7 +67,6 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; -import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -157,7 +155,6 @@ public void testWriteValidationFailsMissingConfiguration() { /** Tests that when reading from a non-existent table, the read fails. */ @Test - @Category(NeedsRunner.class) public void testReadingFailsTableDoesNotExist() throws Exception { final String table = "TEST-TABLE-INVALID"; // Exception will be thrown by read.validate() when read is applied. @@ -169,7 +166,6 @@ public void testReadingFailsTableDoesNotExist() throws Exception { /** Tests that when reading from an empty table, the read succeeds. */ @Test - @Category(NeedsRunner.class) public void testReadingEmptyTable() throws Exception { final String table = "TEST-EMPTY-TABLE"; createTable(table); @@ -178,7 +174,6 @@ public void testReadingEmptyTable() throws Exception { } @Test - @Category(NeedsRunner.class) public void testReading() throws Exception { final String table = "TEST-MANY-ROWS-TABLE"; final int numRows = 1001; @@ -213,7 +208,6 @@ public void testReadingWithSplits() throws Exception { /** Tests reading all rows using a filter. */ @Test - @Category(NeedsRunner.class) public void testReadingWithFilter() throws Exception { final String table = "TEST-FILTER-TABLE"; final int numRows = 1001; @@ -234,7 +228,6 @@ public void testReadingWithFilter() throws Exception { * range [] and that some properties hold across them. */ @Test - @Category(NeedsRunner.class) public void testReadingWithKeyRange() throws Exception { final String table = "TEST-KEY-RANGE-TABLE"; final int numRows = 1001; @@ -271,7 +264,6 @@ public void testReadingDisplayData() { /** Tests that a record gets written to the service and messages are logged. */ @Test - @Category(NeedsRunner.class) public void testWriting() throws Exception { final String table = "table"; final String key = "key"; @@ -304,7 +296,6 @@ public void testWritingFailsTableDoesNotExist() throws Exception { /** Tests that when writing an element fails, the write fails. */ @Test - @Category(NeedsRunner.class) public void testWritingFailsBadElement() throws Exception { final String table = "TEST-TABLE"; final String key = "KEY"; diff --git a/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java b/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java index f07247d578fa8..87402045ac35a 100644 --- a/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java +++ b/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java @@ -44,7 +44,6 @@ import org.apache.activemq.store.memory.MemoryPersistenceAdapter; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -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.Count; @@ -54,7 +53,6 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -122,7 +120,6 @@ private void runPipelineExpectingJmsConnectException(String innerMessage) { } @Test - @Category(NeedsRunner.class) public void testAuthenticationRequired() { pipeline.apply( JmsIO.read() @@ -133,7 +130,6 @@ public void testAuthenticationRequired() { } @Test - @Category(NeedsRunner.class) public void testAuthenticationWithBadPassword() { pipeline.apply( JmsIO.read() @@ -147,7 +143,6 @@ public void testAuthenticationWithBadPassword() { } @Test - @Category(NeedsRunner.class) public void testReadMessages() throws Exception { // produce message @@ -187,7 +182,6 @@ public void testReadMessages() throws Exception { } @Test - @Category(NeedsRunner.class) public void testWriteMessage() throws Exception { ArrayList data = new ArrayList<>(); diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java index 7e77512d785ca..ecbc71d48e5ae 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java @@ -48,7 +48,6 @@ import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.options.PipelineOptionsFactory; -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.Count; @@ -81,7 +80,6 @@ import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; -import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -281,7 +279,6 @@ public static void addCountingAsserts(PCollection input, long numElements) } @Test - @Category(NeedsRunner.class) public void testUnboundedSource() { int numElements = 1000; @@ -319,7 +316,6 @@ public void testUnboundedSourceWithSingleTopic() { } @Test - @Category(NeedsRunner.class) public void testUnboundedSourceWithExplicitPartitions() { int numElements = 1000; @@ -358,7 +354,6 @@ public void processElement(ProcessContext c) throws Exception { } @Test - @Category(NeedsRunner.class) public void testUnboundedSourceTimestamps() { int numElements = 1000; @@ -386,7 +381,6 @@ public void processElement(ProcessContext ctx) throws Exception { } @Test - @Category(NeedsRunner.class) public void testUnboundedSourceSplits() throws Exception { int numElements = 1000; diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java index 994be87a6d64e..71718e3506a1b 100644 --- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java @@ -63,7 +63,6 @@ import org.apache.beam.sdk.io.mongodb.MongoDbGridFSIO.WriteFn; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.testing.TestPipeline; @@ -80,7 +79,6 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; -import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -183,7 +181,6 @@ public static void stop() throws Exception { } @Test - @Category(NeedsRunner.class) public void testFullRead() throws Exception { PCollection output = pipeline.apply( @@ -212,7 +209,6 @@ public Void apply(Iterable> input) { @Test - @Category(NeedsRunner.class) public void testReadWithParser() throws Exception { PCollection> output = pipeline.apply( @@ -295,7 +291,6 @@ public void testSplit() throws Exception { @Test - @Category(NeedsRunner.class) public void testWriteMessage() throws Exception { ArrayList data = new ArrayList<>(100); diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java index e7ff712e0b4c1..c36b7c8d344f8 100644 --- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java @@ -41,7 +41,6 @@ import java.net.ServerSocket; import java.util.ArrayList; -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.Count; @@ -59,7 +58,6 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; -import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -145,7 +143,6 @@ public void stop() throws Exception { } @Test - @Category(NeedsRunner.class) public void testFullRead() throws Exception { PCollection output = pipeline.apply( @@ -178,7 +175,6 @@ public Void apply(Iterable> input) { } @Test - @Category(NeedsRunner.class) public void testReadWithFilter() throws Exception { PCollection output = pipeline.apply( @@ -195,7 +191,6 @@ public void testReadWithFilter() throws Exception { } @Test - @Category(NeedsRunner.class) public void testWrite() throws Exception { ArrayList data = new ArrayList<>(); diff --git a/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java b/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java index da89aa28fcc33..1b3d2da354276 100644 --- a/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java +++ b/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java @@ -29,7 +29,6 @@ import org.apache.beam.sdk.io.mqtt.MqttIO.Read; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; import org.fusesource.hawtbuf.Buffer; @@ -42,7 +41,6 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,7 +76,6 @@ public void startBroker() throws Exception { } @Test(timeout = 60 * 1000) - @Category(ValidatesRunner.class) public void testReadNoClientId() throws Exception { final String topicName = "READ_TOPIC_NO_CLIENT_ID"; Read mqttReader = MqttIO.read() @@ -138,7 +135,6 @@ public void run() { } @Test(timeout = 60 * 1000) - @Category(ValidatesRunner.class) public void testRead() throws Exception { PCollection output = pipeline.apply( MqttIO.read() @@ -198,7 +194,6 @@ public void run() { } @Test - @Category(ValidatesRunner.class) public void testWrite() throws Exception { MQTT client = new MQTT(); client.setHost("tcp://localhost:" + port); diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index 73fbba17f1eca..27fc6148a2f0c 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -68,6 +68,7 @@ elasticsearch google-cloud-platform hadoop-common + hadoop hbase hdfs jdbc @@ -114,5 +115,4 @@ - diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index 9f5a97ac121a1..5955317404dd3 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -188,8 +188,8 @@ def to_runner_api(self, context): # TODO(BEAM-115): Use specialized URNs and components. from apache_beam.runners.api import beam_runner_api_pb2 return beam_runner_api_pb2.Coder( - spec=beam_runner_api_pb2.FunctionSpec( - spec=beam_runner_api_pb2.UrnWithParameter( + spec=beam_runner_api_pb2.SdkFunctionSpec( + spec=beam_runner_api_pb2.FunctionSpec( urn=urns.PICKLED_CODER, parameter=proto_utils.pack_Any( google.protobuf.wrappers_pb2.BytesValue( diff --git a/sdks/python/apache_beam/examples/wordcount.py b/sdks/python/apache_beam/examples/wordcount.py index 50c03286dcdb2..27b9dcb408163 100644 --- a/sdks/python/apache_beam/examples/wordcount.py +++ b/sdks/python/apache_beam/examples/wordcount.py @@ -19,7 +19,6 @@ from __future__ import absolute_import -import argparse import logging import re @@ -67,24 +66,29 @@ def process(self, element): def run(argv=None): """Main entry point; defines and runs the wordcount pipeline.""" - parser = argparse.ArgumentParser() - parser.add_argument('--input', - dest='input', - default='gs://dataflow-samples/shakespeare/kinglear.txt', - help='Input file to process.') - parser.add_argument('--output', - dest='output', - required=True, - help='Output file to write results to.') - known_args, pipeline_args = parser.parse_known_args(argv) + class WordcountOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_value_provider_argument( + '--input', + dest='input', + default='gs://dataflow-samples/shakespeare/kinglear.txt', + help='Input file to process.') + parser.add_value_provider_argument( + '--output', + dest='output', + required=True, + help='Output file to write results to.') + pipeline_options = PipelineOptions(argv) + wordcount_options = pipeline_options.view_as(WordcountOptions) + # We use the save_main_session option because one or more DoFn's in this # workflow rely on global context (e.g., a module imported at module level). - pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True p = beam.Pipeline(options=pipeline_options) # Read the text file[pattern] into a PCollection. - lines = p | 'read' >> ReadFromText(known_args.input) + lines = p | 'read' >> ReadFromText(wordcount_options.input) # Count the occurrences of each word. counts = (lines @@ -99,7 +103,7 @@ def run(argv=None): # Write the output using a "Write" transform that has side effects. # pylint: disable=expression-not-assigned - output | 'write' >> WriteToText(known_args.output) + output | 'write' >> WriteToText(wordcount_options.output) # Actually run the pipeline (all operations above are deferred). result = p.run() diff --git a/sdks/python/apache_beam/internal/gcp/json_value.py b/sdks/python/apache_beam/internal/gcp/json_value.py index c8b5393983f40..4099c1ad5ecad 100644 --- a/sdks/python/apache_beam/internal/gcp/json_value.py +++ b/sdks/python/apache_beam/internal/gcp/json_value.py @@ -25,6 +25,8 @@ extra_types = None # pylint: enable=wrong-import-order, wrong-import-position +from apache_beam.utils.value_provider import ValueProvider + _MAXINT64 = (1 << 63) - 1 _MININT64 = - (1 << 63) @@ -104,6 +106,10 @@ def to_json_value(obj, with_type=False): raise TypeError('Can not encode {} as a 64-bit integer'.format(obj)) elif isinstance(obj, float): return extra_types.JsonValue(double_value=obj) + elif isinstance(obj, ValueProvider): + if obj.is_accessible(): + return to_json_value(obj.get()) + return extra_types.JsonValue(is_null=True) else: raise TypeError('Cannot convert %s to a JSON value.' % repr(obj)) diff --git a/sdks/python/apache_beam/io/filebasedsource.py b/sdks/python/apache_beam/io/filebasedsource.py index a3e066795086b..2e7043f8073cd 100644 --- a/sdks/python/apache_beam/io/filebasedsource.py +++ b/sdks/python/apache_beam/io/filebasedsource.py @@ -32,6 +32,10 @@ from apache_beam.io.filesystem import CompressionTypes from apache_beam.io.filesystems_util import get_filesystem from apache_beam.transforms.display import DisplayDataItem +from apache_beam.utils.value_provider import ValueProvider +from apache_beam.utils.value_provider import StaticValueProvider +from apache_beam.utils.value_provider import check_accessible + MAX_NUM_THREADS_FOR_SIZE_ESTIMATION = 25 @@ -50,7 +54,8 @@ def __init__(self, """Initializes ``FileBasedSource``. Args: - file_pattern: the file glob to read. + file_pattern: the file glob to read a string or a ValueProvider + (placeholder to inject a runtime value). min_bundle_size: minimum size of bundles that should be generated when performing initial splitting on this source. compression_type: compression type to use @@ -68,17 +73,25 @@ def __init__(self, creation time. Raises: TypeError: when compression_type is not valid or if file_pattern is not a - string. + string or a ValueProvider. ValueError: when compression and splittable files are specified. IOError: when the file pattern specified yields an empty result. """ - if not isinstance(file_pattern, basestring): - raise TypeError( - '%s: file_pattern must be a string; got %r instead' % - (self.__class__.__name__, file_pattern)) + if (not (isinstance(file_pattern, basestring) + or isinstance(file_pattern, ValueProvider))): + raise TypeError('%s: file_pattern must be of type string' + ' or ValueProvider; got %r instead' + % (self.__class__.__name__, file_pattern)) + + if isinstance(file_pattern, basestring): + file_pattern = StaticValueProvider(str, file_pattern) self._pattern = file_pattern - self._file_system = get_filesystem(file_pattern) + if file_pattern.is_accessible(): + self._file_system = get_filesystem(file_pattern.get()) + else: + self._file_system = None + self._concat_source = None self._min_bundle_size = min_bundle_size if not CompressionTypes.is_valid_compression_type(compression_type): @@ -91,19 +104,24 @@ def __init__(self, else: # We can't split compressed files efficiently so turn off splitting. self._splittable = False - if validate: + if validate and file_pattern.is_accessible(): self._validate() def display_data(self): - return {'file_pattern': DisplayDataItem(self._pattern, + return {'file_pattern': DisplayDataItem(str(self._pattern), label="File Pattern"), 'compression': DisplayDataItem(str(self._compression_type), label='Compression Type')} + @check_accessible(['_pattern']) def _get_concat_source(self): if self._concat_source is None: + pattern = self._pattern.get() + single_file_sources = [] - match_result = self._file_system.match([self._pattern])[0] + if self._file_system is None: + self._file_system = get_filesystem(pattern) + match_result = self._file_system.match([pattern])[0] files_metadata = match_result.metadata_list # We create a reference for FileBasedSource that will be serialized along @@ -142,14 +160,19 @@ def open_file(self, file_name): file_name, 'application/octet-stream', compression_type=self._compression_type) + @check_accessible(['_pattern']) def _validate(self): """Validate if there are actual files in the specified glob pattern """ + pattern = self._pattern.get() + if self._file_system is None: + self._file_system = get_filesystem(pattern) + # Limit the responses as we only want to check if something exists - match_result = self._file_system.match([self._pattern], limits=[1])[0] + match_result = self._file_system.match([pattern], limits=[1])[0] if len(match_result.metadata_list) <= 0: raise IOError( - 'No files found based on the file pattern %s' % self._pattern) + 'No files found based on the file pattern %s' % pattern) def split( self, desired_bundle_size=None, start_position=None, stop_position=None): @@ -158,8 +181,12 @@ def split( start_position=start_position, stop_position=stop_position) + @check_accessible(['_pattern']) def estimate_size(self): - match_result = self._file_system.match([self._pattern])[0] + pattern = self._pattern.get() + if self._file_system is None: + self._file_system = get_filesystem(pattern) + match_result = self._file_system.match([pattern])[0] return sum([f.size_in_bytes for f in match_result.metadata_list]) def read(self, range_tracker): @@ -184,7 +211,7 @@ def read_records(self, file_name, offset_range_tracker): defined by a given ``RangeTracker``. Returns: - a iterator that gives the records read from the given file. + an iterator that gives the records read from the given file. """ raise NotImplementedError diff --git a/sdks/python/apache_beam/io/filebasedsource_test.py b/sdks/python/apache_beam/io/filebasedsource_test.py index 7b7ec8a680695..c25ca5de29ca3 100644 --- a/sdks/python/apache_beam/io/filebasedsource_test.py +++ b/sdks/python/apache_beam/io/filebasedsource_test.py @@ -43,6 +43,8 @@ from apache_beam.transforms.display_test import DisplayDataItemMatcher from apache_beam.transforms.util import assert_that from apache_beam.transforms.util import equal_to +from apache_beam.utils.value_provider import StaticValueProvider +from apache_beam.utils.value_provider import RuntimeValueProvider class LineSource(FileBasedSource): @@ -221,6 +223,28 @@ def setUp(self): # environments with limited amount of resources. filebasedsource.MAX_NUM_THREADS_FOR_SIZE_ESTIMATION = 2 + def test_string_or_value_provider_only(self): + str_file_pattern = tempfile.NamedTemporaryFile(delete=False).name + self.assertEqual(str_file_pattern, + FileBasedSource(str_file_pattern)._pattern.value) + + static_vp_file_pattern = StaticValueProvider(value_type=str, + value=str_file_pattern) + self.assertEqual(static_vp_file_pattern, + FileBasedSource(static_vp_file_pattern)._pattern) + + runtime_vp_file_pattern = RuntimeValueProvider( + option_name='arg', + value_type=str, + default_value=str_file_pattern, + options_id=1) + self.assertEqual(runtime_vp_file_pattern, + FileBasedSource(runtime_vp_file_pattern)._pattern) + + invalid_file_pattern = 123 + with self.assertRaises(TypeError): + FileBasedSource(invalid_file_pattern) + def test_validation_file_exists(self): file_name, _ = write_data(10) LineSource(file_name) diff --git a/sdks/python/apache_beam/io/fileio.py b/sdks/python/apache_beam/io/fileio.py index f33942a846ff0..84949dc327c95 100644 --- a/sdks/python/apache_beam/io/fileio.py +++ b/sdks/python/apache_beam/io/fileio.py @@ -14,6 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # + """File-based sources and sinks.""" from __future__ import absolute_import @@ -30,6 +31,9 @@ from apache_beam.io.filesystem import CompressionTypes from apache_beam.io.filesystems_util import get_filesystem from apache_beam.transforms.display import DisplayDataItem +from apache_beam.utils.value_provider import ValueProvider +from apache_beam.utils.value_provider import StaticValueProvider +from apache_beam.utils.value_provider import check_accessible MAX_BATCH_OPERATION_SIZE = 100 DEFAULT_SHARD_NAME_TEMPLATE = '-SSSSS-of-NNNNN' @@ -149,25 +153,30 @@ def __init__(self, compression_type=CompressionTypes.AUTO): """ Raises: - TypeError: if file path parameters are not a string or if compression_type - is not member of CompressionTypes. + TypeError: if file path parameters are not a string or ValueProvider, + or if compression_type is not member of CompressionTypes. ValueError: if shard_name_template is not of expected format. """ - if not isinstance(file_path_prefix, basestring): - raise TypeError('file_path_prefix must be a string; got %r instead' % - file_path_prefix) - if not isinstance(file_name_suffix, basestring): - raise TypeError('file_name_suffix must be a string; got %r instead' % - file_name_suffix) + if not (isinstance(file_path_prefix, basestring) + or isinstance(file_path_prefix, ValueProvider)): + raise TypeError('file_path_prefix must be a string or ValueProvider;' + 'got %r instead' % file_path_prefix) + if not (isinstance(file_name_suffix, basestring) + or isinstance(file_name_suffix, ValueProvider)): + raise TypeError('file_name_suffix must be a string or ValueProvider;' + 'got %r instead' % file_name_suffix) if not CompressionTypes.is_valid_compression_type(compression_type): raise TypeError('compression_type must be CompressionType object but ' 'was %s' % type(compression_type)) - if shard_name_template is None: shard_name_template = DEFAULT_SHARD_NAME_TEMPLATE elif shard_name_template is '': num_shards = 1 + if isinstance(file_path_prefix, basestring): + file_path_prefix = StaticValueProvider(str, file_path_prefix) + if isinstance(file_name_suffix, basestring): + file_name_suffix = StaticValueProvider(str, file_name_suffix) self.file_path_prefix = file_path_prefix self.file_name_suffix = file_name_suffix self.num_shards = num_shards @@ -175,7 +184,10 @@ def __init__(self, self.shard_name_format = self._template_to_format(shard_name_template) self.compression_type = compression_type self.mime_type = mime_type - self._file_system = get_filesystem(file_path_prefix) + if file_path_prefix.is_accessible(): + self._file_system = get_filesystem(file_path_prefix.get()) + else: + self._file_system = None def display_data(self): return {'shards': @@ -189,12 +201,15 @@ def display_data(self): self.file_name_suffix), label='File Pattern')} + @check_accessible(['file_path_prefix']) def open(self, temp_path): """Opens ``temp_path``, returning an opaque file handle object. The returned file handle is passed to ``write_[encoded_]record`` and ``close``. """ + if self._file_system is None: + self._file_system = get_filesystem(self.file_path_prefix.get()) return self._file_system.create(temp_path, self.mime_type, self.compression_type) @@ -221,22 +236,33 @@ def close(self, file_handle): if file_handle is not None: file_handle.close() + @check_accessible(['file_path_prefix', 'file_name_suffix']) def initialize_write(self): - tmp_dir = self.file_path_prefix + self.file_name_suffix + time.strftime( + file_path_prefix = self.file_path_prefix.get() + file_name_suffix = self.file_name_suffix.get() + tmp_dir = file_path_prefix + file_name_suffix + time.strftime( '-temp-%Y-%m-%d_%H-%M-%S') + if self._file_system is None: + self._file_system = get_filesystem(file_path_prefix) self._file_system.mkdirs(tmp_dir) return tmp_dir + @check_accessible(['file_path_prefix', 'file_name_suffix']) def open_writer(self, init_result, uid): # A proper suffix is needed for AUTO compression detection. # We also ensure there will be no collisions with uid and a # (possibly unsharded) file_path_prefix and a (possibly empty) # file_name_suffix. + file_path_prefix = self.file_path_prefix.get() + file_name_suffix = self.file_name_suffix.get() suffix = ( - '.' + os.path.basename(self.file_path_prefix) + self.file_name_suffix) + '.' + os.path.basename(file_path_prefix) + file_name_suffix) return FileSinkWriter(self, os.path.join(init_result, uid) + suffix) + @check_accessible(['file_path_prefix', 'file_name_suffix']) def finalize_write(self, init_result, writer_results): + file_path_prefix = self.file_path_prefix.get() + file_name_suffix = self.file_name_suffix.get() writer_results = sorted(writer_results) num_shards = len(writer_results) min_threads = min(num_shards, FileSink._MAX_RENAME_THREADS) @@ -246,8 +272,8 @@ def finalize_write(self, init_result, writer_results): destination_files = [] for shard_num, shard in enumerate(writer_results): final_name = ''.join([ - self.file_path_prefix, self.shard_name_format % dict( - shard_num=shard_num, num_shards=num_shards), self.file_name_suffix + file_path_prefix, self.shard_name_format % dict( + shard_num=shard_num, num_shards=num_shards), file_name_suffix ]) source_files.append(shard) destination_files.append(final_name) @@ -270,6 +296,8 @@ def _rename_batch(batch): """_rename_batch executes batch rename operations.""" source_files, destination_files = batch exceptions = [] + if self._file_system is None: + self._file_system = get_filesystem(file_path_prefix) try: self._file_system.rename(source_files, destination_files) return exceptions diff --git a/sdks/python/apache_beam/io/fileio_test.py b/sdks/python/apache_beam/io/fileio_test.py index 3b4f61f84d075..2409873c9fb20 100644 --- a/sdks/python/apache_beam/io/fileio_test.py +++ b/sdks/python/apache_beam/io/fileio_test.py @@ -34,6 +34,8 @@ from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher +from apache_beam.utils.value_provider import StaticValueProvider + # TODO: Refactor code so all io tests are using same library # TestCaseWithTempDirCleanup class. @@ -93,7 +95,7 @@ class TestFileSink(_TestCaseWithTempDirCleanUp): def test_file_sink_writing(self): temp_path = os.path.join(self._new_tempdir(), 'filesink') sink = MyFileSink( - temp_path, file_name_suffix='.foo', coder=coders.ToStringCoder()) + temp_path, file_name_suffix='.output', coder=coders.ToStringCoder()) # Manually invoke the generic Sink API. init_token = sink.initialize_write() @@ -114,8 +116,8 @@ def test_file_sink_writing(self): res = list(sink.finalize_write(init_token, [res1, res2])) # Check the results. - shard1 = temp_path + '-00000-of-00002.foo' - shard2 = temp_path + '-00001-of-00002.foo' + shard1 = temp_path + '-00000-of-00002.output' + shard2 = temp_path + '-00001-of-00002.output' self.assertEqual(res, [shard1, shard2]) self.assertEqual(open(shard1).read(), '[start][a][b][end]') self.assertEqual(open(shard2).read(), '[start][x][y][z][end]') @@ -126,33 +128,48 @@ def test_file_sink_writing(self): def test_file_sink_display_data(self): temp_path = os.path.join(self._new_tempdir(), 'display') sink = MyFileSink( - temp_path, file_name_suffix='.foo', coder=coders.ToStringCoder()) + temp_path, file_name_suffix='.output', coder=coders.ToStringCoder()) dd = DisplayData.create_from(sink) expected_items = [ DisplayDataItemMatcher( 'compression', 'auto'), DisplayDataItemMatcher( 'file_pattern', - '{}{}'.format(temp_path, - '-%(shard_num)05d-of-%(num_shards)05d.foo'))] - + '{}{}'.format( + temp_path, + '-%(shard_num)05d-of-%(num_shards)05d.output'))] hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items)) def test_empty_write(self): temp_path = tempfile.NamedTemporaryFile().name sink = MyFileSink( - temp_path, file_name_suffix='.foo', coder=coders.ToStringCoder()) + temp_path, file_name_suffix='.output', coder=coders.ToStringCoder() + ) + p = TestPipeline() + p | beam.Create([]) | beam.io.Write(sink) # pylint: disable=expression-not-assigned + p.run() + self.assertEqual( + open(temp_path + '-00000-of-00001.output').read(), '[start][end]') + + def test_static_value_provider_empty_write(self): + temp_path = StaticValueProvider(value_type=str, + value=tempfile.NamedTemporaryFile().name) + sink = MyFileSink( + temp_path, + file_name_suffix=StaticValueProvider(value_type=str, value='.output'), + coder=coders.ToStringCoder() + ) p = TestPipeline() p | beam.Create([]) | beam.io.Write(sink) # pylint: disable=expression-not-assigned p.run() self.assertEqual( - open(temp_path + '-00000-of-00001.foo').read(), '[start][end]') + open(temp_path.get() + '-00000-of-00001.output').read(), '[start][end]') def test_fixed_shard_write(self): temp_path = os.path.join(self._new_tempdir(), 'empty') sink = MyFileSink( temp_path, - file_name_suffix='.foo', + file_name_suffix='.output', num_shards=3, shard_name_template='_NN_SSS_', coder=coders.ToStringCoder()) @@ -162,7 +179,7 @@ def test_fixed_shard_write(self): p.run() concat = ''.join( - open(temp_path + '_03_%03d_.foo' % shard_num).read() + open(temp_path + '_03_%03d_.output' % shard_num).read() for shard_num in range(3)) self.assertTrue('][a][' in concat, concat) self.assertTrue('][b][' in concat, concat) @@ -170,7 +187,7 @@ def test_fixed_shard_write(self): def test_file_sink_multi_shards(self): temp_path = os.path.join(self._new_tempdir(), 'multishard') sink = MyFileSink( - temp_path, file_name_suffix='.foo', coder=coders.ToStringCoder()) + temp_path, file_name_suffix='.output', coder=coders.ToStringCoder()) # Manually invoke the generic Sink API. init_token = sink.initialize_write() @@ -193,7 +210,7 @@ def test_file_sink_multi_shards(self): res = sorted(res_second) for i in range(num_shards): - shard_name = '%s-%05d-of-%05d.foo' % (temp_path, i, num_shards) + shard_name = '%s-%05d-of-%05d.output' % (temp_path, i, num_shards) uuid = 'uuid-%05d' % i self.assertEqual(res[i], shard_name) self.assertEqual( @@ -205,7 +222,7 @@ def test_file_sink_multi_shards(self): def test_file_sink_io_error(self): temp_path = os.path.join(self._new_tempdir(), 'ioerror') sink = MyFileSink( - temp_path, file_name_suffix='.foo', coder=coders.ToStringCoder()) + temp_path, file_name_suffix='.output', coder=coders.ToStringCoder()) # Manually invoke the generic Sink API. init_token = sink.initialize_write() diff --git a/sdks/python/apache_beam/io/gcp/gcsfilesystem.py b/sdks/python/apache_beam/io/gcp/gcsfilesystem.py index 5aef0ab37236b..d79630f7b112e 100644 --- a/sdks/python/apache_beam/io/gcp/gcsfilesystem.py +++ b/sdks/python/apache_beam/io/gcp/gcsfilesystem.py @@ -65,7 +65,7 @@ def _match(pattern, limit): """ if pattern.endswith('/'): pattern += '*' - file_sizes = gcsio.GcsIO().size_of_files_in_glob(pattern) + file_sizes = gcsio.GcsIO().size_of_files_in_glob(pattern, limit) metadata_list = [FileMetadata(path, size) for path, size in file_sizes.iteritems()] return MatchResult(pattern, metadata_list) diff --git a/sdks/python/apache_beam/io/gcp/gcsfilesystem_test.py b/sdks/python/apache_beam/io/gcp/gcsfilesystem_test.py index 73a3893b89b0a..5a1f10d4f3ba6 100644 --- a/sdks/python/apache_beam/io/gcp/gcsfilesystem_test.py +++ b/sdks/python/apache_beam/io/gcp/gcsfilesystem_test.py @@ -54,7 +54,31 @@ def test_match_multiples(self, mock_gcsio): self.assertEqual( set(match_result.metadata_list), expected_results) - gcsio_mock.size_of_files_in_glob.assert_called_once_with('gs://bucket/*') + gcsio_mock.size_of_files_in_glob.assert_called_once_with( + 'gs://bucket/*', None) + + @mock.patch('apache_beam.io.gcp.gcsfilesystem.gcsio') + def test_match_multiples_limit(self, mock_gcsio): + # Prepare mocks. + gcsio_mock = mock.MagicMock() + limit = 1 + gcsfilesystem.gcsio.GcsIO = lambda: gcsio_mock + gcsio_mock.size_of_files_in_glob.return_value = { + 'gs://bucket/file1': 1 + } + expected_results = set([ + FileMetadata('gs://bucket/file1', 1) + ]) + file_system = gcsfilesystem.GCSFileSystem() + match_result = file_system.match(['gs://bucket/'], [limit])[0] + self.assertEqual( + set(match_result.metadata_list), + expected_results) + self.assertEqual( + len(match_result.metadata_list), + limit) + gcsio_mock.size_of_files_in_glob.assert_called_once_with( + 'gs://bucket/*', 1) @mock.patch('apache_beam.io.gcp.gcsfilesystem.gcsio') def test_match_multiples_error(self, mock_gcsio): @@ -71,7 +95,8 @@ def test_match_multiples_error(self, mock_gcsio): self.assertTrue( error.exception.message.startswith('Match operation failed')) self.assertEqual(error.exception.exception_details, expected_results) - gcsio_mock.size_of_files_in_glob.assert_called_once_with('gs://bucket/*') + gcsio_mock.size_of_files_in_glob.assert_called_once_with( + 'gs://bucket/*', None) @mock.patch('apache_beam.io.gcp.gcsfilesystem.gcsio') def test_match_multiple_patterns(self, mock_gcsio): diff --git a/sdks/python/apache_beam/io/gcp/gcsio.py b/sdks/python/apache_beam/io/gcp/gcsio.py index 0a10094d3b8c3..c76c99d14c54b 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio.py +++ b/sdks/python/apache_beam/io/gcp/gcsio.py @@ -29,6 +29,7 @@ import Queue import re import threading +import time import traceback from apache_beam.utils import retry @@ -368,7 +369,7 @@ def size(self, path): @retry.with_exponential_backoff( retry_filter=retry.retry_on_server_errors_and_timeout_filter) - def size_of_files_in_glob(self, pattern): + def size_of_files_in_glob(self, pattern, limit=None): """Returns the size of all the files in the glob as a dictionary Args: @@ -379,16 +380,29 @@ def size_of_files_in_glob(self, pattern): prefix = re.match('^[^[*?]*', name_pattern).group(0) request = storage.StorageObjectsListRequest(bucket=bucket, prefix=prefix) file_sizes = {} + counter = 0 + start_time = time.time() + logging.info("Starting the size estimation of the input") while True: response = self.client.objects.List(request) for item in response.items: if fnmatch.fnmatch(item.name, name_pattern): file_name = 'gs://%s/%s' % (item.bucket, item.name) file_sizes[file_name] = item.size + counter += 1 + if limit is not None and counter >= limit: + break + if counter % 10000 == 0: + logging.info("Finished computing size of: %s files", len(file_sizes)) if response.nextPageToken: request.pageToken = response.nextPageToken + if limit is not None and len(file_sizes) >= limit: + break else: break + logging.info( + "Finished the size estimation of the input at %s files. " +\ + "Estimation took %s seconds", counter, time.time() - start_time) return file_sizes diff --git a/sdks/python/apache_beam/io/gcp/gcsio_test.py b/sdks/python/apache_beam/io/gcp/gcsio_test.py index c028f0d69a131..73d221376a5d4 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio_test.py +++ b/sdks/python/apache_beam/io/gcp/gcsio_test.py @@ -747,6 +747,45 @@ def test_size_of_files_in_glob(self): self.assertEqual( self.gcs.size_of_files_in_glob(file_pattern), expected_file_sizes) + def test_size_of_files_in_glob_limited(self): + bucket_name = 'gcsio-test' + object_names = [ + ('cow/cat/fish', 2), + ('cow/cat/blubber', 3), + ('cow/dog/blubber', 4), + ('apple/dog/blubber', 5), + ('apple/fish/blubber', 6), + ('apple/fish/blowfish', 7), + ('apple/fish/bambi', 8), + ('apple/fish/balloon', 9), + ('apple/fish/cat', 10), + ('apple/fish/cart', 11), + ('apple/fish/carl', 12), + ('apple/dish/bat', 13), + ('apple/dish/cat', 14), + ('apple/dish/carl', 15), + ] + for (object_name, size) in object_names: + file_name = 'gs://%s/%s' % (bucket_name, object_name) + self._insert_random_file(self.client, file_name, size) + test_cases = [ + ('gs://gcsio-test/cow/*', [ + ('cow/cat/fish', 2), + ('cow/cat/blubber', 3), + ('cow/dog/blubber', 4), + ]), + ('gs://gcsio-test/apple/fish/car?', [ + ('apple/fish/cart', 11), + ('apple/fish/carl', 12), + ]) + ] + # Check if limits are followed correctly + limit = 1 + for file_pattern, expected_object_names in test_cases: + expected_num_items = min(len(expected_object_names), limit) + self.assertEqual( + len(self.gcs.glob(file_pattern, limit)), expected_num_items) + @unittest.skipIf(HttpError is None, 'GCP dependencies are not installed') class TestPipeStream(unittest.TestCase): diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index fdb9a9dd865ee..b93167db29a22 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -513,7 +513,7 @@ def to_runner_api(self, context): from apache_beam.runners.api import beam_runner_api_pb2 return beam_runner_api_pb2.PTransform( unique_name=self.full_label, - spec=beam_runner_api_pb2.UrnWithParameter( + spec=beam_runner_api_pb2.FunctionSpec( urn=urns.PICKLED_TRANSFORM, parameter=proto_utils.pack_Any( wrappers_pb2.BytesValue(value=pickler.dumps(self.transform)))), diff --git a/sdks/python/apache_beam/runners/api/beam_runner_api_pb2.py b/sdks/python/apache_beam/runners/api/beam_runner_api_pb2.py index f235ce838f566..abba50dc26cb7 100644 --- a/sdks/python/apache_beam/runners/api/beam_runner_api_pb2.py +++ b/sdks/python/apache_beam/runners/api/beam_runner_api_pb2.py @@ -38,7 +38,7 @@ name='beam_runner_api.proto', package='org.apache.beam.runner_api.v1', syntax='proto3', - serialized_pb=_b('\n\x15\x62\x65\x61m_runner_api.proto\x12\x1dorg.apache.beam.runner_api.v1\x1a\x19google/protobuf/any.proto\"\x8d\x07\n\nComponents\x12M\n\ntransforms\x18\x01 \x03(\x0b\x32\x39.org.apache.beam.runner_api.v1.Components.TransformsEntry\x12Q\n\x0cpcollections\x18\x02 \x03(\x0b\x32;.org.apache.beam.runner_api.v1.Components.PcollectionsEntry\x12`\n\x14windowing_strategies\x18\x03 \x03(\x0b\x32\x42.org.apache.beam.runner_api.v1.Components.WindowingStrategiesEntry\x12\x45\n\x06\x63oders\x18\x04 \x03(\x0b\x32\x35.org.apache.beam.runner_api.v1.Components.CodersEntry\x12Q\n\x0c\x65nvironments\x18\x05 \x03(\x0b\x32;.org.apache.beam.runner_api.v1.Components.EnvironmentsEntry\x1a\\\n\x0fTransformsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32).org.apache.beam.runner_api.v1.PTransform:\x02\x38\x01\x1a_\n\x11PcollectionsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x39\n\x05value\x18\x02 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.PCollection:\x02\x38\x01\x1al\n\x18WindowingStrategiesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12?\n\x05value\x18\x02 \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.WindowingStrategy:\x02\x38\x01\x1aS\n\x0b\x43odersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x33\n\x05value\x18\x02 \x01(\x0b\x32$.org.apache.beam.runner_api.v1.Coder:\x02\x38\x01\x1a_\n\x11\x45nvironmentsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x39\n\x05value\x18\x02 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.Environment:\x02\x38\x01\"\xe4\x06\n\x15MessageWithComponents\x12=\n\ncomponents\x18\x01 \x01(\x0b\x32).org.apache.beam.runner_api.v1.Components\x12\x35\n\x05\x63oder\x18\x02 \x01(\x0b\x32$.org.apache.beam.runner_api.v1.CoderH\x00\x12H\n\x0f\x63ombine_payload\x18\x03 \x01(\x0b\x32-.org.apache.beam.runner_api.v1.CombinePayloadH\x00\x12\x44\n\rfunction_spec\x18\x04 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpecH\x00\x12\x45\n\x0epar_do_payload\x18\x06 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.ParDoPayloadH\x00\x12?\n\nptransform\x18\x07 \x01(\x0b\x32).org.apache.beam.runner_api.v1.PTransformH\x00\x12\x41\n\x0bpcollection\x18\x08 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.PCollectionH\x00\x12\x42\n\x0cread_payload\x18\t \x01(\x0b\x32*.org.apache.beam.runner_api.v1.ReadPayloadH\x00\x12>\n\nside_input\x18\x0b \x01(\x0b\x32(.org.apache.beam.runner_api.v1.SideInputH\x00\x12O\n\x13window_into_payload\x18\x0c \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.WindowIntoPayloadH\x00\x12N\n\x12windowing_strategy\x18\r \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.WindowingStrategyH\x00\x12M\n\x12urn_with_parameter\x18\x0e \x01(\x0b\x32/.org.apache.beam.runner_api.v1.UrnWithParameterH\x00\x42\x06\n\x04root\"\xa6\x01\n\x08Pipeline\x12=\n\ncomponents\x18\x01 \x01(\x0b\x32).org.apache.beam.runner_api.v1.Components\x12\x19\n\x11root_transform_id\x18\x02 \x01(\t\x12@\n\x0c\x64isplay_data\x18\x03 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.DisplayData\"\xa8\x03\n\nPTransform\x12\x13\n\x0bunique_name\x18\x05 \x01(\t\x12=\n\x04spec\x18\x01 \x01(\x0b\x32/.org.apache.beam.runner_api.v1.UrnWithParameter\x12\x15\n\rsubtransforms\x18\x02 \x03(\t\x12\x45\n\x06inputs\x18\x03 \x03(\x0b\x32\x35.org.apache.beam.runner_api.v1.PTransform.InputsEntry\x12G\n\x07outputs\x18\x04 \x03(\x0b\x32\x36.org.apache.beam.runner_api.v1.PTransform.OutputsEntry\x12@\n\x0c\x64isplay_data\x18\x06 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.DisplayData\x1a-\n\x0bInputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a.\n\x0cOutputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xd3\x01\n\x0bPCollection\x12\x13\n\x0bunique_name\x18\x01 \x01(\t\x12\x10\n\x08\x63oder_id\x18\x02 \x01(\t\x12<\n\nis_bounded\x18\x03 \x01(\x0e\x32(.org.apache.beam.runner_api.v1.IsBounded\x12\x1d\n\x15windowing_strategy_id\x18\x04 \x01(\t\x12@\n\x0c\x64isplay_data\x18\x05 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.DisplayData\"\xb5\x03\n\x0cParDoPayload\x12:\n\x05\x64o_fn\x18\x01 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\x12<\n\nparameters\x18\x02 \x03(\x0b\x32(.org.apache.beam.runner_api.v1.Parameter\x12P\n\x0bside_inputs\x18\x03 \x03(\x0b\x32;.org.apache.beam.runner_api.v1.ParDoPayload.SideInputsEntry\x12=\n\x0bstate_specs\x18\x04 \x03(\x0b\x32(.org.apache.beam.runner_api.v1.StateSpec\x12=\n\x0btimer_specs\x18\x05 \x03(\x0b\x32(.org.apache.beam.runner_api.v1.TimerSpec\x1a[\n\x0fSideInputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32(.org.apache.beam.runner_api.v1.SideInput:\x02\x38\x01\"\x8b\x01\n\tParameter\x12;\n\x04type\x18\x01 \x01(\x0e\x32-.org.apache.beam.runner_api.v1.Parameter.Type\"A\n\x04Type\x12\n\n\x06WINDOW\x10\x00\x12\x14\n\x10PIPELINE_OPTIONS\x10\x01\x12\x17\n\x13RESTRICTION_TRACKER\x10\x02\"\x0b\n\tStateSpec\"\x0b\n\tTimerSpec\"\x88\x01\n\x0bReadPayload\x12;\n\x06source\x18\x01 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\x12<\n\nis_bounded\x18\x02 \x01(\x0e\x32(.org.apache.beam.runner_api.v1.IsBounded\"S\n\x11WindowIntoPayload\x12>\n\twindow_fn\x18\x01 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\"\xde\x02\n\x0e\x43ombinePayload\x12?\n\ncombine_fn\x18\x01 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\x12\x1c\n\x14\x61\x63\x63umulator_coder_id\x18\x02 \x01(\t\x12<\n\nparameters\x18\x03 \x03(\x0b\x32(.org.apache.beam.runner_api.v1.Parameter\x12R\n\x0bside_inputs\x18\x04 \x03(\x0b\x32=.org.apache.beam.runner_api.v1.CombinePayload.SideInputsEntry\x1a[\n\x0fSideInputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32(.org.apache.beam.runner_api.v1.SideInput:\x02\x38\x01\"_\n\x05\x43oder\x12\x39\n\x04spec\x18\x01 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\x12\x1b\n\x13\x63omponent_coder_ids\x18\x02 \x03(\t\"\xd7\x03\n\x11WindowingStrategy\x12>\n\twindow_fn\x18\x01 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\x12@\n\x0cmerge_status\x18\x02 \x01(\x0e\x32*.org.apache.beam.runner_api.v1.MergeStatus\x12\x17\n\x0fwindow_coder_id\x18\x03 \x01(\t\x12\x37\n\x07trigger\x18\x04 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x12J\n\x11\x61\x63\x63umulation_mode\x18\x05 \x01(\x0e\x32/.org.apache.beam.runner_api.v1.AccumulationMode\x12>\n\x0boutput_time\x18\x06 \x01(\x0e\x32).org.apache.beam.runner_api.v1.OutputTime\x12H\n\x10\x63losing_behavior\x18\x07 \x01(\x0e\x32..org.apache.beam.runner_api.v1.ClosingBehavior\x12\x18\n\x10\x61llowed_lateness\x18\x08 \x01(\x03\"\xac\r\n\x07Trigger\x12\x44\n\tafter_all\x18\x01 \x01(\x0b\x32/.org.apache.beam.runner_api.v1.Trigger.AfterAllH\x00\x12\x44\n\tafter_any\x18\x02 \x01(\x0b\x32/.org.apache.beam.runner_api.v1.Trigger.AfterAnyH\x00\x12\x46\n\nafter_each\x18\x03 \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.Trigger.AfterEachH\x00\x12U\n\x12\x61\x66ter_end_of_widow\x18\x04 \x01(\x0b\x32\x37.org.apache.beam.runner_api.v1.Trigger.AfterEndOfWindowH\x00\x12[\n\x15\x61\x66ter_processing_time\x18\x05 \x01(\x0b\x32:.org.apache.beam.runner_api.v1.Trigger.AfterProcessingTimeH\x00\x12t\n\"after_synchronized_processing_time\x18\x06 \x01(\x0b\x32\x46.org.apache.beam.runner_api.v1.Trigger.AfterSynchronizedProcessingTimeH\x00\x12?\n\x06\x61lways\x18\x0c \x01(\x0b\x32-.org.apache.beam.runner_api.v1.Trigger.AlwaysH\x00\x12\x41\n\x07\x64\x65\x66\x61ult\x18\x07 \x01(\x0b\x32..org.apache.beam.runner_api.v1.Trigger.DefaultH\x00\x12L\n\relement_count\x18\x08 \x01(\x0b\x32\x33.org.apache.beam.runner_api.v1.Trigger.ElementCountH\x00\x12=\n\x05never\x18\t \x01(\x0b\x32,.org.apache.beam.runner_api.v1.Trigger.NeverH\x00\x12\x46\n\nor_finally\x18\n \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.Trigger.OrFinallyH\x00\x12?\n\x06repeat\x18\x0b \x01(\x0b\x32-.org.apache.beam.runner_api.v1.Trigger.RepeatH\x00\x1aG\n\x08\x41\x66terAll\x12;\n\x0bsubtriggers\x18\x01 \x03(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1aG\n\x08\x41\x66terAny\x12;\n\x0bsubtriggers\x18\x01 \x03(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1aH\n\tAfterEach\x12;\n\x0bsubtriggers\x18\x01 \x03(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1a\x8f\x01\n\x10\x41\x66terEndOfWindow\x12=\n\rearly_firings\x18\x01 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x12<\n\x0clate_firings\x18\x02 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1a\x66\n\x13\x41\x66terProcessingTime\x12O\n\x14timestamp_transforms\x18\x01 \x03(\x0b\x32\x31.org.apache.beam.runner_api.v1.TimestampTransform\x1a!\n\x1f\x41\x66terSynchronizedProcessingTime\x1a\t\n\x07\x44\x65\x66\x61ult\x1a%\n\x0c\x45lementCount\x12\x15\n\relement_count\x18\x01 \x01(\x05\x1a\x07\n\x05Never\x1a\x08\n\x06\x41lways\x1az\n\tOrFinally\x12\x34\n\x04main\x18\x01 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x12\x37\n\x07\x66inally\x18\x02 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1a\x44\n\x06Repeat\x12:\n\nsubtrigger\x18\x01 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.TriggerB\t\n\x07trigger\"\x8e\x02\n\x12TimestampTransform\x12H\n\x05\x64\x65lay\x18\x01 \x01(\x0b\x32\x37.org.apache.beam.runner_api.v1.TimestampTransform.DelayH\x00\x12M\n\x08\x61lign_to\x18\x02 \x01(\x0b\x32\x39.org.apache.beam.runner_api.v1.TimestampTransform.AlignToH\x00\x1a\x1d\n\x05\x44\x65lay\x12\x14\n\x0c\x64\x65lay_millis\x18\x01 \x01(\x03\x1a)\n\x07\x41lignTo\x12\x0e\n\x06period\x18\x03 \x01(\x03\x12\x0e\n\x06offset\x18\x04 \x01(\x03\x42\x15\n\x13timestamp_transform\"\xda\x01\n\tSideInput\x12G\n\x0e\x61\x63\x63\x65ss_pattern\x18\x01 \x01(\x0b\x32/.org.apache.beam.runner_api.v1.UrnWithParameter\x12<\n\x07view_fn\x18\x02 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\x12\x46\n\x11window_mapping_fn\x18\x03 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\"\x1a\n\x0b\x45nvironment\x12\x0b\n\x03url\x18\x01 \x01(\t\"e\n\x0c\x46unctionSpec\x12=\n\x04spec\x18\x01 \x01(\x0b\x32/.org.apache.beam.runner_api.v1.UrnWithParameter\x12\x16\n\x0e\x65nvironment_id\x18\x02 \x01(\t\"H\n\x10UrnWithParameter\x12\x0b\n\x03urn\x18\x01 \x01(\t\x12\'\n\tparameter\x18\x02 \x01(\x0b\x32\x14.google.protobuf.Any\"\xf7\x03\n\x0b\x44isplayData\x12>\n\x05items\x18\x01 \x03(\x0b\x32/.org.apache.beam.runner_api.v1.DisplayData.Item\x1a\x46\n\nIdentifier\x12\x14\n\x0ctransform_id\x18\x01 \x01(\t\x12\x15\n\rtransform_urn\x18\x02 \x01(\t\x12\x0b\n\x03key\x18\x03 \x01(\t\x1a\xf9\x01\n\x04Item\x12\x41\n\x02id\x18\x01 \x01(\x0b\x32\x35.org.apache.beam.runner_api.v1.DisplayData.Identifier\x12=\n\x04type\x18\x02 \x01(\x0e\x32/.org.apache.beam.runner_api.v1.DisplayData.Type\x12#\n\x05value\x18\x03 \x01(\x0b\x32\x14.google.protobuf.Any\x12)\n\x0bshort_value\x18\x04 \x01(\x0b\x32\x14.google.protobuf.Any\x12\r\n\x05label\x18\x05 \x01(\t\x12\x10\n\x08link_url\x18\x06 \x01(\t\"d\n\x04Type\x12\n\n\x06STRING\x10\x00\x12\x0b\n\x07INTEGER\x10\x01\x12\t\n\x05\x46LOAT\x10\x02\x12\x0b\n\x07\x42OOLEAN\x10\x03\x12\r\n\tTIMESTAMP\x10\x04\x12\x0c\n\x08\x44URATION\x10\x05\x12\x0e\n\nJAVA_CLASS\x10\x06*\'\n\tIsBounded\x12\x0b\n\x07\x42OUNDED\x10\x00\x12\r\n\tUNBOUNDED\x10\x01*C\n\x0bMergeStatus\x12\x0f\n\x0bNON_MERGING\x10\x00\x12\x0f\n\x0bNEEDS_MERGE\x10\x01\x12\x12\n\x0e\x41LREADY_MERGED\x10\x02*4\n\x10\x41\x63\x63umulationMode\x12\x0e\n\nDISCARDING\x10\x00\x12\x10\n\x0c\x41\x43\x43UMULATING\x10\x01*8\n\x0f\x43losingBehavior\x12\x0f\n\x0b\x45MIT_ALWAYS\x10\x00\x12\x14\n\x10\x45MIT_IF_NONEMPTY\x10\x01*I\n\nOutputTime\x12\x11\n\rEND_OF_WINDOW\x10\x00\x12\x12\n\x0eLATEST_IN_PANE\x10\x01\x12\x14\n\x10\x45\x41RLIEST_IN_PANE\x10\x02*S\n\nTimeDomain\x12\x0e\n\nEVENT_TIME\x10\x00\x12\x13\n\x0fPROCESSING_TIME\x10\x01\x12 \n\x1cSYNCHRONIZED_PROCESSING_TIME\x10\x02\x42\x31\n$org.apache.beam.sdk.common.runner.v1B\tRunnerApib\x06proto3') + serialized_pb=_b('\n\x15\x62\x65\x61m_runner_api.proto\x12\x1dorg.apache.beam.runner_api.v1\x1a\x19google/protobuf/any.proto\"\x8d\x07\n\nComponents\x12M\n\ntransforms\x18\x01 \x03(\x0b\x32\x39.org.apache.beam.runner_api.v1.Components.TransformsEntry\x12Q\n\x0cpcollections\x18\x02 \x03(\x0b\x32;.org.apache.beam.runner_api.v1.Components.PcollectionsEntry\x12`\n\x14windowing_strategies\x18\x03 \x03(\x0b\x32\x42.org.apache.beam.runner_api.v1.Components.WindowingStrategiesEntry\x12\x45\n\x06\x63oders\x18\x04 \x03(\x0b\x32\x35.org.apache.beam.runner_api.v1.Components.CodersEntry\x12Q\n\x0c\x65nvironments\x18\x05 \x03(\x0b\x32;.org.apache.beam.runner_api.v1.Components.EnvironmentsEntry\x1a\\\n\x0fTransformsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32).org.apache.beam.runner_api.v1.PTransform:\x02\x38\x01\x1a_\n\x11PcollectionsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x39\n\x05value\x18\x02 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.PCollection:\x02\x38\x01\x1al\n\x18WindowingStrategiesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12?\n\x05value\x18\x02 \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.WindowingStrategy:\x02\x38\x01\x1aS\n\x0b\x43odersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x33\n\x05value\x18\x02 \x01(\x0b\x32$.org.apache.beam.runner_api.v1.Coder:\x02\x38\x01\x1a_\n\x11\x45nvironmentsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x39\n\x05value\x18\x02 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.Environment:\x02\x38\x01\"\xe2\x06\n\x15MessageWithComponents\x12=\n\ncomponents\x18\x01 \x01(\x0b\x32).org.apache.beam.runner_api.v1.Components\x12\x35\n\x05\x63oder\x18\x02 \x01(\x0b\x32$.org.apache.beam.runner_api.v1.CoderH\x00\x12H\n\x0f\x63ombine_payload\x18\x03 \x01(\x0b\x32-.org.apache.beam.runner_api.v1.CombinePayloadH\x00\x12K\n\x11sdk_function_spec\x18\x04 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpecH\x00\x12\x45\n\x0epar_do_payload\x18\x06 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.ParDoPayloadH\x00\x12?\n\nptransform\x18\x07 \x01(\x0b\x32).org.apache.beam.runner_api.v1.PTransformH\x00\x12\x41\n\x0bpcollection\x18\x08 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.PCollectionH\x00\x12\x42\n\x0cread_payload\x18\t \x01(\x0b\x32*.org.apache.beam.runner_api.v1.ReadPayloadH\x00\x12>\n\nside_input\x18\x0b \x01(\x0b\x32(.org.apache.beam.runner_api.v1.SideInputH\x00\x12O\n\x13window_into_payload\x18\x0c \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.WindowIntoPayloadH\x00\x12N\n\x12windowing_strategy\x18\r \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.WindowingStrategyH\x00\x12\x44\n\rfunction_spec\x18\x0e \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpecH\x00\x42\x06\n\x04root\"\xa6\x01\n\x08Pipeline\x12=\n\ncomponents\x18\x01 \x01(\x0b\x32).org.apache.beam.runner_api.v1.Components\x12\x19\n\x11root_transform_id\x18\x02 \x01(\t\x12@\n\x0c\x64isplay_data\x18\x03 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.DisplayData\"\xa4\x03\n\nPTransform\x12\x13\n\x0bunique_name\x18\x05 \x01(\t\x12\x39\n\x04spec\x18\x01 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\x12\x15\n\rsubtransforms\x18\x02 \x03(\t\x12\x45\n\x06inputs\x18\x03 \x03(\x0b\x32\x35.org.apache.beam.runner_api.v1.PTransform.InputsEntry\x12G\n\x07outputs\x18\x04 \x03(\x0b\x32\x36.org.apache.beam.runner_api.v1.PTransform.OutputsEntry\x12@\n\x0c\x64isplay_data\x18\x06 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.DisplayData\x1a-\n\x0bInputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a.\n\x0cOutputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xd3\x01\n\x0bPCollection\x12\x13\n\x0bunique_name\x18\x01 \x01(\t\x12\x10\n\x08\x63oder_id\x18\x02 \x01(\t\x12<\n\nis_bounded\x18\x03 \x01(\x0e\x32(.org.apache.beam.runner_api.v1.IsBounded\x12\x1d\n\x15windowing_strategy_id\x18\x04 \x01(\t\x12@\n\x0c\x64isplay_data\x18\x05 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.DisplayData\"\xb8\x03\n\x0cParDoPayload\x12=\n\x05\x64o_fn\x18\x01 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\x12<\n\nparameters\x18\x02 \x03(\x0b\x32(.org.apache.beam.runner_api.v1.Parameter\x12P\n\x0bside_inputs\x18\x03 \x03(\x0b\x32;.org.apache.beam.runner_api.v1.ParDoPayload.SideInputsEntry\x12=\n\x0bstate_specs\x18\x04 \x03(\x0b\x32(.org.apache.beam.runner_api.v1.StateSpec\x12=\n\x0btimer_specs\x18\x05 \x03(\x0b\x32(.org.apache.beam.runner_api.v1.TimerSpec\x1a[\n\x0fSideInputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32(.org.apache.beam.runner_api.v1.SideInput:\x02\x38\x01\"\x8b\x01\n\tParameter\x12;\n\x04type\x18\x01 \x01(\x0e\x32-.org.apache.beam.runner_api.v1.Parameter.Type\"A\n\x04Type\x12\n\n\x06WINDOW\x10\x00\x12\x14\n\x10PIPELINE_OPTIONS\x10\x01\x12\x17\n\x13RESTRICTION_TRACKER\x10\x02\"\x0b\n\tStateSpec\"\x0b\n\tTimerSpec\"\x8b\x01\n\x0bReadPayload\x12>\n\x06source\x18\x01 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\x12<\n\nis_bounded\x18\x02 \x01(\x0e\x32(.org.apache.beam.runner_api.v1.IsBounded\"V\n\x11WindowIntoPayload\x12\x41\n\twindow_fn\x18\x01 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\"\xe1\x02\n\x0e\x43ombinePayload\x12\x42\n\ncombine_fn\x18\x01 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\x12\x1c\n\x14\x61\x63\x63umulator_coder_id\x18\x02 \x01(\t\x12<\n\nparameters\x18\x03 \x03(\x0b\x32(.org.apache.beam.runner_api.v1.Parameter\x12R\n\x0bside_inputs\x18\x04 \x03(\x0b\x32=.org.apache.beam.runner_api.v1.CombinePayload.SideInputsEntry\x1a[\n\x0fSideInputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32(.org.apache.beam.runner_api.v1.SideInput:\x02\x38\x01\"b\n\x05\x43oder\x12<\n\x04spec\x18\x01 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\x12\x1b\n\x13\x63omponent_coder_ids\x18\x02 \x03(\t\"\xda\x03\n\x11WindowingStrategy\x12\x41\n\twindow_fn\x18\x01 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\x12@\n\x0cmerge_status\x18\x02 \x01(\x0e\x32*.org.apache.beam.runner_api.v1.MergeStatus\x12\x17\n\x0fwindow_coder_id\x18\x03 \x01(\t\x12\x37\n\x07trigger\x18\x04 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x12J\n\x11\x61\x63\x63umulation_mode\x18\x05 \x01(\x0e\x32/.org.apache.beam.runner_api.v1.AccumulationMode\x12>\n\x0boutput_time\x18\x06 \x01(\x0e\x32).org.apache.beam.runner_api.v1.OutputTime\x12H\n\x10\x63losing_behavior\x18\x07 \x01(\x0e\x32..org.apache.beam.runner_api.v1.ClosingBehavior\x12\x18\n\x10\x61llowed_lateness\x18\x08 \x01(\x03\"\xad\r\n\x07Trigger\x12\x44\n\tafter_all\x18\x01 \x01(\x0b\x32/.org.apache.beam.runner_api.v1.Trigger.AfterAllH\x00\x12\x44\n\tafter_any\x18\x02 \x01(\x0b\x32/.org.apache.beam.runner_api.v1.Trigger.AfterAnyH\x00\x12\x46\n\nafter_each\x18\x03 \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.Trigger.AfterEachH\x00\x12V\n\x13\x61\x66ter_end_of_window\x18\x04 \x01(\x0b\x32\x37.org.apache.beam.runner_api.v1.Trigger.AfterEndOfWindowH\x00\x12[\n\x15\x61\x66ter_processing_time\x18\x05 \x01(\x0b\x32:.org.apache.beam.runner_api.v1.Trigger.AfterProcessingTimeH\x00\x12t\n\"after_synchronized_processing_time\x18\x06 \x01(\x0b\x32\x46.org.apache.beam.runner_api.v1.Trigger.AfterSynchronizedProcessingTimeH\x00\x12?\n\x06\x61lways\x18\x0c \x01(\x0b\x32-.org.apache.beam.runner_api.v1.Trigger.AlwaysH\x00\x12\x41\n\x07\x64\x65\x66\x61ult\x18\x07 \x01(\x0b\x32..org.apache.beam.runner_api.v1.Trigger.DefaultH\x00\x12L\n\relement_count\x18\x08 \x01(\x0b\x32\x33.org.apache.beam.runner_api.v1.Trigger.ElementCountH\x00\x12=\n\x05never\x18\t \x01(\x0b\x32,.org.apache.beam.runner_api.v1.Trigger.NeverH\x00\x12\x46\n\nor_finally\x18\n \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.Trigger.OrFinallyH\x00\x12?\n\x06repeat\x18\x0b \x01(\x0b\x32-.org.apache.beam.runner_api.v1.Trigger.RepeatH\x00\x1aG\n\x08\x41\x66terAll\x12;\n\x0bsubtriggers\x18\x01 \x03(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1aG\n\x08\x41\x66terAny\x12;\n\x0bsubtriggers\x18\x01 \x03(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1aH\n\tAfterEach\x12;\n\x0bsubtriggers\x18\x01 \x03(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1a\x8f\x01\n\x10\x41\x66terEndOfWindow\x12=\n\rearly_firings\x18\x01 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x12<\n\x0clate_firings\x18\x02 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1a\x66\n\x13\x41\x66terProcessingTime\x12O\n\x14timestamp_transforms\x18\x01 \x03(\x0b\x32\x31.org.apache.beam.runner_api.v1.TimestampTransform\x1a!\n\x1f\x41\x66terSynchronizedProcessingTime\x1a\t\n\x07\x44\x65\x66\x61ult\x1a%\n\x0c\x45lementCount\x12\x15\n\relement_count\x18\x01 \x01(\x05\x1a\x07\n\x05Never\x1a\x08\n\x06\x41lways\x1az\n\tOrFinally\x12\x34\n\x04main\x18\x01 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x12\x37\n\x07\x66inally\x18\x02 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1a\x44\n\x06Repeat\x12:\n\nsubtrigger\x18\x01 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.TriggerB\t\n\x07trigger\"\x8e\x02\n\x12TimestampTransform\x12H\n\x05\x64\x65lay\x18\x01 \x01(\x0b\x32\x37.org.apache.beam.runner_api.v1.TimestampTransform.DelayH\x00\x12M\n\x08\x61lign_to\x18\x02 \x01(\x0b\x32\x39.org.apache.beam.runner_api.v1.TimestampTransform.AlignToH\x00\x1a\x1d\n\x05\x44\x65lay\x12\x14\n\x0c\x64\x65lay_millis\x18\x01 \x01(\x03\x1a)\n\x07\x41lignTo\x12\x0e\n\x06period\x18\x03 \x01(\x03\x12\x0e\n\x06offset\x18\x04 \x01(\x03\x42\x15\n\x13timestamp_transform\"\xdc\x01\n\tSideInput\x12\x43\n\x0e\x61\x63\x63\x65ss_pattern\x18\x01 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\x12?\n\x07view_fn\x18\x02 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\x12I\n\x11window_mapping_fn\x18\x03 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\"\x1a\n\x0b\x45nvironment\x12\x0b\n\x03url\x18\x01 \x01(\t\"d\n\x0fSdkFunctionSpec\x12\x39\n\x04spec\x18\x01 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\x12\x16\n\x0e\x65nvironment_id\x18\x02 \x01(\t\"D\n\x0c\x46unctionSpec\x12\x0b\n\x03urn\x18\x01 \x01(\t\x12\'\n\tparameter\x18\x02 \x01(\x0b\x32\x14.google.protobuf.Any\"\xf7\x03\n\x0b\x44isplayData\x12>\n\x05items\x18\x01 \x03(\x0b\x32/.org.apache.beam.runner_api.v1.DisplayData.Item\x1a\x46\n\nIdentifier\x12\x14\n\x0ctransform_id\x18\x01 \x01(\t\x12\x15\n\rtransform_urn\x18\x02 \x01(\t\x12\x0b\n\x03key\x18\x03 \x01(\t\x1a\xf9\x01\n\x04Item\x12\x41\n\x02id\x18\x01 \x01(\x0b\x32\x35.org.apache.beam.runner_api.v1.DisplayData.Identifier\x12=\n\x04type\x18\x02 \x01(\x0e\x32/.org.apache.beam.runner_api.v1.DisplayData.Type\x12#\n\x05value\x18\x03 \x01(\x0b\x32\x14.google.protobuf.Any\x12)\n\x0bshort_value\x18\x04 \x01(\x0b\x32\x14.google.protobuf.Any\x12\r\n\x05label\x18\x05 \x01(\t\x12\x10\n\x08link_url\x18\x06 \x01(\t\"d\n\x04Type\x12\n\n\x06STRING\x10\x00\x12\x0b\n\x07INTEGER\x10\x01\x12\t\n\x05\x46LOAT\x10\x02\x12\x0b\n\x07\x42OOLEAN\x10\x03\x12\r\n\tTIMESTAMP\x10\x04\x12\x0c\n\x08\x44URATION\x10\x05\x12\x0e\n\nJAVA_CLASS\x10\x06*\'\n\tIsBounded\x12\x0b\n\x07\x42OUNDED\x10\x00\x12\r\n\tUNBOUNDED\x10\x01*C\n\x0bMergeStatus\x12\x0f\n\x0bNON_MERGING\x10\x00\x12\x0f\n\x0bNEEDS_MERGE\x10\x01\x12\x12\n\x0e\x41LREADY_MERGED\x10\x02*4\n\x10\x41\x63\x63umulationMode\x12\x0e\n\nDISCARDING\x10\x00\x12\x10\n\x0c\x41\x43\x43UMULATING\x10\x01*8\n\x0f\x43losingBehavior\x12\x0f\n\x0b\x45MIT_ALWAYS\x10\x00\x12\x14\n\x10\x45MIT_IF_NONEMPTY\x10\x01*I\n\nOutputTime\x12\x11\n\rEND_OF_WINDOW\x10\x00\x12\x12\n\x0eLATEST_IN_PANE\x10\x01\x12\x14\n\x10\x45\x41RLIEST_IN_PANE\x10\x02*S\n\nTimeDomain\x12\x0e\n\nEVENT_TIME\x10\x00\x12\x13\n\x0fPROCESSING_TIME\x10\x01\x12 \n\x1cSYNCHRONIZED_PROCESSING_TIME\x10\x02\x42\x31\n$org.apache.beam.sdk.common.runner.v1B\tRunnerApib\x06proto3') , dependencies=[google_dot_protobuf_dot_any__pb2.DESCRIPTOR,]) _sym_db.RegisterFileDescriptor(DESCRIPTOR) @@ -60,8 +60,8 @@ ], containing_type=None, options=None, - serialized_start=7348, - serialized_end=7387, + serialized_start=7358, + serialized_end=7397, ) _sym_db.RegisterEnumDescriptor(_ISBOUNDED) @@ -87,8 +87,8 @@ ], containing_type=None, options=None, - serialized_start=7389, - serialized_end=7456, + serialized_start=7399, + serialized_end=7466, ) _sym_db.RegisterEnumDescriptor(_MERGESTATUS) @@ -110,8 +110,8 @@ ], containing_type=None, options=None, - serialized_start=7458, - serialized_end=7510, + serialized_start=7468, + serialized_end=7520, ) _sym_db.RegisterEnumDescriptor(_ACCUMULATIONMODE) @@ -133,8 +133,8 @@ ], containing_type=None, options=None, - serialized_start=7512, - serialized_end=7568, + serialized_start=7522, + serialized_end=7578, ) _sym_db.RegisterEnumDescriptor(_CLOSINGBEHAVIOR) @@ -160,8 +160,8 @@ ], containing_type=None, options=None, - serialized_start=7570, - serialized_end=7643, + serialized_start=7580, + serialized_end=7653, ) _sym_db.RegisterEnumDescriptor(_OUTPUTTIME) @@ -187,8 +187,8 @@ ], containing_type=None, options=None, - serialized_start=7645, - serialized_end=7728, + serialized_start=7655, + serialized_end=7738, ) _sym_db.RegisterEnumDescriptor(_TIMEDOMAIN) @@ -231,8 +231,8 @@ ], containing_type=None, options=None, - serialized_start=3191, - serialized_end=3256, + serialized_start=3188, + serialized_end=3253, ) _sym_db.RegisterEnumDescriptor(_PARAMETER_TYPE) @@ -273,8 +273,8 @@ ], containing_type=None, options=None, - serialized_start=7246, - serialized_end=7346, + serialized_start=7256, + serialized_end=7356, ) _sym_db.RegisterEnumDescriptor(_DISPLAYDATA_TYPE) @@ -552,7 +552,7 @@ is_extension=False, extension_scope=None, options=None), _descriptor.FieldDescriptor( - name='function_spec', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.function_spec', index=3, + name='sdk_function_spec', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.sdk_function_spec', index=3, number=4, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, @@ -608,7 +608,7 @@ is_extension=False, extension_scope=None, options=None), _descriptor.FieldDescriptor( - name='urn_with_parameter', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.urn_with_parameter', index=11, + name='function_spec', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.function_spec', index=11, number=14, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, @@ -630,7 +630,7 @@ index=0, containing_type=None, fields=[]), ], serialized_start=996, - serialized_end=1864, + serialized_end=1862, ) @@ -674,8 +674,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1867, - serialized_end=2033, + serialized_start=1865, + serialized_end=2031, ) @@ -712,8 +712,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=2367, - serialized_end=2412, + serialized_start=2361, + serialized_end=2406, ) _PTRANSFORM_OUTPUTSENTRY = _descriptor.Descriptor( @@ -749,8 +749,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=2414, - serialized_end=2460, + serialized_start=2408, + serialized_end=2454, ) _PTRANSFORM = _descriptor.Descriptor( @@ -814,8 +814,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=2036, - serialized_end=2460, + serialized_start=2034, + serialized_end=2454, ) @@ -873,8 +873,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=2463, - serialized_end=2674, + serialized_start=2457, + serialized_end=2668, ) @@ -911,8 +911,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3023, - serialized_end=3114, + serialized_start=3020, + serialized_end=3111, ) _PARDOPAYLOAD = _descriptor.Descriptor( @@ -969,8 +969,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=2677, - serialized_end=3114, + serialized_start=2671, + serialized_end=3111, ) @@ -1001,8 +1001,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3117, - serialized_end=3256, + serialized_start=3114, + serialized_end=3253, ) @@ -1025,8 +1025,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3258, - serialized_end=3269, + serialized_start=3255, + serialized_end=3266, ) @@ -1049,8 +1049,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3271, - serialized_end=3282, + serialized_start=3268, + serialized_end=3279, ) @@ -1087,7 +1087,7 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3285, + serialized_start=3282, serialized_end=3421, ) @@ -1119,7 +1119,7 @@ oneofs=[ ], serialized_start=3423, - serialized_end=3506, + serialized_end=3509, ) @@ -1156,8 +1156,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3023, - serialized_end=3114, + serialized_start=3020, + serialized_end=3111, ) _COMBINEPAYLOAD = _descriptor.Descriptor( @@ -1207,8 +1207,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3509, - serialized_end=3859, + serialized_start=3512, + serialized_end=3865, ) @@ -1245,8 +1245,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3861, - serialized_end=3956, + serialized_start=3867, + serialized_end=3965, ) @@ -1325,8 +1325,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=3959, - serialized_end=4430, + serialized_start=3968, + serialized_end=4442, ) @@ -1356,8 +1356,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=5364, - serialized_end=5435, + serialized_start=5377, + serialized_end=5448, ) _TRIGGER_AFTERANY = _descriptor.Descriptor( @@ -1386,8 +1386,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=5437, - serialized_end=5508, + serialized_start=5450, + serialized_end=5521, ) _TRIGGER_AFTEREACH = _descriptor.Descriptor( @@ -1416,8 +1416,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=5510, - serialized_end=5582, + serialized_start=5523, + serialized_end=5595, ) _TRIGGER_AFTERENDOFWINDOW = _descriptor.Descriptor( @@ -1453,8 +1453,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=5585, - serialized_end=5728, + serialized_start=5598, + serialized_end=5741, ) _TRIGGER_AFTERPROCESSINGTIME = _descriptor.Descriptor( @@ -1483,8 +1483,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=5730, - serialized_end=5832, + serialized_start=5743, + serialized_end=5845, ) _TRIGGER_AFTERSYNCHRONIZEDPROCESSINGTIME = _descriptor.Descriptor( @@ -1506,8 +1506,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=5834, - serialized_end=5867, + serialized_start=5847, + serialized_end=5880, ) _TRIGGER_DEFAULT = _descriptor.Descriptor( @@ -1529,8 +1529,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=5869, - serialized_end=5878, + serialized_start=5882, + serialized_end=5891, ) _TRIGGER_ELEMENTCOUNT = _descriptor.Descriptor( @@ -1559,8 +1559,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=5880, - serialized_end=5917, + serialized_start=5893, + serialized_end=5930, ) _TRIGGER_NEVER = _descriptor.Descriptor( @@ -1582,8 +1582,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=5919, - serialized_end=5926, + serialized_start=5932, + serialized_end=5939, ) _TRIGGER_ALWAYS = _descriptor.Descriptor( @@ -1605,8 +1605,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=5928, - serialized_end=5936, + serialized_start=5941, + serialized_end=5949, ) _TRIGGER_ORFINALLY = _descriptor.Descriptor( @@ -1642,8 +1642,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=5938, - serialized_end=6060, + serialized_start=5951, + serialized_end=6073, ) _TRIGGER_REPEAT = _descriptor.Descriptor( @@ -1672,8 +1672,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6062, - serialized_end=6130, + serialized_start=6075, + serialized_end=6143, ) _TRIGGER = _descriptor.Descriptor( @@ -1705,7 +1705,7 @@ is_extension=False, extension_scope=None, options=None), _descriptor.FieldDescriptor( - name='after_end_of_widow', full_name='org.apache.beam.runner_api.v1.Trigger.after_end_of_widow', index=3, + name='after_end_of_window', full_name='org.apache.beam.runner_api.v1.Trigger.after_end_of_window', index=3, number=4, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, @@ -1782,8 +1782,8 @@ name='trigger', full_name='org.apache.beam.runner_api.v1.Trigger.trigger', index=0, containing_type=None, fields=[]), ], - serialized_start=4433, - serialized_end=6141, + serialized_start=4445, + serialized_end=6154, ) @@ -1813,8 +1813,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6319, - serialized_end=6348, + serialized_start=6332, + serialized_end=6361, ) _TIMESTAMPTRANSFORM_ALIGNTO = _descriptor.Descriptor( @@ -1850,8 +1850,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6350, - serialized_end=6391, + serialized_start=6363, + serialized_end=6404, ) _TIMESTAMPTRANSFORM = _descriptor.Descriptor( @@ -1890,8 +1890,8 @@ name='timestamp_transform', full_name='org.apache.beam.runner_api.v1.TimestampTransform.timestamp_transform', index=0, containing_type=None, fields=[]), ], - serialized_start=6144, - serialized_end=6414, + serialized_start=6157, + serialized_end=6427, ) @@ -1935,8 +1935,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6417, - serialized_end=6635, + serialized_start=6430, + serialized_end=6650, ) @@ -1966,27 +1966,27 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6637, - serialized_end=6663, + serialized_start=6652, + serialized_end=6678, ) -_FUNCTIONSPEC = _descriptor.Descriptor( - name='FunctionSpec', - full_name='org.apache.beam.runner_api.v1.FunctionSpec', +_SDKFUNCTIONSPEC = _descriptor.Descriptor( + name='SdkFunctionSpec', + full_name='org.apache.beam.runner_api.v1.SdkFunctionSpec', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='spec', full_name='org.apache.beam.runner_api.v1.FunctionSpec.spec', index=0, + name='spec', full_name='org.apache.beam.runner_api.v1.SdkFunctionSpec.spec', index=0, number=1, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None), _descriptor.FieldDescriptor( - name='environment_id', full_name='org.apache.beam.runner_api.v1.FunctionSpec.environment_id', index=1, + name='environment_id', full_name='org.apache.beam.runner_api.v1.SdkFunctionSpec.environment_id', index=1, number=2, type=9, cpp_type=9, label=1, has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, @@ -2004,27 +2004,27 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6665, - serialized_end=6766, + serialized_start=6680, + serialized_end=6780, ) -_URNWITHPARAMETER = _descriptor.Descriptor( - name='UrnWithParameter', - full_name='org.apache.beam.runner_api.v1.UrnWithParameter', +_FUNCTIONSPEC = _descriptor.Descriptor( + name='FunctionSpec', + full_name='org.apache.beam.runner_api.v1.FunctionSpec', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='urn', full_name='org.apache.beam.runner_api.v1.UrnWithParameter.urn', index=0, + name='urn', full_name='org.apache.beam.runner_api.v1.FunctionSpec.urn', index=0, number=1, type=9, cpp_type=9, label=1, has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None), _descriptor.FieldDescriptor( - name='parameter', full_name='org.apache.beam.runner_api.v1.UrnWithParameter.parameter', index=1, + name='parameter', full_name='org.apache.beam.runner_api.v1.FunctionSpec.parameter', index=1, number=2, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, @@ -2042,8 +2042,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6768, - serialized_end=6840, + serialized_start=6782, + serialized_end=6850, ) @@ -2087,8 +2087,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6922, - serialized_end=6992, + serialized_start=6932, + serialized_end=7002, ) _DISPLAYDATA_ITEM = _descriptor.Descriptor( @@ -2152,8 +2152,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6995, - serialized_end=7244, + serialized_start=7005, + serialized_end=7254, ) _DISPLAYDATA = _descriptor.Descriptor( @@ -2183,8 +2183,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6843, - serialized_end=7346, + serialized_start=6853, + serialized_end=7356, ) _COMPONENTS_TRANSFORMSENTRY.fields_by_name['value'].message_type = _PTRANSFORM @@ -2205,7 +2205,7 @@ _MESSAGEWITHCOMPONENTS.fields_by_name['components'].message_type = _COMPONENTS _MESSAGEWITHCOMPONENTS.fields_by_name['coder'].message_type = _CODER _MESSAGEWITHCOMPONENTS.fields_by_name['combine_payload'].message_type = _COMBINEPAYLOAD -_MESSAGEWITHCOMPONENTS.fields_by_name['function_spec'].message_type = _FUNCTIONSPEC +_MESSAGEWITHCOMPONENTS.fields_by_name['sdk_function_spec'].message_type = _SDKFUNCTIONSPEC _MESSAGEWITHCOMPONENTS.fields_by_name['par_do_payload'].message_type = _PARDOPAYLOAD _MESSAGEWITHCOMPONENTS.fields_by_name['ptransform'].message_type = _PTRANSFORM _MESSAGEWITHCOMPONENTS.fields_by_name['pcollection'].message_type = _PCOLLECTION @@ -2213,7 +2213,7 @@ _MESSAGEWITHCOMPONENTS.fields_by_name['side_input'].message_type = _SIDEINPUT _MESSAGEWITHCOMPONENTS.fields_by_name['window_into_payload'].message_type = _WINDOWINTOPAYLOAD _MESSAGEWITHCOMPONENTS.fields_by_name['windowing_strategy'].message_type = _WINDOWINGSTRATEGY -_MESSAGEWITHCOMPONENTS.fields_by_name['urn_with_parameter'].message_type = _URNWITHPARAMETER +_MESSAGEWITHCOMPONENTS.fields_by_name['function_spec'].message_type = _FUNCTIONSPEC _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( _MESSAGEWITHCOMPONENTS.fields_by_name['coder']) _MESSAGEWITHCOMPONENTS.fields_by_name['coder'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] @@ -2221,8 +2221,8 @@ _MESSAGEWITHCOMPONENTS.fields_by_name['combine_payload']) _MESSAGEWITHCOMPONENTS.fields_by_name['combine_payload'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( - _MESSAGEWITHCOMPONENTS.fields_by_name['function_spec']) -_MESSAGEWITHCOMPONENTS.fields_by_name['function_spec'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] + _MESSAGEWITHCOMPONENTS.fields_by_name['sdk_function_spec']) +_MESSAGEWITHCOMPONENTS.fields_by_name['sdk_function_spec'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( _MESSAGEWITHCOMPONENTS.fields_by_name['par_do_payload']) _MESSAGEWITHCOMPONENTS.fields_by_name['par_do_payload'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] @@ -2245,13 +2245,13 @@ _MESSAGEWITHCOMPONENTS.fields_by_name['windowing_strategy']) _MESSAGEWITHCOMPONENTS.fields_by_name['windowing_strategy'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( - _MESSAGEWITHCOMPONENTS.fields_by_name['urn_with_parameter']) -_MESSAGEWITHCOMPONENTS.fields_by_name['urn_with_parameter'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] + _MESSAGEWITHCOMPONENTS.fields_by_name['function_spec']) +_MESSAGEWITHCOMPONENTS.fields_by_name['function_spec'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] _PIPELINE.fields_by_name['components'].message_type = _COMPONENTS _PIPELINE.fields_by_name['display_data'].message_type = _DISPLAYDATA _PTRANSFORM_INPUTSENTRY.containing_type = _PTRANSFORM _PTRANSFORM_OUTPUTSENTRY.containing_type = _PTRANSFORM -_PTRANSFORM.fields_by_name['spec'].message_type = _URNWITHPARAMETER +_PTRANSFORM.fields_by_name['spec'].message_type = _FUNCTIONSPEC _PTRANSFORM.fields_by_name['inputs'].message_type = _PTRANSFORM_INPUTSENTRY _PTRANSFORM.fields_by_name['outputs'].message_type = _PTRANSFORM_OUTPUTSENTRY _PTRANSFORM.fields_by_name['display_data'].message_type = _DISPLAYDATA @@ -2259,23 +2259,23 @@ _PCOLLECTION.fields_by_name['display_data'].message_type = _DISPLAYDATA _PARDOPAYLOAD_SIDEINPUTSENTRY.fields_by_name['value'].message_type = _SIDEINPUT _PARDOPAYLOAD_SIDEINPUTSENTRY.containing_type = _PARDOPAYLOAD -_PARDOPAYLOAD.fields_by_name['do_fn'].message_type = _FUNCTIONSPEC +_PARDOPAYLOAD.fields_by_name['do_fn'].message_type = _SDKFUNCTIONSPEC _PARDOPAYLOAD.fields_by_name['parameters'].message_type = _PARAMETER _PARDOPAYLOAD.fields_by_name['side_inputs'].message_type = _PARDOPAYLOAD_SIDEINPUTSENTRY _PARDOPAYLOAD.fields_by_name['state_specs'].message_type = _STATESPEC _PARDOPAYLOAD.fields_by_name['timer_specs'].message_type = _TIMERSPEC _PARAMETER.fields_by_name['type'].enum_type = _PARAMETER_TYPE _PARAMETER_TYPE.containing_type = _PARAMETER -_READPAYLOAD.fields_by_name['source'].message_type = _FUNCTIONSPEC +_READPAYLOAD.fields_by_name['source'].message_type = _SDKFUNCTIONSPEC _READPAYLOAD.fields_by_name['is_bounded'].enum_type = _ISBOUNDED -_WINDOWINTOPAYLOAD.fields_by_name['window_fn'].message_type = _FUNCTIONSPEC +_WINDOWINTOPAYLOAD.fields_by_name['window_fn'].message_type = _SDKFUNCTIONSPEC _COMBINEPAYLOAD_SIDEINPUTSENTRY.fields_by_name['value'].message_type = _SIDEINPUT _COMBINEPAYLOAD_SIDEINPUTSENTRY.containing_type = _COMBINEPAYLOAD -_COMBINEPAYLOAD.fields_by_name['combine_fn'].message_type = _FUNCTIONSPEC +_COMBINEPAYLOAD.fields_by_name['combine_fn'].message_type = _SDKFUNCTIONSPEC _COMBINEPAYLOAD.fields_by_name['parameters'].message_type = _PARAMETER _COMBINEPAYLOAD.fields_by_name['side_inputs'].message_type = _COMBINEPAYLOAD_SIDEINPUTSENTRY -_CODER.fields_by_name['spec'].message_type = _FUNCTIONSPEC -_WINDOWINGSTRATEGY.fields_by_name['window_fn'].message_type = _FUNCTIONSPEC +_CODER.fields_by_name['spec'].message_type = _SDKFUNCTIONSPEC +_WINDOWINGSTRATEGY.fields_by_name['window_fn'].message_type = _SDKFUNCTIONSPEC _WINDOWINGSTRATEGY.fields_by_name['merge_status'].enum_type = _MERGESTATUS _WINDOWINGSTRATEGY.fields_by_name['trigger'].message_type = _TRIGGER _WINDOWINGSTRATEGY.fields_by_name['accumulation_mode'].enum_type = _ACCUMULATIONMODE @@ -2305,7 +2305,7 @@ _TRIGGER.fields_by_name['after_all'].message_type = _TRIGGER_AFTERALL _TRIGGER.fields_by_name['after_any'].message_type = _TRIGGER_AFTERANY _TRIGGER.fields_by_name['after_each'].message_type = _TRIGGER_AFTEREACH -_TRIGGER.fields_by_name['after_end_of_widow'].message_type = _TRIGGER_AFTERENDOFWINDOW +_TRIGGER.fields_by_name['after_end_of_window'].message_type = _TRIGGER_AFTERENDOFWINDOW _TRIGGER.fields_by_name['after_processing_time'].message_type = _TRIGGER_AFTERPROCESSINGTIME _TRIGGER.fields_by_name['after_synchronized_processing_time'].message_type = _TRIGGER_AFTERSYNCHRONIZEDPROCESSINGTIME _TRIGGER.fields_by_name['always'].message_type = _TRIGGER_ALWAYS @@ -2324,8 +2324,8 @@ _TRIGGER.fields_by_name['after_each']) _TRIGGER.fields_by_name['after_each'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] _TRIGGER.oneofs_by_name['trigger'].fields.append( - _TRIGGER.fields_by_name['after_end_of_widow']) -_TRIGGER.fields_by_name['after_end_of_widow'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] + _TRIGGER.fields_by_name['after_end_of_window']) +_TRIGGER.fields_by_name['after_end_of_window'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] _TRIGGER.oneofs_by_name['trigger'].fields.append( _TRIGGER.fields_by_name['after_processing_time']) _TRIGGER.fields_by_name['after_processing_time'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] @@ -2360,11 +2360,11 @@ _TIMESTAMPTRANSFORM.oneofs_by_name['timestamp_transform'].fields.append( _TIMESTAMPTRANSFORM.fields_by_name['align_to']) _TIMESTAMPTRANSFORM.fields_by_name['align_to'].containing_oneof = _TIMESTAMPTRANSFORM.oneofs_by_name['timestamp_transform'] -_SIDEINPUT.fields_by_name['access_pattern'].message_type = _URNWITHPARAMETER -_SIDEINPUT.fields_by_name['view_fn'].message_type = _FUNCTIONSPEC -_SIDEINPUT.fields_by_name['window_mapping_fn'].message_type = _FUNCTIONSPEC -_FUNCTIONSPEC.fields_by_name['spec'].message_type = _URNWITHPARAMETER -_URNWITHPARAMETER.fields_by_name['parameter'].message_type = google_dot_protobuf_dot_any__pb2._ANY +_SIDEINPUT.fields_by_name['access_pattern'].message_type = _FUNCTIONSPEC +_SIDEINPUT.fields_by_name['view_fn'].message_type = _SDKFUNCTIONSPEC +_SIDEINPUT.fields_by_name['window_mapping_fn'].message_type = _SDKFUNCTIONSPEC +_SDKFUNCTIONSPEC.fields_by_name['spec'].message_type = _FUNCTIONSPEC +_FUNCTIONSPEC.fields_by_name['parameter'].message_type = google_dot_protobuf_dot_any__pb2._ANY _DISPLAYDATA_IDENTIFIER.containing_type = _DISPLAYDATA _DISPLAYDATA_ITEM.fields_by_name['id'].message_type = _DISPLAYDATA_IDENTIFIER _DISPLAYDATA_ITEM.fields_by_name['type'].enum_type = _DISPLAYDATA_TYPE @@ -2391,8 +2391,8 @@ DESCRIPTOR.message_types_by_name['TimestampTransform'] = _TIMESTAMPTRANSFORM DESCRIPTOR.message_types_by_name['SideInput'] = _SIDEINPUT DESCRIPTOR.message_types_by_name['Environment'] = _ENVIRONMENT +DESCRIPTOR.message_types_by_name['SdkFunctionSpec'] = _SDKFUNCTIONSPEC DESCRIPTOR.message_types_by_name['FunctionSpec'] = _FUNCTIONSPEC -DESCRIPTOR.message_types_by_name['UrnWithParameter'] = _URNWITHPARAMETER DESCRIPTOR.message_types_by_name['DisplayData'] = _DISPLAYDATA DESCRIPTOR.enum_types_by_name['IsBounded'] = _ISBOUNDED DESCRIPTOR.enum_types_by_name['MergeStatus'] = _MERGESTATUS @@ -2711,6 +2711,13 @@ )) _sym_db.RegisterMessage(Environment) +SdkFunctionSpec = _reflection.GeneratedProtocolMessageType('SdkFunctionSpec', (_message.Message,), dict( + DESCRIPTOR = _SDKFUNCTIONSPEC, + __module__ = 'beam_runner_api_pb2' + # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.SdkFunctionSpec) + )) +_sym_db.RegisterMessage(SdkFunctionSpec) + FunctionSpec = _reflection.GeneratedProtocolMessageType('FunctionSpec', (_message.Message,), dict( DESCRIPTOR = _FUNCTIONSPEC, __module__ = 'beam_runner_api_pb2' @@ -2718,13 +2725,6 @@ )) _sym_db.RegisterMessage(FunctionSpec) -UrnWithParameter = _reflection.GeneratedProtocolMessageType('UrnWithParameter', (_message.Message,), dict( - DESCRIPTOR = _URNWITHPARAMETER, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.UrnWithParameter) - )) -_sym_db.RegisterMessage(UrnWithParameter) - DisplayData = _reflection.GeneratedProtocolMessageType('DisplayData', (_message.Message,), dict( Identifier = _reflection.GeneratedProtocolMessageType('Identifier', (_message.Message,), dict( diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py index 6fa2f26ad3a1a..6d4e538dc0164 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py @@ -233,6 +233,7 @@ def __init__(self, packages, options, environment_version): options_dict = {k: v for k, v in sdk_pipeline_options.iteritems() if v is not None} + options_dict['_options_id'] = options._options_id self.proto.sdkPipelineOptions.additionalProperties.append( dataflow.Environment.SdkPipelineOptionsValue.AdditionalProperty( key='options', value=to_json_value(options_dict))) diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py index efad2e0c70876..1a5775f7f49ad 100644 --- a/sdks/python/apache_beam/runners/direct/direct_runner.py +++ b/sdks/python/apache_beam/runners/direct/direct_runner.py @@ -33,6 +33,7 @@ from apache_beam.runners.runner import PipelineState from apache_beam.runners.runner import PValueCache from apache_beam.utils.pipeline_options import DirectOptions +from apache_beam.utils.value_provider import RuntimeValueProvider class DirectRunner(PipelineRunner): @@ -86,6 +87,9 @@ def run(self, pipeline): evaluation_context) # Start the executor. This is a non-blocking call, it will start the # execution in background threads and return. + + if pipeline.options: + RuntimeValueProvider.set_runtime_options(pipeline.options._options_id, {}) executor.start(self.visitor.root_transforms) result = DirectPipelineResult(executor, evaluation_context) @@ -95,6 +99,11 @@ def run(self, pipeline): result.wait_until_finish() self._cache.finalize() + # Unset runtime options after the pipeline finishes. + # TODO: Move this to a post finish hook and clean for all cases. + if pipeline.options: + RuntimeValueProvider.unset_runtime_options(pipeline.options._options_id) + return result @property diff --git a/sdks/python/apache_beam/runners/runner.py b/sdks/python/apache_beam/runners/runner.py index b203c8badfb99..528b03f5fcb60 100644 --- a/sdks/python/apache_beam/runners/runner.py +++ b/sdks/python/apache_beam/runners/runner.py @@ -119,6 +119,35 @@ def visit_transform(self, transform_node): logging.error('Error while visiting %s', transform_node.full_label) raise + class GroupByKeyInputVisitor(PipelineVisitor): + """A visitor that replaces `Any` element type for input `PCollection` of + a `GroupByKey` with a `KV` type. + + TODO(BEAM-115): Once Python SDk is compatible with the new Runner API, + we could directly replace the coder instead of mutating the element type. + """ + def visit_transform(self, transform_node): + # Imported here to avoid circular dependencies. + # pylint: disable=wrong-import-order, wrong-import-position + from apache_beam import GroupByKey + from apache_beam import typehints + if isinstance(transform_node.transform, GroupByKey): + pcoll = transform_node.inputs[0] + input_type = pcoll.element_type + if not isinstance(input_type, typehints.TupleHint.TupleConstraint): + if isinstance(input_type, typehints.AnyTypeConstraint): + # `Any` type needs to be replaced with a KV[Any, Any] to + # force a KV coder as the main output coder for the pcollection + # preceding a GroupByKey. + pcoll.element_type = typehints.KV[typehints.Any, typehints.Any] + else: + # TODO: Handle other valid types, + # e.g. Union[KV[str, int], KV[str, float]] + raise ValueError( + "Input to GroupByKey must be of Tuple or Any type. " + "Found %s for %s" % (input_type, pcoll)) + + pipeline.visit(GroupByKeyInputVisitor()) pipeline.visit(RunVisitor(self)) def clear(self, pipeline, node=None): diff --git a/sdks/python/apache_beam/transforms/display.py b/sdks/python/apache_beam/transforms/display.py index 2ced1af2c123c..f2ce0fc98ac94 100644 --- a/sdks/python/apache_beam/transforms/display.py +++ b/sdks/python/apache_beam/transforms/display.py @@ -40,6 +40,7 @@ import inspect import json + __all__ = ['HasDisplayData', 'DisplayDataItem', 'DisplayData'] diff --git a/sdks/python/apache_beam/transforms/display_test.py b/sdks/python/apache_beam/transforms/display_test.py index 5e106e5319267..7d1130ba1ccf5 100644 --- a/sdks/python/apache_beam/transforms/display_test.py +++ b/sdks/python/apache_beam/transforms/display_test.py @@ -114,6 +114,42 @@ def display_data(self): with self.assertRaises(ValueError): DisplayData.create_from_options(MyDisplayComponent()) + def test_value_provider_display_data(self): + class TestOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_value_provider_argument( + '--int_flag', + type=int, + help='int_flag description') + parser.add_value_provider_argument( + '--str_flag', + type=str, + default='hello', + help='str_flag description') + parser.add_value_provider_argument( + '--float_flag', + type=float, + help='float_flag description') + options = TestOptions(['--int_flag', '1']) + items = DisplayData.create_from_options(options).items + expected_items = [ + DisplayDataItemMatcher( + 'int_flag', + '1'), + DisplayDataItemMatcher( + 'str_flag', + 'RuntimeValueProvider(option: str_flag,' + ' type: str, default_value: \'hello\')' + ), + DisplayDataItemMatcher( + 'float_flag', + 'RuntimeValueProvider(option: float_flag,' + ' type: float, default_value: None)' + ) + ] + hc.assert_that(items, hc.contains_inanyorder(*expected_items)) + def test_create_list_display_data(self): flags = ['--extra_package', 'package1', '--extra_package', 'package2'] pipeline_options = PipelineOptions(flags=flags) diff --git a/sdks/python/apache_beam/transforms/trigger.py b/sdks/python/apache_beam/transforms/trigger.py index e35c34988f8c6..5976af4a7293e 100644 --- a/sdks/python/apache_beam/transforms/trigger.py +++ b/sdks/python/apache_beam/transforms/trigger.py @@ -192,7 +192,7 @@ def from_runner_api(proto, context): 'after_all': AfterAll, 'after_any': AfterFirst, 'after_each': AfterEach, - 'after_end_of_widow': AfterWatermark, + 'after_end_of_window': AfterWatermark, # after_processing_time, after_synchronized_processing_time # always 'default': DefaultTrigger, @@ -338,12 +338,12 @@ def __hash__(self): def from_runner_api(proto, context): return AfterWatermark( early=TriggerFn.from_runner_api( - proto.after_end_of_widow.early_firings, context) - if proto.after_end_of_widow.HasField('early_firings') + proto.after_end_of_window.early_firings, context) + if proto.after_end_of_window.HasField('early_firings') else None, late=TriggerFn.from_runner_api( - proto.after_end_of_widow.late_firings, context) - if proto.after_end_of_widow.HasField('late_firings') + proto.after_end_of_window.late_firings, context) + if proto.after_end_of_window.HasField('late_firings') else None) def to_runner_api(self, context): @@ -352,7 +352,7 @@ def to_runner_api(self, context): late_proto = self.late.underlying.to_runner_api( context) if self.late else None return beam_runner_api_pb2.Trigger( - after_end_of_widow=beam_runner_api_pb2.Trigger.AfterEndOfWindow( + after_end_of_window=beam_runner_api_pb2.Trigger.AfterEndOfWindow( early_firings=early_proto, late_firings=late_proto)) diff --git a/sdks/python/apache_beam/transforms/window.py b/sdks/python/apache_beam/transforms/window.py index dcc58b7d41e15..319a7b4deccdc 100644 --- a/sdks/python/apache_beam/transforms/window.py +++ b/sdks/python/apache_beam/transforms/window.py @@ -158,8 +158,8 @@ def from_runner_api(cls, fn_proto, context): def to_runner_api(self, context): urn, typed_param = self.to_runner_api_parameter(context) - return beam_runner_api_pb2.FunctionSpec( - spec=beam_runner_api_pb2.UrnWithParameter( + return beam_runner_api_pb2.SdkFunctionSpec( + spec=beam_runner_api_pb2.FunctionSpec( urn=urn, parameter=proto_utils.pack_Any(typed_param))) diff --git a/sdks/python/apache_beam/utils/pipeline_options.py b/sdks/python/apache_beam/utils/pipeline_options.py index c2a44addacab5..769beb3e386ff 100644 --- a/sdks/python/apache_beam/utils/pipeline_options.py +++ b/sdks/python/apache_beam/utils/pipeline_options.py @@ -18,8 +18,81 @@ """Pipeline options obtained from command line parsing.""" import argparse +import itertools from apache_beam.transforms.display import HasDisplayData +from apache_beam.utils.value_provider import StaticValueProvider +from apache_beam.utils.value_provider import RuntimeValueProvider +from apache_beam.utils.value_provider import ValueProvider + + +def _static_value_provider_of(value_type): + """"Helper function to plug a ValueProvider into argparse. + + Args: + value_type: the type of the value. Since the type param of argparse's + add_argument will always be ValueProvider, we need to + preserve the type of the actual value. + Returns: + A partially constructed StaticValueProvider in the form of a function. + + """ + def _f(value): + _f.func_name = value_type.__name__ + return StaticValueProvider(value_type, value) + return _f + + +class BeamArgumentParser(argparse.ArgumentParser): + """An ArgumentParser that supports ValueProvider options. + + Example Usage:: + + class TemplateUserOptions(PipelineOptions): + @classmethod + + def _add_argparse_args(cls, parser): + parser.add_value_provider_argument('--vp-arg1', default='start') + parser.add_value_provider_argument('--vp-arg2') + parser.add_argument('--non-vp-arg') + + """ + def __init__(self, options_id, *args, **kwargs): + self._options_id = options_id + super(BeamArgumentParser, self).__init__(*args, **kwargs) + + def add_value_provider_argument(self, *args, **kwargs): + """ValueProvider arguments can be either of type keyword or positional. + At runtime, even positional arguments will need to be supplied in the + key/value form. + """ + # Extract the option name from positional argument ['pos_arg'] + assert args != () and len(args[0]) >= 1 + if args[0][0] != '-': + option_name = args[0] + if kwargs.get('nargs') is None: # make them optionally templated + kwargs['nargs'] = '?' + else: + # or keyword arguments like [--kw_arg, -k, -w] or [--kw-arg] + option_name = [i.replace('--', '') for i in args if i[:2] == '--'][0] + + # reassign the type to make room for using + # StaticValueProvider as the type for add_argument + value_type = kwargs.get('type') or str + kwargs['type'] = _static_value_provider_of(value_type) + + # reassign default to default_value to make room for using + # RuntimeValueProvider as the default for add_argument + default_value = kwargs.get('default') + kwargs['default'] = RuntimeValueProvider( + option_name=option_name, + value_type=value_type, + default_value=default_value, + options_id=self._options_id + ) + + # have add_argument do most of the work + self.add_argument(*args, **kwargs) class PipelineOptions(HasDisplayData): @@ -49,8 +122,9 @@ def _add_argparse_args(cls, parser): By default the options classes will use command line arguments to initialize the options. """ + _options_id_generator = itertools.count(1) - def __init__(self, flags=None, **kwargs): + def __init__(self, flags=None, options_id=None, **kwargs): """Initialize an options class. The initializer will traverse all subclasses, add all their argparse @@ -67,7 +141,10 @@ def __init__(self, flags=None, **kwargs): """ self._flags = flags self._all_options = kwargs - parser = argparse.ArgumentParser() + self._options_id = ( + options_id or PipelineOptions._options_id_generator.next()) + parser = BeamArgumentParser(self._options_id) + for cls in type(self).mro(): if cls == PipelineOptions: break @@ -119,13 +196,12 @@ def get_all_options(self, drop_default=False): # TODO(BEAM-1319): PipelineOption sub-classes in the main session might be # repeated. Pick last unique instance of each subclass to avoid conflicts. - parser = argparse.ArgumentParser() subset = {} + parser = BeamArgumentParser(self._options_id) for cls in PipelineOptions.__subclasses__(): subset[str(cls)] = cls for cls in subset.values(): cls._add_argparse_args(parser) # pylint: disable=protected-access - known_args, _ = parser.parse_known_args(self._flags) result = vars(known_args) @@ -133,7 +209,9 @@ def get_all_options(self, drop_default=False): for k in result.keys(): if k in self._all_options: result[k] = self._all_options[k] - if drop_default and parser.get_default(k) == result[k]: + if (drop_default and + parser.get_default(k) == result[k] and + not isinstance(parser.get_default(k), ValueProvider)): del result[k] return result @@ -142,7 +220,7 @@ def display_data(self): return self.get_all_options(True) def view_as(self, cls): - view = cls(self._flags) + view = cls(self._flags, options_id=self._options_id) view._all_options = self._all_options return view @@ -166,7 +244,7 @@ def __getattr__(self, name): (type(self).__name__, name)) def __setattr__(self, name, value): - if name in ('_flags', '_all_options', '_visible_options'): + if name in ('_flags', '_all_options', '_visible_options', '_options_id'): super(PipelineOptions, self).__setattr__(name, value) elif name in self._visible_option_list(): self._all_options[name] = value diff --git a/sdks/python/apache_beam/utils/pipeline_options_test.py b/sdks/python/apache_beam/utils/pipeline_options_test.py index 507a8275500d9..633d7daf5d80a 100644 --- a/sdks/python/apache_beam/utils/pipeline_options_test.py +++ b/sdks/python/apache_beam/utils/pipeline_options_test.py @@ -24,9 +24,13 @@ from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher from apache_beam.utils.pipeline_options import PipelineOptions +from apache_beam.utils.value_provider import StaticValueProvider +from apache_beam.utils.value_provider import RuntimeValueProvider class PipelineOptionsTest(unittest.TestCase): + def setUp(self): + RuntimeValueProvider.runtime_options_map = {} TEST_CASES = [ {'flags': ['--num_workers', '5'], @@ -131,7 +135,7 @@ def test_override_options(self): options.view_as(PipelineOptionsTest.MockOptions).mock_flag = True self.assertEqual(options.get_all_options()['num_workers'], 5) - self.assertEqual(options.get_all_options()['mock_flag'], True) + self.assertTrue(options.get_all_options()['mock_flag']) def test_experiments(self): options = PipelineOptions(['--experiment', 'abc', '--experiment', 'def']) @@ -185,7 +189,51 @@ def _add_argparse_args(cls, parser): parser.add_argument('--redefined_flag', action='store_true') options = PipelineOptions(['--redefined_flag']) - self.assertEqual(options.get_all_options()['redefined_flag'], True) + self.assertTrue(options.get_all_options()['redefined_flag']) + + def test_value_provider_options(self): + class UserOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_value_provider_argument( + '--vp_arg', + help='This flag is a value provider') + + parser.add_value_provider_argument( + '--vp_arg2', + default=1, + type=int) + + parser.add_argument( + '--non_vp_arg', + default=1, + type=int + ) + + # Provide values: if not provided, the option becomes of the type runtime vp + options = UserOptions(['--vp_arg', 'hello']) + self.assertIsInstance(options.vp_arg, StaticValueProvider) + self.assertIsInstance(options.vp_arg2, RuntimeValueProvider) + self.assertIsInstance(options.non_vp_arg, int) + + # Values can be overwritten + options = UserOptions(vp_arg=5, + vp_arg2=StaticValueProvider(value_type=str, + value='bye'), + non_vp_arg=RuntimeValueProvider( + option_name='foo', + value_type=int, + default_value=10, + options_id=10)) + self.assertEqual(options.vp_arg, 5) + self.assertTrue(options.vp_arg2.is_accessible(), + '%s is not accessible' % options.vp_arg2) + self.assertEqual(options.vp_arg2.get(), 'bye') + self.assertFalse(options.non_vp_arg.is_accessible()) + + with self.assertRaises(RuntimeError): + options.non_vp_arg.get() + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/sdks/python/apache_beam/utils/value_provider.py b/sdks/python/apache_beam/utils/value_provider.py new file mode 100644 index 0000000000000..a72fc4c71ce4d --- /dev/null +++ b/sdks/python/apache_beam/utils/value_provider.py @@ -0,0 +1,110 @@ +# +# 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. +# + +"""A ValueProvider class to implement templates with both statically +and dynamically provided values. +""" + +from functools import wraps + + +class ValueProvider(object): + def is_accessible(self): + raise NotImplementedError( + 'ValueProvider.is_accessible implemented in derived classes' + ) + + def get(self): + raise NotImplementedError( + 'ValueProvider.get implemented in derived classes' + ) + + +class StaticValueProvider(ValueProvider): + def __init__(self, value_type, value): + self.value_type = value_type + self.value = value_type(value) + + def is_accessible(self): + return True + + def get(self): + return self.value + + def __str__(self): + return str(self.value) + + +class RuntimeValueProvider(ValueProvider): + runtime_options_map = {} + + def __init__(self, option_name, value_type, default_value, options_id): + assert options_id is not None + self.option_name = option_name + self.default_value = default_value + self.value_type = value_type + self.options_id = options_id + + def is_accessible(self): + return RuntimeValueProvider.runtime_options_map.get( + self.options_id) is not None + + def get(self): + runtime_options = ( + RuntimeValueProvider.runtime_options_map.get(self.options_id)) + if runtime_options is None: + raise RuntimeError('%s.get() not called from a runtime context' % self) + + candidate = runtime_options.get(self.option_name) + if candidate: + value = self.value_type(candidate) + else: + value = self.default_value + return value + + @classmethod + def set_runtime_options(cls, options_id, pipeline_options): + assert options_id not in RuntimeValueProvider.runtime_options_map + RuntimeValueProvider.runtime_options_map[options_id] = pipeline_options + + @classmethod + def unset_runtime_options(cls, options_id): + assert options_id in RuntimeValueProvider.runtime_options_map + del RuntimeValueProvider.runtime_options_map[options_id] + + def __str__(self): + return '%s(option: %s, type: %s, default_value: %s)' % ( + self.__class__.__name__, + self.option_name, + self.value_type.__name__, + repr(self.default_value) + ) + + +def check_accessible(value_provider_list): + """Check accessibility of a list of ValueProvider objects.""" + assert isinstance(value_provider_list, list) + + def _check_accessible(fnc): + @wraps(fnc) + def _f(self, *args, **kwargs): + for obj in [getattr(self, vp) for vp in value_provider_list]: + if not obj.is_accessible(): + raise RuntimeError('%s not accessible' % obj) + return fnc(self, *args, **kwargs) + return _f + return _check_accessible diff --git a/sdks/python/apache_beam/utils/value_provider_test.py b/sdks/python/apache_beam/utils/value_provider_test.py new file mode 100644 index 0000000000000..83cb5e9eb038d --- /dev/null +++ b/sdks/python/apache_beam/utils/value_provider_test.py @@ -0,0 +1,165 @@ +# +# 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. +# + +"""Unit tests for the ValueProvider class.""" + +import unittest + +from apache_beam.utils.pipeline_options import PipelineOptions +from apache_beam.utils.value_provider import RuntimeValueProvider +from apache_beam.utils.value_provider import StaticValueProvider + + +class ValueProviderTests(unittest.TestCase): + def test_static_value_provider_keyword_argument(self): + class UserDefinedOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_value_provider_argument( + '--vp_arg', + help='This keyword argument is a value provider', + default='some value') + options = UserDefinedOptions(['--vp_arg', 'abc']) + self.assertTrue(isinstance(options.vp_arg, StaticValueProvider)) + self.assertTrue(options.vp_arg.is_accessible()) + self.assertEqual(options.vp_arg.get(), 'abc') + + def test_runtime_value_provider_keyword_argument(self): + class UserDefinedOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_value_provider_argument( + '--vp_arg', + help='This keyword argument is a value provider') + options = UserDefinedOptions() + self.assertTrue(isinstance(options.vp_arg, RuntimeValueProvider)) + self.assertFalse(options.vp_arg.is_accessible()) + with self.assertRaises(RuntimeError): + options.vp_arg.get() + + def test_static_value_provider_positional_argument(self): + class UserDefinedOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_value_provider_argument( + 'vp_pos_arg', + help='This positional argument is a value provider', + default='some value') + options = UserDefinedOptions(['abc']) + self.assertTrue(isinstance(options.vp_pos_arg, StaticValueProvider)) + self.assertTrue(options.vp_pos_arg.is_accessible()) + self.assertEqual(options.vp_pos_arg.get(), 'abc') + + def test_runtime_value_provider_positional_argument(self): + class UserDefinedOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_value_provider_argument( + 'vp_pos_arg', + help='This positional argument is a value provider') + options = UserDefinedOptions([]) + self.assertTrue(isinstance(options.vp_pos_arg, RuntimeValueProvider)) + self.assertFalse(options.vp_pos_arg.is_accessible()) + with self.assertRaises(RuntimeError): + options.vp_pos_arg.get() + + def test_static_value_provider_type_cast(self): + class UserDefinedOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_value_provider_argument( + '--vp_arg', + type=int, + help='This flag is a value provider') + + options = UserDefinedOptions(['--vp_arg', '123']) + self.assertTrue(isinstance(options.vp_arg, StaticValueProvider)) + self.assertTrue(options.vp_arg.is_accessible()) + self.assertEqual(options.vp_arg.get(), 123) + + def test_set_runtime_option(self): + # define ValueProvider ptions, with and without default values + class UserDefinedOptions1(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_value_provider_argument( + '--vp_arg', + help='This keyword argument is a value provider') # set at runtime + + parser.add_value_provider_argument( # not set, had default int + '-v', '--vp_arg2', # with short form + default=123, + type=int) + + parser.add_value_provider_argument( # not set, had default str + '--vp-arg3', # with dash in name + default='123', + type=str) + + parser.add_value_provider_argument( # not set and no default + '--vp_arg4', + type=float) + + parser.add_value_provider_argument( # positional argument set + 'vp_pos_arg', # default & runtime ignored + help='This positional argument is a value provider', + type=float, + default=5.4) + + # provide values at graph-construction time + # (options not provided here become of the type RuntimeValueProvider) + options = UserDefinedOptions1(['1.2']) + self.assertFalse(options.vp_arg.is_accessible()) + self.assertFalse(options.vp_arg2.is_accessible()) + self.assertFalse(options.vp_arg3.is_accessible()) + self.assertFalse(options.vp_arg4.is_accessible()) + self.assertTrue(options.vp_pos_arg.is_accessible()) + + # provide values at job-execution time + # (options not provided here will use their default, if they have one) + RuntimeValueProvider.set_runtime_options( + options._options_id, {'vp_arg': 'abc', 'vp_pos_arg':'3.2'}) + self.assertTrue(options.vp_arg.is_accessible()) + self.assertEqual(options.vp_arg.get(), 'abc') + self.assertTrue(options.vp_arg2.is_accessible()) + self.assertEqual(options.vp_arg2.get(), 123) + self.assertTrue(options.vp_arg3.is_accessible()) + self.assertEqual(options.vp_arg3.get(), '123') + self.assertTrue(options.vp_arg4.is_accessible()) + self.assertIsNone(options.vp_arg4.get()) + self.assertTrue(options.vp_pos_arg.is_accessible()) + self.assertEqual(options.vp_pos_arg.get(), 1.2) + + def test_options_id(self): + class Opt1(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_value_provider_argument('--arg1') + + class Opt2(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_value_provider_argument('--arg2') + + opt1 = Opt1() + opt2 = Opt2() + self.assertFalse(opt1.arg1.is_accessible()) + self.assertFalse(opt2.arg2.is_accessible()) + RuntimeValueProvider.set_runtime_options( + opt1.arg1.options_id, {'arg1': 'val1'}) + self.assertTrue(opt1.arg1.is_accessible()) + self.assertFalse(opt2.arg2.is_accessible())