Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit e47e716

Browse files
committed
Merge branch 'master' of https://github.com/apache/beam into users/damccorm/prismFallbackLogic
2 parents ed97204 + e27c927 commit e47e716

6 files changed

Lines changed: 68 additions & 33 deletions

File tree

runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleMonitoringInfoBuilder.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@
3030

3131
import java.util.HashMap;
3232
import java.util.Map;
33+
import java.util.Optional;
3334
import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo;
3435
import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfoSpec;
3536
import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfoSpecs;
@@ -212,4 +213,9 @@ public void merge(MonitoringInfo monitoringInfo) {
212213
}
213214
return result;
214215
}
216+
217+
public Optional<String> validate() {
218+
final MonitoringInfo result = this.builder.build();
219+
return VALIDATOR.validate(result);
220+
}
215221
}

sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java

Lines changed: 27 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
*/
1818
package org.apache.beam.fn.harness;
1919

20+
import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
2021
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;
2122
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables.getOnlyElement;
2223

@@ -37,6 +38,7 @@
3738
import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleSplitResponse;
3839
import org.apache.beam.model.fnexecution.v1.BeamFnApi.RemoteGrpcPort;
3940
import org.apache.beam.model.pipeline.v1.Endpoints;
41+
import org.apache.beam.model.pipeline.v1.MetricsApi;
4042
import org.apache.beam.model.pipeline.v1.RunnerApi;
4143
import org.apache.beam.runners.core.metrics.MonitoringInfoConstants;
4244
import org.apache.beam.runners.core.metrics.MonitoringInfoConstants.Urns;
@@ -59,9 +61,6 @@
5961
* Registers as a consumer for data over the Beam Fn API. Multiplexes any received data to all
6062
* receivers in a specified output map.
6163
*/
62-
@SuppressWarnings({
63-
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
64-
})
6564
public class BeamFnDataReadRunner<OutputT> {
6665

6766
private static final Logger LOG = LoggerFactory.getLogger(BeamFnDataReadRunner.class);
@@ -121,9 +120,9 @@ <OutputT> BeamFnDataReadRunner<OutputT> addReadRunnerForPTransform(Context conte
121120
private final Object splittingLock = new Object();
122121
// 0-based index of the current element being processed. -1 if we have yet to process an element.
123122
// stopIndex if we are done processing.
124-
private long index;
123+
private long index = -1;
125124
// 0-based index of the first element to not process, aka the first element of the residual
126-
private long stopIndex;
125+
private long stopIndex = Long.MAX_VALUE;
127126

128127
BeamFnDataReadRunner(
129128
ShortIdMap shortIdMap,
@@ -143,10 +142,17 @@ <OutputT> BeamFnDataReadRunner<OutputT> addReadRunnerForPTransform(Context conte
143142
this.consumer = consumer;
144143

145144
RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(components);
145+
146+
RunnerApi.Coder coderProto =
147+
checkArgumentNotNull(
148+
components.getCodersMap().get(port.getCoderId()),
149+
"Corrupt pipeline: coder %s not defined",
150+
port.getCoderId());
151+
146152
this.coder =
147153
(Coder<WindowedValue<OutputT>>)
148154
CoderTranslation.fromProto(
149-
components.getCodersMap().get(port.getCoderId()),
155+
coderProto,
150156
rehydratedComponents,
151157
new StateBackedIterableTranslationContext() {
152158
@Override
@@ -165,13 +171,21 @@ public Supplier<String> getCurrentInstructionId() {
165171
}
166172
});
167173

168-
dataChannelReadIndexShortId =
169-
shortIdMap.getOrCreateShortId(
170-
new SimpleMonitoringInfoBuilder()
171-
.setUrn(Urns.DATA_CHANNEL_READ_INDEX)
172-
.setType(MonitoringInfoConstants.TypeUrns.SUM_INT64_TYPE)
173-
.setLabel(MonitoringInfoConstants.Labels.PTRANSFORM, pTransformId)
174-
.build());
174+
SimpleMonitoringInfoBuilder monitoringInfoBuilder =
175+
new SimpleMonitoringInfoBuilder()
176+
.setUrn(Urns.DATA_CHANNEL_READ_INDEX)
177+
.setType(MonitoringInfoConstants.TypeUrns.SUM_INT64_TYPE)
178+
.setLabel(MonitoringInfoConstants.Labels.PTRANSFORM, pTransformId);
179+
180+
MetricsApi.MonitoringInfo monitoringInfo = monitoringInfoBuilder.build();
181+
if (monitoringInfo == null) {
182+
throw new IllegalArgumentException(
183+
String.format(
184+
"Internal error initializing BeamFnDataReadRunner: invalid monitoring info: %s",
185+
monitoringInfoBuilder.validate()));
186+
}
187+
188+
dataChannelReadIndexShortId = shortIdMap.getOrCreateShortId(monitoringInfo);
175189
addBundleProgressReporter.accept(
176190
new BundleProgressReporter() {
177191
@Override
@@ -202,8 +216,6 @@ public void reset() {
202216
// no-op
203217
}
204218
});
205-
206-
clearSplitIndices();
207219
}
208220

209221
public void forwardElementToConsumer(WindowedValue<OutputT> element) throws Exception {

sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java

Lines changed: 11 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
*/
1818
package org.apache.beam.fn.harness;
1919

20+
import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
2021
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables.getOnlyElement;
2122

2223
import com.google.auto.service.AutoService;
@@ -27,6 +28,7 @@
2728
import org.apache.beam.fn.harness.state.StateBackedIterable.StateBackedIterableTranslationContext;
2829
import org.apache.beam.model.fnexecution.v1.BeamFnApi;
2930
import org.apache.beam.model.fnexecution.v1.BeamFnApi.RemoteGrpcPort;
31+
import org.apache.beam.model.pipeline.v1.RunnerApi;
3032
import org.apache.beam.sdk.coders.Coder;
3133
import org.apache.beam.sdk.fn.data.RemoteGrpcPortWrite;
3234
import org.apache.beam.sdk.util.WindowedValue;
@@ -40,10 +42,7 @@
4042
*
4143
* <p>Can be re-used serially across {@link BeamFnApi.ProcessBundleRequest}s.
4244
*/
43-
@SuppressWarnings({
44-
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
45-
})
46-
public class BeamFnDataWriteRunner<InputT> {
45+
public class BeamFnDataWriteRunner {
4746

4847
/** A registrar which provides a factory to handle writing to the Fn Api Data Plane. */
4948
@AutoService(PTransformRunnerFactory.Registrar.class)
@@ -66,10 +65,17 @@ public void addRunnerForPTransform(Context context) throws IOException {
6665
private <InputT> void addWriteRunner(Context context) throws IOException {
6766
RemoteGrpcPort port = RemoteGrpcPortWrite.fromPTransform(context.getPTransform()).getPort();
6867
RehydratedComponents components = RehydratedComponents.forComponents(context.getComponents());
68+
69+
RunnerApi.Coder coderProto =
70+
checkArgumentNotNull(
71+
context.getComponents().getCodersMap().get(port.getCoderId()),
72+
"Corrupt pipeline: coder %s not defined",
73+
port.getCoderId());
74+
6975
Coder<WindowedValue<InputT>> coder =
7076
(Coder<WindowedValue<InputT>>)
7177
CoderTranslation.fromProto(
72-
context.getComponents().getCodersMap().get(port.getCoderId()),
78+
coderProto,
7379
components,
7480
new StateBackedIterableTranslationContext() {
7581
@Override

sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java

Lines changed: 21 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,9 @@
1717
*/
1818
package org.apache.beam.fn.harness;
1919

20+
import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
21+
import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
22+
2023
import com.google.auto.service.AutoService;
2124
import java.io.IOException;
2225
import java.util.Map;
@@ -40,11 +43,9 @@
4043
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
4144
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
4245
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
46+
import org.checkerframework.checker.nullness.qual.Nullable;
4347

4448
/** Executes different components of Combine PTransforms. */
45-
@SuppressWarnings({
46-
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
47-
})
4849
public class CombineRunners {
4950

5051
/** A registrar which provides a factory to handle combine component PTransforms. */
@@ -74,7 +75,7 @@ private static class PrecombineRunner<KeyT, InputT, AccumT> {
7475
private final CombineFn<InputT, AccumT, ?> combineFn;
7576
private final FnDataReceiver<WindowedValue<KV<KeyT, AccumT>>> output;
7677
private final Coder<KeyT> keyCoder;
77-
private PrecombineGroupingTable<KeyT, InputT, AccumT> groupingTable;
78+
private @Nullable PrecombineGroupingTable<KeyT, InputT, AccumT> groupingTable;
7879
private boolean isGloballyWindowed;
7980

8081
PrecombineRunner(
@@ -104,6 +105,12 @@ private static class PrecombineRunner<KeyT, InputT, AccumT> {
104105
this.isGloballyWindowed = isGloballyWindowed;
105106
}
106107

108+
private PrecombineGroupingTable<KeyT, InputT, AccumT> getGroupingTable() {
109+
return checkStateNotNull(
110+
this.groupingTable,
111+
"groupingTable not initialized. Did you call this outside of the context of bundle processing?");
112+
}
113+
107114
void startBundle() {
108115
groupingTable =
109116
PrecombineGroupingTable.combiningAndSampling(
@@ -116,11 +123,11 @@ void startBundle() {
116123
}
117124

118125
void processElement(WindowedValue<KV<KeyT, InputT>> elem) throws Exception {
119-
groupingTable.put(elem, output::accept);
126+
getGroupingTable().put(elem, output::accept);
120127
}
121128

122129
void finishBundle() throws Exception {
123-
groupingTable.flush(output::accept);
130+
getGroupingTable().flush(output::accept);
124131
groupingTable = null;
125132
}
126133
}
@@ -140,11 +147,15 @@ private <KeyT, InputT, AccumT> void addPrecombineRunner(Context context) throws
140147
RehydratedComponents.forComponents(context.getComponents());
141148
String mainInputTag =
142149
Iterables.getOnlyElement(context.getPTransform().getInputsMap().keySet());
150+
143151
RunnerApi.PCollection mainInput =
144-
context
145-
.getComponents()
146-
.getPcollectionsMap()
147-
.get(context.getPTransform().getInputsOrThrow(mainInputTag));
152+
checkArgumentNotNull(
153+
context
154+
.getComponents()
155+
.getPcollectionsMap()
156+
.get(context.getPTransform().getInputsOrThrow(mainInputTag)),
157+
"PreCombine missing main input for tag: %s",
158+
mainInputTag);
148159

149160
// Input coder may sometimes be WindowedValueCoder depending on runner, instead of the
150161
// expected KvCoder.

sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -138,9 +138,7 @@
138138
* differently.
139139
*/
140140
@SuppressWarnings({
141-
"rawtypes", // TODO(https://github.com/apache/beam/issues/20447)
142-
"nullness", // TODO(https://github.com/apache/beam/issues/20497)
143-
"keyfor"
141+
"rawtypes" // TODO(https://github.com/apache/beam/issues/20447)
144142
})
145143
public class FnApiDoFnRunner<InputT, RestrictionT, PositionT, WatermarkEstimatorStateT, OutputT> {
146144
/** A registrar which provides a factory to handle Java {@link DoFn}s. */

sdks/python/tox.ini

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -106,6 +106,7 @@ commands =
106106
[testenv:py{39,310,311}-ml]
107107
# Don't set TMPDIR to avoid "AF_UNIX path too long" errors in certain tests.
108108
deps =
109+
pip==25.0.1
109110
accelerate>=1.6.0
110111
setenv =
111112
extras = test,gcp,dataframe,ml_test
@@ -473,6 +474,7 @@ deps =
473474
latest: accelerate>=1.6.0
474475
tensorflow==2.12.0
475476
protobuf==4.25.5
477+
pip==25.0.1
476478
extras = test,gcp,ml_test
477479
commands =
478480
# Log transformers and its dependencies version for debugging

0 commit comments

Comments
 (0)