@@ -59,14 +59,15 @@ public class Combine {
5959 /**
6060 * Returns a {@link Globally Combine.Globally} {@code PTransform}
6161 * that uses the given {@code SerializableFunction} to combine all
62- * the elements of the input {@code PCollection} into a singleton
63- * {@code PCollection} value . The types of the input elements and the
64- * output value must be the same.
62+ * the elements in each window of the input {@code PCollection} into a
63+ * single value in the output {@code PCollection}. The types of the input
64+ * elements and the output elements must be the same.
6565 *
66- * <p>If the input {@code PCollection} is empty, the ouput will contain a the
67- * default value of the combining function if the input is windowed into
68- * the {@link GlobalWindows}; otherwise, the output will be empty. Note: this
69- * behavior is subject to change.
66+ * <p> If the input {@code PCollection} is windowed into {@link GlobalWindows},
67+ * a default value in the {@link GlobalWindow} will be output if the input
68+ * {@code PCollection} is empty. To use this with inputs with other windowing,
69+ * either {@link Globally#withoutDefaults} or {@link Globally#asSingletonView}
70+ * must be called.
7071 *
7172 * <p> See {@link Globally Combine.Globally} for more information.
7273 */
@@ -77,21 +78,22 @@ public static <V> Globally<V, V> globally(
7778
7879 /**
7980 * Returns a {@link Globally Combine.Globally} {@code PTransform}
80- * that uses the given {@code CombineFn } to combine all the elements
81- * of the input {@code PCollection} into a singleton {@code PCollection}
82- * value. The types of the input elements and the output value can
83- * differ.
81+ * that uses the given {@code SerializableFunction } to combine all
82+ * the elements in each window of the input {@code PCollection} into a
83+ * single value in the output {@code PCollection} . The types of the input
84+ * elements and the output elements can differ
8485 *
85- * If the input {@code PCollection} is empty, the ouput will contain a the
86- * default value of the combining function if the input is windowed into
87- * the {@link GlobalWindows}; otherwise, the output will be empty. Note: this
88- * behavior is subject to change.
86+ * <p> If the input {@code PCollection} is windowed into {@link GlobalWindows},
87+ * a default value in the {@link GlobalWindow} will be output if the input
88+ * {@code PCollection} is empty. To use this with inputs with other windowing,
89+ * either {@link Globally#withoutDefaults} or {@link Globally#asSingletonView}
90+ * must be called.
8991 *
9092 * <p> See {@link Globally Combine.Globally} for more information.
9193 */
9294 public static <VI , VO > Globally <VI , VO > globally (
9395 CombineFn <? super VI , ?, VO > fn ) {
94- return new Globally <>(fn );
96+ return new Globally <>(fn , true );
9597 }
9698
9799 /**
@@ -1053,10 +1055,9 @@ public Coder<VO> getDefaultOutputCoder(
10531055
10541056 /**
10551057 * {@code Combine.Globally<VI, VO>} takes a {@code PCollection<VI>}
1056- * and returns a {@code PCollection<VO>} whose single element is the result of
1057- * combining all the elements of the input {@code PCollection},
1058- * using a specified}
1059- * {@link CombineFn CombineFn<VI, VA, VO>}. It is common
1058+ * and returns a {@code PCollection<VO>} whose elements are the result of
1059+ * combining all the elements in each window of the input {@code PCollection},
1060+ * using a specified {@link CombineFn CombineFn<VI, VA, VO>}. It is common
10601061 * for {@code VI == VO}, but not required. Common combining
10611062 * functions include sums, mins, maxes, and averages of numbers,
10621063 * conjunctions and disjunctions of booleans, statistical
@@ -1074,6 +1075,11 @@ public Coder<VO> getDefaultOutputCoder(
10741075 * intermediate results combined further, in an arbitrary tree
10751076 * reduction pattern, until a single result value is produced.
10761077 *
1078+ * <p> If the input {@code PCollection} is windowed into {@link GlobalWindows},
1079+ * a default value in the {@link GlobalWindow} will be output if the input
1080+ * {@code PCollection} is empty. To use this with inputs with other windowing,
1081+ * either {@link #withoutDefaults} or {@link #asSingletonView} must be called.
1082+ *
10771083 * <p> By default, the {@code Coder} of the output {@code PValue<VO>}
10781084 * is inferred from the concrete type of the
10791085 * {@code CombineFn<VI, VA, VO>}'s output type {@code VO}.
@@ -1090,9 +1096,36 @@ public static class Globally<VI, VO>
10901096 extends PTransform <PCollection <VI >, PCollection <VO >> {
10911097
10921098 private final CombineFn <? super VI , ?, VO > fn ;
1099+ private final boolean insertDefault ;
10931100
1094- private Globally (CombineFn <? super VI , ?, VO > fn ) {
1101+ private Globally (CombineFn <? super VI , ?, VO > fn , boolean insertDefault ) {
10951102 this .fn = fn ;
1103+ this .insertDefault = insertDefault ;
1104+ }
1105+
1106+ @ Override
1107+ @ SuppressWarnings ("unchecked" )
1108+ public Globally <VI , VO > withName (String name ) {
1109+ return (Globally <VI , VO >) super .withName (name );
1110+ }
1111+
1112+ /**
1113+ * Returns a {@link PTransform} that produces a {@code PCollectionView}
1114+ * whose elements are the result of combining elements per-window in
1115+ * the input {@code PCollection}. If a value is requested from the view
1116+ * for a window that is not present, the result of calling the {@code CombineFn}
1117+ * on empty input will returned.
1118+ */
1119+ public GloballyAsSingletonView <VI , VO > asSingletonView () {
1120+ return new GloballyAsSingletonView <>(fn , insertDefault );
1121+ }
1122+
1123+ /**
1124+ * Returns a {@link PTransform} identical to this, but that does not attempt to
1125+ * provide a default value in the case of empty input.
1126+ */
1127+ public Globally <VI , VO > withoutDefaults () {
1128+ return new Globally <>(fn , false );
10961129 }
10971130
10981131 @ Override
@@ -1103,7 +1136,13 @@ public PCollection<VO> apply(PCollection<VI> input) {
11031136 .apply (Combine .<Void , VI , VO >perKey (fn .<Void >asKeyedFn ()))
11041137 .apply (Values .<VO >create ());
11051138
1106- if (input .getWindowFn ().isCompatible (new GlobalWindows ())) {
1139+ if (insertDefault ) {
1140+ if (!output .getWindowFn ().isCompatible (new GlobalWindows ())) {
1141+ throw new IllegalStateException (
1142+ "Attempted to add default value to PCollection not windowed by GlobalWindows. "
1143+ + "Instead, use Combine.globally().withoutDefaults() or "
1144+ + "Combine.globally().asSingletonView()." );
1145+ }
11071146 return insertDefaultValueIfEmpty (output );
11081147 } else {
11091148 return output ;
@@ -1117,15 +1156,15 @@ private PCollection<VO> insertDefaultValueIfEmpty(PCollection<VO> maybeEmpty) {
11171156 .apply (Create .of ((Void ) null )).setCoder (VoidCoder .of ())
11181157 .apply (ParDo .of (
11191158 new DoFn <Void , VO >() {
1120- @ Override
1121- public void processElement (DoFn <Void , VO >.ProcessContext c ) {
1122- Iterator <VO > combined = c .sideInput (maybeEmptyView ).iterator ();
1123- if (combined .hasNext ()) {
1124- c .output (combined .next ());
1125- } else {
1126- c .output (fn .apply (Collections .<VI >emptyList ()));
1127- }
1159+ @ Override
1160+ public void processElement (DoFn <Void , VO >.ProcessContext c ) {
1161+ Iterator <VO > combined = c .sideInput (maybeEmptyView ).iterator ();
1162+ if (combined .hasNext ()) {
1163+ c .output (combined .next ());
1164+ } else {
1165+ c .output (fn .apply (Collections .<VI >emptyList ()));
11281166 }
1167+ }
11291168 }).withSideInputs (maybeEmptyView ))
11301169 .setCoder (maybeEmpty .getCoder ());
11311170 }
@@ -1136,6 +1175,81 @@ protected String getKindString() {
11361175 }
11371176 }
11381177
1178+ /**
1179+ * {@code Combine.GloballyAsSingletonView<VI, VO>} takes a {@code PCollection<VI>}
1180+ * and returns a {@code PCollectionView<VO>} whose elements are the result of
1181+ * combining all the elements in each window of the input {@code PCollection},
1182+ * using a specified {@link CombineFn CombineFn<VI, VA, VO>}. It is common for
1183+ * {@code VI == VO}, but not required. Common combining
1184+ * functions include sums, mins, maxes, and averages of numbers,
1185+ * conjunctions and disjunctions of booleans, statistical
1186+ * aggregations, etc.
1187+ *
1188+ * <p> Example of use:
1189+ * <pre> {@code
1190+ * PCollection<Integer> pc = ...;
1191+ * PCollection<Integer> sum = pc.apply(
1192+ * Combine.globally(new Sum.SumIntegerFn()));
1193+ * } </pre>
1194+ *
1195+ * <p> Combining can happen in parallel, with different subsets of the
1196+ * input {@code PCollection} being combined separately, and their
1197+ * intermediate results combined further, in an arbitrary tree
1198+ * reduction pattern, until a single result value is produced.
1199+ *
1200+ * <p> If a value is requested from the view for a window that is not present
1201+ * and {@code insertDefault} is true, the result of calling the {@code CombineFn}
1202+ * on empty input will returned. If {@code insertDefault} is false, an
1203+ * exception will be thrown instead.
1204+ *
1205+ * <p> By default, the {@code Coder} of the output {@code PValue<VO>}
1206+ * is inferred from the concrete type of the
1207+ * {@code CombineFn<VI, VA, VO>}'s output type {@code VO}.
1208+ *
1209+ * <p> See also {@link #perKey}/{@link PerKey Combine.PerKey} and
1210+ * {@link #groupedValues}/{@link GroupedValues Combine.GroupedValues},
1211+ * which are useful for combining values associated with each key in
1212+ * a {@code PCollection} of {@code KV}s.
1213+ *
1214+ * @param <VI> type of input values
1215+ * @param <VO> type of output values
1216+ */
1217+ public static class GloballyAsSingletonView <VI , VO >
1218+ extends PTransform <PCollection <VI >, PCollectionView <VO >> {
1219+
1220+ private final CombineFn <? super VI , ?, VO > fn ;
1221+ private final boolean insertDefault ;
1222+
1223+ private GloballyAsSingletonView (CombineFn <? super VI , ?, VO > fn , boolean insertDefault ) {
1224+ this .fn = fn ;
1225+ this .insertDefault = insertDefault ;
1226+ }
1227+
1228+ @ Override
1229+ @ SuppressWarnings ("unchecked" )
1230+ public GloballyAsSingletonView <VI , VO > withName (String name ) {
1231+ return (GloballyAsSingletonView <VI , VO >) super .withName (name );
1232+ }
1233+
1234+ @ Override
1235+ public PCollectionView <VO > apply (PCollection <VI > input ) {
1236+ PCollection <VO > combined = input
1237+ .apply (Combine .globally (fn ).withoutDefaults ());
1238+ if (insertDefault ) {
1239+ return combined
1240+ .apply (View .<VO >asSingleton ().withDefaultValue (
1241+ fn .apply (Collections .<VI >emptyList ())));
1242+ } else {
1243+ return combined .apply (View .<VO >asSingleton ());
1244+ }
1245+ }
1246+
1247+ @ Override
1248+ protected String getKindString () {
1249+ return "Combine.GloballyAsSingletonView" ;
1250+ }
1251+ }
1252+
11391253 /**
11401254 * Converts a {@link SerializableFunction} from {@code Iterable<V>}s
11411255 * to {@code V}s into a simple {@link CombineFn} over {@code V}s.
0 commit comments