From cd630e4dd4b819e7e804683493361f5be19c64a3 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Wed, 25 Jan 2023 17:03:11 -0800 Subject: [PATCH 01/42] Data Sampling Java Impl --- .../org/apache/beam/fn/harness/FnHarness.java | 42 ++++++- .../fn/harness/PTransformRunnerFactory.java | 9 ++ .../ProcessBundleDescriptorModifier.java | 38 ++++++ .../harness/control/ProcessBundleHandler.java | 32 ++--- .../beam/fn/harness/debug/DataSampler.java | 110 ++++++++++++++++++ .../debug/DataSamplingDescriptorModifier.java | 29 +++++ .../harness/debug/DataSamplingFnRunner.java | 110 ++++++++++++++++++ .../PTransformRunnerFactoryTestContext.java | 3 + .../control/ProcessBundleHandlerTest.java | 48 +++++--- .../debug/DataSamplerModifierTest.java | 91 +++++++++++++++ .../fn/harness/debug/DataSamplerTest.java | 93 +++++++++++++++ .../debug/DataSamplingFnRunnerTest.java | 110 ++++++++++++++++++ 12 files changed, 680 insertions(+), 35 deletions(-) create mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java create mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java create mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java create mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java create mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerModifierTest.java create mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java create mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index 4c91874e63f6..a85d878ea5dc 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -17,8 +17,10 @@ */ package org.apache.beam.fn.harness; +import java.util.ArrayList; import java.util.Collections; import java.util.EnumMap; +import java.util.List; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.function.Function; @@ -29,6 +31,8 @@ import org.apache.beam.fn.harness.control.HarnessMonitoringInfosInstructionHandler; import org.apache.beam.fn.harness.control.ProcessBundleHandler; import org.apache.beam.fn.harness.data.BeamFnDataGrpcClient; +import org.apache.beam.fn.harness.debug.DataSampler; +import org.apache.beam.fn.harness.debug.DataSamplingDescriptorModifier; import org.apache.beam.fn.harness.logging.BeamFnLoggingClient; import org.apache.beam.fn.harness.state.BeamFnStateGrpcClientCache; import org.apache.beam.fn.harness.status.BeamFnStatusClient; @@ -91,6 +95,10 @@ public class FnHarness { private static final String RUNNER_CAPABILITIES = "RUNNER_CAPABILITIES"; private static final Logger LOG = LoggerFactory.getLogger(FnHarness.class); + private static final String ENABLE_DATA_SAMPLING_EXPERIMENT = "enable_data_sampling"; + + private static final DataSampler dataSampler = new DataSampler(); + private static Endpoints.ApiServiceDescriptor getApiServiceDescriptor(String descriptor) throws TextFormat.ParseException { Endpoints.ApiServiceDescriptor.Builder apiServiceDescriptorBuilder = @@ -248,22 +256,45 @@ public static void main( FinalizeBundleHandler finalizeBundleHandler = new FinalizeBundleHandler(executorService); + // Add any graph modifications. + List modifierList = new ArrayList<>(); + List experimentList = options.as(ExperimentalOptions.class).getExperiments(); + + if (experimentList != null && experimentList.contains(ENABLE_DATA_SAMPLING_EXPERIMENT)) { + modifierList.add(new DataSamplingDescriptorModifier()); + } + + // Retrieves the ProcessBundleDescriptor from cache. Requests the PBD from the Runner if it + // doesn't exist. Additionally, runs any graph modifications. Function getProcessBundleDescriptor = new Function() { private static final String PROCESS_BUNDLE_DESCRIPTORS = "ProcessBundleDescriptors"; private final Cache cache = Caches.subCache(processWideCache, PROCESS_BUNDLE_DESCRIPTORS); + private final List modifiers = modifierList; + @Override public BeamFnApi.ProcessBundleDescriptor apply(String id) { return cache.computeIfAbsent(id, this::loadDescriptor); } private BeamFnApi.ProcessBundleDescriptor loadDescriptor(String id) { - return blockingControlStub.getProcessBundleDescriptor( - BeamFnApi.GetProcessBundleDescriptorRequest.newBuilder() - .setProcessBundleDescriptorId(id) - .build()); + ProcessBundleDescriptor descriptor = + blockingControlStub.getProcessBundleDescriptor( + BeamFnApi.GetProcessBundleDescriptorRequest.newBuilder() + .setProcessBundleDescriptorId(id) + .build()); + for (ProcessBundleDescriptorModifier modifier : modifiers) { + try { + LOG.debug("Modifying graph with " + modifier); + descriptor = modifier.ModifyProcessBundleDescriptor(descriptor); + } catch (ProcessBundleDescriptorModifier.GraphModificationException e) { + LOG.warn("Could not modify graph with " + modifier + ": " + e.getMessage()); + } + } + + return descriptor; } }; @@ -279,7 +310,8 @@ private BeamFnApi.ProcessBundleDescriptor loadDescriptor(String id) { finalizeBundleHandler, metricsShortIds, executionStateSampler, - processWideCache); + processWideCache, + dataSampler); logging.setProcessBundleHandler(processBundleHandler); BeamFnStatusClient beamFnStatusClient = null; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java index e7b29550afdf..418a08bc302f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java @@ -20,11 +20,13 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.function.Supplier; import org.apache.beam.fn.harness.control.BundleProgressReporter; import org.apache.beam.fn.harness.control.BundleSplitListener; import org.apache.beam.fn.harness.data.BeamFnDataClient; +import org.apache.beam.fn.harness.debug.DataSampler; import org.apache.beam.fn.harness.state.BeamFnStateClient; import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleRequest; import org.apache.beam.model.pipeline.v1.Endpoints; @@ -163,6 +165,13 @@ void addIncomingTimerEndpoint( * instant provides the timeout on how long the finalization callback is valid for. */ DoFn.BundleFinalizer getBundleFinalizer(); + + /** + * A DataSampler can be used to sample in-flight elements. This is used to plumb a global + * DataSampler to DataSampler operations in order to perform said sampling. Only present when + * using the "enable_data_sampling" experiment. + */ + Optional getDataSampler(); } /** diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java new file mode 100644 index 000000000000..893109ad7335 --- /dev/null +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java @@ -0,0 +1,38 @@ +/* + * 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.fn.harness; + +import org.apache.beam.model.fnexecution.v1.BeamFnApi; + +public interface ProcessBundleDescriptorModifier { + class GraphModificationException extends Exception { + public GraphModificationException() {} + + public GraphModificationException(String message) { + super(message); + } + } + + /** + * Modifies the given ProcessBundleDescriptor in-place. Throws a `GraphModificationException` on + * failure. Can be used to instrument functionality onto an existing ProcessBundleDescriptor. For + * instance, this is used to add DataSampling PTransforms to a graph to sample in-flight elements. + */ + BeamFnApi.ProcessBundleDescriptor ModifyProcessBundleDescriptor( + BeamFnApi.ProcessBundleDescriptor pbd) throws GraphModificationException; +} diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index 560369a3907a..6992dfffbb79 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -20,17 +20,7 @@ import com.google.auto.value.AutoValue; import java.io.IOException; import java.time.Duration; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.ServiceLoader; -import java.util.Set; -import java.util.WeakHashMap; +import java.util.*; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Phaser; @@ -52,6 +42,7 @@ import org.apache.beam.fn.harness.data.BeamFnDataClient; import org.apache.beam.fn.harness.data.PCollectionConsumerRegistry; import org.apache.beam.fn.harness.data.PTransformFunctionRegistry; +import org.apache.beam.fn.harness.debug.DataSampler; import org.apache.beam.fn.harness.state.BeamFnStateClient; import org.apache.beam.fn.harness.state.BeamFnStateGrpcClientCache; import org.apache.beam.model.fnexecution.v1.BeamFnApi; @@ -106,6 +97,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static java.util.Optional.ofNullable; + /** * Processes {@link BeamFnApi.ProcessBundleRequest}s and {@link * BeamFnApi.ProcessBundleSplitRequest}s. @@ -165,6 +158,8 @@ public class ProcessBundleHandler { @VisibleForTesting final BundleProcessorCache bundleProcessorCache; private final Set runnerCapabilities; + private DataSampler dataSampler; + public ProcessBundleHandler( PipelineOptions options, Set runnerCapabilities, @@ -174,7 +169,8 @@ public ProcessBundleHandler( FinalizeBundleHandler finalizeBundleHandler, ShortIdMap shortIds, ExecutionStateSampler executionStateSampler, - Cache processWideCache) { + Cache processWideCache, + DataSampler dataSampler) { this( options, runnerCapabilities, @@ -186,7 +182,8 @@ public ProcessBundleHandler( executionStateSampler, REGISTERED_RUNNER_FACTORIES, processWideCache, - new BundleProcessorCache()); + new BundleProcessorCache(), + dataSampler); } @VisibleForTesting @@ -201,7 +198,8 @@ public ProcessBundleHandler( ExecutionStateSampler executionStateSampler, Map urnToPTransformRunnerFactoryMap, Cache processWideCache, - BundleProcessorCache bundleProcessorCache) { + BundleProcessorCache bundleProcessorCache, + DataSampler dataSampler) { this.options = options; this.fnApiRegistry = fnApiRegistry; this.beamFnDataClient = beamFnDataClient; @@ -218,6 +216,7 @@ public ProcessBundleHandler( new UnknownPTransformRunnerFactory(urnToPTransformRunnerFactoryMap.keySet()); this.processWideCache = processWideCache; this.bundleProcessorCache = bundleProcessorCache; + this.dataSampler = dataSampler; } private void createRunnerAndConsumersForPTransformRecursively( @@ -481,6 +480,11 @@ public BundleSplitListener getSplitListener() { public BundleFinalizer getBundleFinalizer() { return bundleFinalizer; } + + @Override + public Optional getDataSampler() { + return Optional.ofNullable(dataSampler); + } }); if (runner instanceof BeamFnDataReadRunner) { channelRoots.add((BeamFnDataReadRunner) runner); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java new file mode 100644 index 000000000000..e408cb47d2c1 --- /dev/null +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -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. + */ +package org.apache.beam.fn.harness.debug; + +import org.apache.beam.fn.harness.FnHarness; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class DataSampler { + public static class OutputSampler { + private final Coder coder; + private final List buffer = new ArrayList<>(); + private static final Logger LOG = LoggerFactory.getLogger(OutputSampler.class); + + private final int maxElements = 10; + + private final int sampleEveryN = 1000; + private long numSamples = 0; + + private int resampleIndex = 0; + + public OutputSampler(Coder coder) { + this.coder = coder; + } + + public void sample(T element) { + // Only sample the first 10 elements then after every `sampleEveryN`th element. + numSamples += 1; + if (numSamples > 10 && numSamples % sampleEveryN != 0) { + return; + } + + // Fill buffer until maxElements. + if (buffer.size() < maxElements) { + buffer.add(element); + } else { + // Then rewrite sampled elements as a circular buffer. + buffer.set(resampleIndex, element); + resampleIndex = (resampleIndex + 1) % maxElements; + } + } + + public List samples() { + List ret = new ArrayList<>(); + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + for (T el : buffer) { + try { + coder.encode(el, stream); + ret.add(stream.toByteArray()); + } catch (Exception exception) { + LOG.warn("Could not encode element \"" + el + "\" to bytes: " + exception); + } + } + + clear(); + return ret; + } + + private void clear() { + buffer.clear(); + resampleIndex = 0; + } + } + + private final Map> outputSamplers = new HashMap<>(); + + public OutputSampler sampleOutput(String outputName, Coder coder) { + if (outputSamplers.containsKey(outputName)) { + return (OutputSampler)outputSamplers.get(outputName); + } + + OutputSampler sampler = new OutputSampler(coder); + outputSamplers.put(outputName, sampler); + return sampler; + } + + public Map> samples() { + Map> samples = new HashMap<>(); + outputSamplers.forEach((pcollectionId, outputSampler) -> { + samples.putIfAbsent(pcollectionId, new ArrayList<>()); + samples.get(pcollectionId).addAll(outputSampler.samples()); + }); + return samples; + } +} diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java new file mode 100644 index 000000000000..5f5996a78d82 --- /dev/null +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java @@ -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. + */ +package org.apache.beam.fn.harness.debug; + +import org.apache.beam.fn.harness.ProcessBundleDescriptorModifier; +import org.apache.beam.model.fnexecution.v1.BeamFnApi; + +public class DataSamplingDescriptorModifier implements ProcessBundleDescriptorModifier { + @Override + public BeamFnApi.ProcessBundleDescriptor ModifyProcessBundleDescriptor( + BeamFnApi.ProcessBundleDescriptor pbd) throws GraphModificationException { + return pbd.toBuilder().build(); + } +} diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java new file mode 100644 index 000000000000..9b6aeac1de46 --- /dev/null +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java @@ -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. + */ +package org.apache.beam.fn.harness.debug; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables.getOnlyElement; + +import com.google.auto.service.AutoService; +import java.io.IOException; +import java.util.Map; +import java.util.Optional; + +import org.apache.beam.fn.harness.FlattenRunner; +import org.apache.beam.fn.harness.PTransformRunnerFactory; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.runners.core.construction.RehydratedComponents; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.fn.data.FnDataReceiver; +import org.apache.beam.sdk.function.ThrowingFunction; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({ + "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) +}) +public class DataSamplingFnRunner { + public static final String URN = "beam:internal:sampling:v1"; + + private static final Logger LOG = LoggerFactory.getLogger(DataSamplingFnRunner.class); + + @AutoService(PTransformRunnerFactory.Registrar.class) + public static class Registrar implements PTransformRunnerFactory.Registrar { + + @Override + public Map getPTransformRunnerFactories() { + return ImmutableMap.of(URN, new Factory()); + } + } + + public static class Factory + implements PTransformRunnerFactory { + @Override + public DataSamplingFnRunner createRunnerForPTransform(Context context) + throws IOException { + DataSamplingFnRunner runner = new DataSamplingFnRunner(); + + Optional maybeDataSampler = context.getDataSampler(); + if (!maybeDataSampler.isPresent()) { + LOG.warn("Trying to sample output but DataSampler is not present. Is " + + "\"enable_data_sampling\" set?"); + return runner; + } + + DataSampler dataSampler = maybeDataSampler.get(); + + String inputPCollectionId = + Iterables.getOnlyElement(context.getPTransform().getInputsMap().values()); + + RunnerApi.PCollection inputPCollection = context + .getPCollections() + .get(inputPCollectionId); + + if (inputPCollection == null) { + LOG.warn("Expected input PCollection \"" + inputPCollectionId + "\" does not exist in " + + "PCollections map."); + return runner; + } + + String inputCoderId = inputPCollection.getCoderId(); + + RehydratedComponents rehydratedComponents = + RehydratedComponents.forComponents( + RunnerApi.Components.newBuilder() + .putAllCoders(context.getCoders()) + .putAllPcollections(context.getPCollections()) + .putAllWindowingStrategies(context.getWindowingStrategies()) + .build()) + .withPipeline(Pipeline.create()); + Coder inputCoder = (Coder)rehydratedComponents.getCoder(inputCoderId); + + DataSampler.OutputSampler outputSampler = dataSampler.sampleOutput( + inputPCollectionId, + inputCoder); + + String pCollectionId = + Iterables.getOnlyElement(context.getPTransform().getInputsMap().values()); + context.addPCollectionConsumer(pCollectionId, + (FnDataReceiver>) input -> outputSampler.sample(input.getValue())); + return runner; + } + } +} diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java index 57fe54156692..f84676df7d2f 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java @@ -31,6 +31,7 @@ import org.apache.beam.fn.harness.control.BundleProgressReporter; import org.apache.beam.fn.harness.control.BundleSplitListener; import org.apache.beam.fn.harness.data.BeamFnDataClient; +import org.apache.beam.fn.harness.debug.DataSampler; import org.apache.beam.fn.harness.state.BeamFnStateClient; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.fnexecution.v1.BeamFnApi.BundleApplication; @@ -206,6 +207,8 @@ default Builder processBundleInstructionId(String value) { Builder timerApiServiceDescriptor(ApiServiceDescriptor value); + Builder dataSampler(DataSampler dataSampler); + PTransformRunnerFactoryTestContext build(); } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java index 7df9ed2f894d..52bb72f97894 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java @@ -376,7 +376,8 @@ public void testTrySplitBeforeBundleDoesNotFail() { executionStateSampler, ImmutableMap.of(), Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); BeamFnApi.InstructionResponse response = handler @@ -406,7 +407,8 @@ public void testProgressBeforeBundleDoesNotFail() throws Exception { executionStateSampler, ImmutableMap.of(), Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); handler.progress( BeamFnApi.InstructionRequest.newBuilder() @@ -485,7 +487,8 @@ public void testOrderOfStartAndFinishCalls() throws Exception { DATA_INPUT_URN, startFinishRecorder, DATA_OUTPUT_URN, startFinishRecorder), Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); handler.processBundle( BeamFnApi.InstructionRequest.newBuilder() @@ -589,7 +592,8 @@ public void testOrderOfSetupTeardownCalls() throws Exception { executionStateSampler, urnToPTransformRunnerFactoryMap, Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); handler.processBundle( BeamFnApi.InstructionRequest.newBuilder() @@ -640,7 +644,8 @@ public void testBundleProcessorIsResetWhenAddedBackToCache() throws Exception { executionStateSampler, ImmutableMap.of(DATA_INPUT_URN, (context) -> null), Caches.noop(), - new TestBundleProcessorCache()); + new TestBundleProcessorCache(), + null /* dataSampler */); assertThat(TestBundleProcessor.resetCnt, equalTo(0)); @@ -806,7 +811,8 @@ public void testCreatingPTransformExceptionsArePropagated() throws Exception { throw new IllegalStateException("TestException"); }), Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); assertThrows( "TestException", IllegalStateException.class, @@ -856,7 +862,8 @@ public void testBundleFinalizationIsPropagated() throws Exception { return null; }), Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); BeamFnApi.InstructionResponse.Builder response = handler.processBundle( BeamFnApi.InstructionRequest.newBuilder() @@ -909,7 +916,8 @@ public void testPTransformStartExceptionsArePropagated() { return null; }), Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); assertThrows( "TestException", IllegalStateException.class, @@ -1086,7 +1094,8 @@ public void onCompleted() {} executionStateSampler, urnToPTransformRunnerFactoryMap, Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); } @Test @@ -1418,7 +1427,8 @@ public void testInstructionIsUnregisteredFromBeamFnDataClientOnSuccess() throws return null; }), Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); handler.processBundle( BeamFnApi.InstructionRequest.newBuilder() .setInstructionId("instructionId") @@ -1490,7 +1500,8 @@ public void testDataProcessingExceptionsArePropagated() throws Exception { return null; }), Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); assertThrows( "TestException", IllegalStateException.class, @@ -1539,7 +1550,8 @@ public void testPTransformFinishExceptionsArePropagated() throws Exception { return null; }), Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); assertThrows( "TestException", IllegalStateException.class, @@ -1634,7 +1646,8 @@ private void doStateCalls(BeamFnStateClient beamFnStateClient) { } }), Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); handler.processBundle( BeamFnApi.InstructionRequest.newBuilder() .setProcessBundle( @@ -1684,7 +1697,8 @@ private void doStateCalls(BeamFnStateClient beamFnStateClient) { } }), Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); assertThrows( "State API calls are unsupported", IllegalStateException.class, @@ -1786,7 +1800,8 @@ public void reset() { executionStateSampler, ImmutableMap.of(DATA_INPUT_URN, startFinishGuard), Caches.noop(), - bundleProcessorCache); + bundleProcessorCache, + null /* dataSampler */); AtomicBoolean progressShouldExit = new AtomicBoolean(); Future bundleProcessorTask = @@ -1914,7 +1929,8 @@ public Object createRunnerForPTransform(Context context) throws IOException { } }), Caches.noop(), - new BundleProcessorCache()); + new BundleProcessorCache(), + null /* dataSampler */); assertThrows( "Timers are unsupported", IllegalStateException.class, diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerModifierTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerModifierTest.java new file mode 100644 index 000000000000..d2e9699ba337 --- /dev/null +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerModifierTest.java @@ -0,0 +1,91 @@ +package org.apache.beam.fn.harness.debug; + +import org.apache.beam.fn.harness.PTransformRunnerFactoryTestContext; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.runners.core.construction.CoderTranslation; +import org.apache.beam.runners.core.construction.RehydratedComponents; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.ByteArrayOutputStream; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; + +@RunWith(JUnit4.class) +public class DataSamplerModifierTest { + + @Test + public void testCreatingAndProcessingWithSampling() throws Exception { + // Create the DataSampling PTransform. + String pTransformId = "pTransformId"; + + RunnerApi.FunctionSpec functionSpec = + RunnerApi.FunctionSpec.newBuilder() + .setUrn(DataSamplingFnRunner.URN) + .build(); + RunnerApi.PTransform pTransform = + RunnerApi.PTransform.newBuilder() + .setSpec(functionSpec) + .putInputs("input", "inputTarget") + .build(); + + // Populate fake input PCollections. + Map pCollectionMap = new HashMap<>(); + pCollectionMap.put( + "inputTarget", + RunnerApi.PCollection.newBuilder() + .setUniqueName("inputTarget") + .setCoderId("coder-id") + .build()); + + // Populate the PTransform context that includes the DataSampler. + DataSampler dataSampler = new DataSampler(); + RunnerApi.Coder coder = CoderTranslation.toProto(StringUtf8Coder.of()).getCoder(); + PTransformRunnerFactoryTestContext context = + PTransformRunnerFactoryTestContext.builder(pTransformId, pTransform) + .processBundleInstructionId("instruction-id") + .pCollections(pCollectionMap) + .coders(Collections.singletonMap("coder-id", coder)) + .dataSampler(dataSampler) + .build(); + + // Create the runner which samples the input PCollection. + new DataSamplingFnRunner.Factory<>().createRunnerForPTransform(context); + assertThat( + context.getPCollectionConsumers().keySet(), + contains("inputTarget")); + + // Send in a test value that should be sampled. + context.getPCollectionConsumer("inputTarget").accept(valueInGlobalWindow("Hello, World!")); + + // Rehydrate the given utf-8 string coder. + RehydratedComponents rehydratedComponents = + RehydratedComponents.forComponents( + RunnerApi.Components.newBuilder() + .putAllCoders(context.getCoders()) + .putAllPcollections(context.getPCollections()) + .putAllWindowingStrategies(context.getWindowingStrategies()) + .build()) + .withPipeline(Pipeline.create()); + Coder rehydratedCoder = (Coder)rehydratedComponents.getCoder("coder-id"); + + Map> samples = dataSampler.samples(); + assertThat(samples.keySet(), contains("inputTarget")); + + // Ensure that the value was sampled. + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + rehydratedCoder.encode("Hello, World!", outputStream); + byte[] encodedValue = outputStream.toByteArray(); + assertThat(samples.get("inputTarget"), contains(encodedValue)); + } +} diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java new file mode 100644 index 000000000000..e5ee4cc3fb76 --- /dev/null +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -0,0 +1,93 @@ +package org.apache.beam.fn.harness.debug; + +import org.apache.beam.fn.harness.PTransformRunnerFactoryTestContext; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.runners.core.construction.CoderTranslation; +import org.apache.beam.runners.core.construction.RehydratedComponents; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.ByteArrayOutputStream; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; + +@RunWith(JUnit4.class) +public class DataSamplerTest { + + @Test + public void testCreatingAndProcessingWithSampling() throws Exception { + + + // Create the DataSampling PTransform. + String pTransformId = "pTransformId"; + + RunnerApi.FunctionSpec functionSpec = + RunnerApi.FunctionSpec.newBuilder() + .setUrn(DataSamplingFnRunner.URN) + .build(); + RunnerApi.PTransform pTransform = + RunnerApi.PTransform.newBuilder() + .setSpec(functionSpec) + .putInputs("input", "inputTarget") + .build(); + + // Populate fake input PCollections. + Map pCollectionMap = new HashMap<>(); + pCollectionMap.put( + "inputTarget", + RunnerApi.PCollection.newBuilder() + .setUniqueName("inputTarget") + .setCoderId("coder-id") + .build()); + + // Populate the PTransform context that includes the DataSampler. + DataSampler dataSampler = new DataSampler(); + RunnerApi.Coder coder = CoderTranslation.toProto(StringUtf8Coder.of()).getCoder(); + PTransformRunnerFactoryTestContext context = + PTransformRunnerFactoryTestContext.builder(pTransformId, pTransform) + .processBundleInstructionId("instruction-id") + .pCollections(pCollectionMap) + .coders(Collections.singletonMap("coder-id", coder)) + .dataSampler(dataSampler) + .build(); + + // Create the runner which samples the input PCollection. + new DataSamplingFnRunner.Factory<>().createRunnerForPTransform(context); + assertThat( + context.getPCollectionConsumers().keySet(), + contains("inputTarget")); + + // Send in a test value that should be sampled. + context.getPCollectionConsumer("inputTarget").accept(valueInGlobalWindow("Hello, World!")); + + // Rehydrate the given utf-8 string coder. + RehydratedComponents rehydratedComponents = + RehydratedComponents.forComponents( + RunnerApi.Components.newBuilder() + .putAllCoders(context.getCoders()) + .putAllPcollections(context.getPCollections()) + .putAllWindowingStrategies(context.getWindowingStrategies()) + .build()) + .withPipeline(Pipeline.create()); + Coder rehydratedCoder = (Coder)rehydratedComponents.getCoder("coder-id"); + + Map> samples = dataSampler.samples(); + assertThat(samples.keySet(), contains("inputTarget")); + + // Ensure that the value was sampled. + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + rehydratedCoder.encode("Hello, World!", outputStream); + byte[] encodedValue = outputStream.toByteArray(); + assertThat(samples.get("inputTarget"), contains(encodedValue)); + } +} diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java new file mode 100644 index 000000000000..3da59da7cddf --- /dev/null +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java @@ -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. + */ +package org.apache.beam.fn.harness.debug; + +import org.apache.beam.fn.harness.FlattenRunner; +import org.apache.beam.fn.harness.PTransformRunnerFactoryTestContext; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; +import org.apache.beam.runners.core.construction.CoderTranslation; +import org.apache.beam.runners.core.construction.PTransformTranslation; +import org.apache.beam.runners.core.construction.RehydratedComponents; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.fn.data.FnDataReceiver; +import org.apache.beam.sdk.util.WindowedValue; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.ByteArrayOutputStream; +import java.util.*; + +import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.*; + +@RunWith(JUnit4.class) +public class DataSamplingFnRunnerTest { + + @Test + public void testCreatingAndProcessingWithSampling() throws Exception { + // Create the DataSampling PTransform. + String pTransformId = "pTransformId"; + + RunnerApi.FunctionSpec functionSpec = + RunnerApi.FunctionSpec.newBuilder() + .setUrn(DataSamplingFnRunner.URN) + .build(); + RunnerApi.PTransform pTransform = + RunnerApi.PTransform.newBuilder() + .setSpec(functionSpec) + .putInputs("input", "inputTarget") + .build(); + + // Populate fake input PCollections. + Map pCollectionMap = new HashMap<>(); + pCollectionMap.put( + "inputTarget", + RunnerApi.PCollection.newBuilder() + .setUniqueName("inputTarget") + .setCoderId("coder-id") + .build()); + + // Populate the PTransform context that includes the DataSampler. + DataSampler dataSampler = new DataSampler(); + RunnerApi.Coder coder = CoderTranslation.toProto(StringUtf8Coder.of()).getCoder(); + PTransformRunnerFactoryTestContext context = + PTransformRunnerFactoryTestContext.builder(pTransformId, pTransform) + .processBundleInstructionId("instruction-id") + .pCollections(pCollectionMap) + .coders(Collections.singletonMap("coder-id", coder)) + .dataSampler(dataSampler) + .build(); + + // Create the runner which samples the input PCollection. + new DataSamplingFnRunner.Factory<>().createRunnerForPTransform(context); + assertThat( + context.getPCollectionConsumers().keySet(), + contains("inputTarget")); + + // Send in a test value that should be sampled. + context.getPCollectionConsumer("inputTarget").accept(valueInGlobalWindow("Hello, World!")); + + // Rehydrate the given utf-8 string coder. + RehydratedComponents rehydratedComponents = + RehydratedComponents.forComponents( + RunnerApi.Components.newBuilder() + .putAllCoders(context.getCoders()) + .putAllPcollections(context.getPCollections()) + .putAllWindowingStrategies(context.getWindowingStrategies()) + .build()) + .withPipeline(Pipeline.create()); + Coder rehydratedCoder = (Coder)rehydratedComponents.getCoder("coder-id"); + + Map> samples = dataSampler.samples(); + assertThat(samples.keySet(), contains("inputTarget")); + + // Ensure that the value was sampled. + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + rehydratedCoder.encode("Hello, World!", outputStream); + byte[] encodedValue = outputStream.toByteArray(); + assertThat(samples.get("inputTarget"), contains(encodedValue)); + } +} From e530d14e1929fe1cabf9219f0c101508f5a2c58f Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Wed, 25 Jan 2023 18:53:19 -0800 Subject: [PATCH 02/42] comments --- .../src/main/java/org/apache/beam/fn/harness/FnHarness.java | 3 +-- .../java/org/apache/beam/fn/harness/debug/DataSampler.java | 5 +++++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index a85d878ea5dc..e0d5b007438d 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -93,9 +93,8 @@ public class FnHarness { private static final String STATUS_API_SERVICE_DESCRIPTOR = "STATUS_API_SERVICE_DESCRIPTOR"; private static final String PIPELINE_OPTIONS = "PIPELINE_OPTIONS"; private static final String RUNNER_CAPABILITIES = "RUNNER_CAPABILITIES"; - private static final Logger LOG = LoggerFactory.getLogger(FnHarness.class); - private static final String ENABLE_DATA_SAMPLING_EXPERIMENT = "enable_data_sampling"; + private static final Logger LOG = LoggerFactory.getLogger(FnHarness.class); private static final DataSampler dataSampler = new DataSampler(); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index e408cb47d2c1..bcf2e4e966d3 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -37,11 +37,16 @@ public static class OutputSampler { private final List buffer = new ArrayList<>(); private static final Logger LOG = LoggerFactory.getLogger(OutputSampler.class); + // Maximum number of elements in buffer. private final int maxElements = 10; + // Sampling rate. private final int sampleEveryN = 1000; + + // Total number of samples taken. private long numSamples = 0; + // Index into the buffer of where to overwrite samples. private int resampleIndex = 0; public OutputSampler(Coder coder) { From a09f5d6d8093810336fa138f84ed5106edded832 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 31 Jan 2023 09:09:01 -0800 Subject: [PATCH 03/42] add PBD id to context --- .../fn/harness/PTransformRunnerFactory.java | 3 + .../harness/control/ProcessBundleHandler.java | 5 ++ .../beam/fn/harness/debug/DataSampler.java | 45 ++++++++++--- .../fn/harness/debug/DataSamplerTest.java | 67 ++----------------- 4 files changed, 47 insertions(+), 73 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java index 418a08bc302f..4ddfcdfae4a1 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java @@ -61,6 +61,9 @@ interface Context { /** A client for handling state requests. */ BeamFnStateClient getBeamFnStateClient(); + /** The id of the parent ProcessBundleDescriptor. */ + String getProcessBundleDescriptorId(); + /** The id of the PTransform. */ String getPTransformId(); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index 6992dfffbb79..14ecc1c9d150 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -319,6 +319,11 @@ public BeamFnStateClient getBeamFnStateClient() { return beamFnStateClient; } + @Override + public String getProcessBundleDescriptorId() { + return processBundleDescriptor.getId(); + } + @Override public String getPTransformId() { return pTransformId; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index bcf2e4e966d3..acd6210f4e10 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -28,8 +28,10 @@ import java.io.OutputStream; import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; public class DataSampler { public static class OutputSampler { @@ -92,24 +94,45 @@ private void clear() { } } - private final Map> outputSamplers = new HashMap<>(); + private final Map>> outputSamplers = new HashMap<>(); - public OutputSampler sampleOutput(String outputName, Coder coder) { - if (outputSamplers.containsKey(outputName)) { - return (OutputSampler)outputSamplers.get(outputName); - } + public OutputSampler sampleOutput(String processBundleDescriptorId, String pcollectionId, Coder coder) { + outputSamplers.putIfAbsent(processBundleDescriptorId, new HashMap<>()); + Map> samplers = outputSamplers.get(processBundleDescriptorId); + samplers.putIfAbsent(pcollectionId, new OutputSampler(coder)); - OutputSampler sampler = new OutputSampler(coder); - outputSamplers.put(outputName, sampler); - return sampler; + return (OutputSampler)samplers.get(pcollectionId); } public Map> samples() { + return samplesFor(new HashSet<>(), new HashSet<>()); + } + + public Map> samplesFor(Set descriptors, Set pcollections) { Map> samples = new HashMap<>(); - outputSamplers.forEach((pcollectionId, outputSampler) -> { - samples.putIfAbsent(pcollectionId, new ArrayList<>()); - samples.get(pcollectionId).addAll(outputSampler.samples()); + outputSamplers.forEach((descriptorId, samplers) -> { + if (!descriptors.isEmpty() && !descriptors.contains(descriptorId)) { + return; + } + + samplers.forEach((pcollectionId, outputSampler) -> { + if (!pcollections.isEmpty() && !pcollections.contains(pcollectionId)) { + return; + } + + samples.putIfAbsent(pcollectionId, new ArrayList<>()); + samples.get(pcollectionId).addAll(outputSampler.samples()); + }); }); + return samples; } + + public Map> samplesForDescriptors(Set descriptors) { + return samplesFor(descriptors, new HashSet<>()); + } + + public Map> samplesForPCollections(Set pcollections) { + return samplesFor(new HashSet<>(), pcollections); + } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index e5ee4cc3fb76..82a96dd28af9 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -7,6 +7,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarIntCoder; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -23,71 +24,13 @@ @RunWith(JUnit4.class) public class DataSamplerTest { - @Test public void testCreatingAndProcessingWithSampling() throws Exception { + DataSampler sampler = new DataSampler(); + VarIntCoder coder = VarIntCoder.of(); + sampler.sampleOutput("descriptor-id", "pcollection-id", coder).sample(12345); - // Create the DataSampling PTransform. - String pTransformId = "pTransformId"; - - RunnerApi.FunctionSpec functionSpec = - RunnerApi.FunctionSpec.newBuilder() - .setUrn(DataSamplingFnRunner.URN) - .build(); - RunnerApi.PTransform pTransform = - RunnerApi.PTransform.newBuilder() - .setSpec(functionSpec) - .putInputs("input", "inputTarget") - .build(); - - // Populate fake input PCollections. - Map pCollectionMap = new HashMap<>(); - pCollectionMap.put( - "inputTarget", - RunnerApi.PCollection.newBuilder() - .setUniqueName("inputTarget") - .setCoderId("coder-id") - .build()); - - // Populate the PTransform context that includes the DataSampler. - DataSampler dataSampler = new DataSampler(); - RunnerApi.Coder coder = CoderTranslation.toProto(StringUtf8Coder.of()).getCoder(); - PTransformRunnerFactoryTestContext context = - PTransformRunnerFactoryTestContext.builder(pTransformId, pTransform) - .processBundleInstructionId("instruction-id") - .pCollections(pCollectionMap) - .coders(Collections.singletonMap("coder-id", coder)) - .dataSampler(dataSampler) - .build(); - - // Create the runner which samples the input PCollection. - new DataSamplingFnRunner.Factory<>().createRunnerForPTransform(context); - assertThat( - context.getPCollectionConsumers().keySet(), - contains("inputTarget")); - - // Send in a test value that should be sampled. - context.getPCollectionConsumer("inputTarget").accept(valueInGlobalWindow("Hello, World!")); - - // Rehydrate the given utf-8 string coder. - RehydratedComponents rehydratedComponents = - RehydratedComponents.forComponents( - RunnerApi.Components.newBuilder() - .putAllCoders(context.getCoders()) - .putAllPcollections(context.getPCollections()) - .putAllWindowingStrategies(context.getWindowingStrategies()) - .build()) - .withPipeline(Pipeline.create()); - Coder rehydratedCoder = (Coder)rehydratedComponents.getCoder("coder-id"); - - Map> samples = dataSampler.samples(); - assertThat(samples.keySet(), contains("inputTarget")); - - // Ensure that the value was sampled. - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - rehydratedCoder.encode("Hello, World!", outputStream); - byte[] encodedValue = outputStream.toByteArray(); - assertThat(samples.get("inputTarget"), contains(encodedValue)); + System.out.println(sampler.samples()); } } From 1cb08f9be0816d11882b5e30856e4d5394133fc6 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 31 Jan 2023 09:13:50 -0800 Subject: [PATCH 04/42] merge --- .../src/main/java/org/apache/beam/fn/harness/FnHarness.java | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index e0d5b007438d..e649d21682eb 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -95,7 +95,6 @@ public class FnHarness { private static final String RUNNER_CAPABILITIES = "RUNNER_CAPABILITIES"; private static final String ENABLE_DATA_SAMPLING_EXPERIMENT = "enable_data_sampling"; private static final Logger LOG = LoggerFactory.getLogger(FnHarness.class); - private static final DataSampler dataSampler = new DataSampler(); private static Endpoints.ApiServiceDescriptor getApiServiceDescriptor(String descriptor) From 566c7b85d51f303efb4498923371134f01f981a6 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 31 Jan 2023 15:37:56 -0800 Subject: [PATCH 05/42] Add more tests and spotless --- .../ProcessBundleDescriptorModifier.java | 4 + .../harness/control/ProcessBundleHandler.java | 4 +- .../beam/fn/harness/debug/DataSampler.java | 126 ++++--------- .../debug/DataSamplingDescriptorModifier.java | 30 ++- .../harness/debug/DataSamplingFnRunner.java | 82 +++++--- .../beam/fn/harness/debug/OutputSampler.java | 93 +++++++++ .../PTransformRunnerFactoryTestContext.java | 2 + .../debug/DataSamplerModifierTest.java | 91 --------- .../fn/harness/debug/DataSamplerTest.java | 177 +++++++++++++++--- .../DataSamplingDescriptorModifierTest.java | 72 +++++++ .../debug/DataSamplingFnRunnerTest.java | 71 +++---- .../fn/harness/debug/OutputSamplerTest.java | 85 +++++++++ 12 files changed, 570 insertions(+), 267 deletions(-) create mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java delete mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerModifierTest.java create mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java create mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java index 893109ad7335..9044abd837f9 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java @@ -26,6 +26,10 @@ public GraphModificationException() {} public GraphModificationException(String message) { super(message); } + + public GraphModificationException(String message, Throwable throwable) { + super(throwable); + } } /** diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index 14ecc1c9d150..0e30f715e4ad 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -97,8 +97,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static java.util.Optional.ofNullable; - /** * Processes {@link BeamFnApi.ProcessBundleRequest}s and {@link * BeamFnApi.ProcessBundleSplitRequest}s. @@ -488,7 +486,7 @@ public BundleFinalizer getBundleFinalizer() { @Override public Optional getDataSampler() { - return Optional.ofNullable(dataSampler); + return Optional.ofNullable(dataSampler); } }); if (runner instanceof BeamFnDataReadRunner) { diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index acd6210f4e10..77af639989a1 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -17,122 +17,74 @@ */ package org.apache.beam.fn.harness.debug; -import org.apache.beam.fn.harness.FnHarness; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.OutputStream; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.beam.sdk.coders.Coder; public class DataSampler { - public static class OutputSampler { - private final Coder coder; - private final List buffer = new ArrayList<>(); - private static final Logger LOG = LoggerFactory.getLogger(OutputSampler.class); - - // Maximum number of elements in buffer. - private final int maxElements = 10; - - // Sampling rate. - private final int sampleEveryN = 1000; - - // Total number of samples taken. - private long numSamples = 0; - - // Index into the buffer of where to overwrite samples. - private int resampleIndex = 0; - - public OutputSampler(Coder coder) { - this.coder = coder; - } - - public void sample(T element) { - // Only sample the first 10 elements then after every `sampleEveryN`th element. - numSamples += 1; - if (numSamples > 10 && numSamples % sampleEveryN != 0) { - return; - } - - // Fill buffer until maxElements. - if (buffer.size() < maxElements) { - buffer.add(element); - } else { - // Then rewrite sampled elements as a circular buffer. - buffer.set(resampleIndex, element); - resampleIndex = (resampleIndex + 1) % maxElements; - } - } - - public List samples() { - List ret = new ArrayList<>(); - ByteArrayOutputStream stream = new ByteArrayOutputStream(); - for (T el : buffer) { - try { - coder.encode(el, stream); - ret.add(stream.toByteArray()); - } catch (Exception exception) { - LOG.warn("Could not encode element \"" + el + "\" to bytes: " + exception); - } - } - - clear(); - return ret; - } - - private void clear() { - buffer.clear(); - resampleIndex = 0; - } + + public DataSampler() {} + + public DataSampler(int maxSamples, int sampleEveryN) { + this.maxSamples = maxSamples; + this.sampleEveryN = sampleEveryN; } + public static Set EMPTY = new HashSet<>(); + + // Maximum number of elements in buffer. + private int maxSamples = 10; + + // Sampling rate. + private int sampleEveryN = 1000; + private final Map>> outputSamplers = new HashMap<>(); - public OutputSampler sampleOutput(String processBundleDescriptorId, String pcollectionId, Coder coder) { + public OutputSampler sampleOutput( + String processBundleDescriptorId, String pcollectionId, Coder coder) { outputSamplers.putIfAbsent(processBundleDescriptorId, new HashMap<>()); Map> samplers = outputSamplers.get(processBundleDescriptorId); - samplers.putIfAbsent(pcollectionId, new OutputSampler(coder)); + samplers.putIfAbsent( + pcollectionId, new OutputSampler(coder, this.maxSamples, this.sampleEveryN)); - return (OutputSampler)samplers.get(pcollectionId); + return (OutputSampler) samplers.get(pcollectionId); } public Map> samples() { - return samplesFor(new HashSet<>(), new HashSet<>()); + return samplesFor(EMPTY, EMPTY); } public Map> samplesFor(Set descriptors, Set pcollections) { Map> samples = new HashMap<>(); - outputSamplers.forEach((descriptorId, samplers) -> { - if (!descriptors.isEmpty() && !descriptors.contains(descriptorId)) { - return; - } - - samplers.forEach((pcollectionId, outputSampler) -> { - if (!pcollections.isEmpty() && !pcollections.contains(pcollectionId)) { - return; - } - - samples.putIfAbsent(pcollectionId, new ArrayList<>()); - samples.get(pcollectionId).addAll(outputSampler.samples()); - }); - }); + outputSamplers.forEach( + (descriptorId, samplers) -> { + if (!descriptors.isEmpty() && !descriptors.contains(descriptorId)) { + return; + } + + samplers.forEach( + (pcollectionId, outputSampler) -> { + if (!pcollections.isEmpty() && !pcollections.contains(pcollectionId)) { + return; + } + + samples.putIfAbsent(pcollectionId, new ArrayList<>()); + samples.get(pcollectionId).addAll(outputSampler.samples()); + }); + }); return samples; } public Map> samplesForDescriptors(Set descriptors) { - return samplesFor(descriptors, new HashSet<>()); + return samplesFor(descriptors, EMPTY); } public Map> samplesForPCollections(Set pcollections) { - return samplesFor(new HashSet<>(), pcollections); + return samplesFor(EMPTY, pcollections); } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java index 5f5996a78d82..cc0293699c59 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java @@ -19,11 +19,39 @@ import org.apache.beam.fn.harness.ProcessBundleDescriptorModifier; import org.apache.beam.model.fnexecution.v1.BeamFnApi; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; public class DataSamplingDescriptorModifier implements ProcessBundleDescriptorModifier { @Override public BeamFnApi.ProcessBundleDescriptor ModifyProcessBundleDescriptor( BeamFnApi.ProcessBundleDescriptor pbd) throws GraphModificationException { - return pbd.toBuilder().build(); + BeamFnApi.ProcessBundleDescriptor.Builder builder = pbd.toBuilder(); + for (String pcollectionId : pbd.getPcollectionsMap().keySet()) { + RunnerApi.PCollection pcollection = pbd.getPcollectionsMap().get(pcollectionId); + String coderId = pcollection.getCoderId(); + String transformId = "synthetic-data-sampling-transform-" + pcollectionId; + try { + builder.putTransforms( + transformId, + RunnerApi.PTransform.newBuilder() + .setUniqueName(transformId) + .setSpec( + RunnerApi.FunctionSpec.newBuilder() + .setUrn(DataSamplingFnRunner.URN) + .setPayload( + ByteString.copyFrom( + DataSamplingFnRunner.Payload.encode(pcollectionId, coderId)))) + .putInputs("main", pcollectionId) + .build()); + } catch (Exception exception) { + throw new GraphModificationException( + "Failed to modify graph: could not encode payload for synthetic data " + + "sampling operation.", + exception); + } + } + + return builder.build(); } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java index 9b6aeac1de46..06a4274b5f0a 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java @@ -17,35 +17,69 @@ */ package org.apache.beam.fn.harness.debug; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables.getOnlyElement; - import com.google.auto.service.AutoService; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Map; import java.util.Optional; - -import org.apache.beam.fn.harness.FlattenRunner; import org.apache.beam.fn.harness.PTransformRunnerFactory; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.core.construction.RehydratedComponents; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; -import org.apache.beam.sdk.function.ThrowingFunction; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @SuppressWarnings({ - "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) + "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) public class DataSamplingFnRunner { public static final String URN = "beam:internal:sampling:v1"; private static final Logger LOG = LoggerFactory.getLogger(DataSamplingFnRunner.class); + public static class Payload { + public String pcollectionId; + public String coderId; + + public static Coder> coder = + KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); + + public static Payload of(String pcollectionId, String coderId) { + return new Payload(pcollectionId, coderId); + } + + private Payload(String pcollectionId, String coderId) { + this.pcollectionId = pcollectionId; + this.coderId = coderId; + } + + public static byte[] encode(String pcollectionId, String coderId) throws IOException { + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + coder.encode(KV.of(pcollectionId, coderId), outputStream); + return outputStream.toByteArray(); + } + + public static Payload decode(byte[] payload) throws IOException { + ByteArrayInputStream inputStream = new ByteArrayInputStream(payload); + KV decodedPayload = coder.decode(inputStream); + + if (decodedPayload == null) { + throw new IOException("Could not decode payload for DataSamplingFnRunner."); + } + + return new Payload(decodedPayload.getKey(), decodedPayload.getValue()); + } + } + @AutoService(PTransformRunnerFactory.Registrar.class) public static class Registrar implements PTransformRunnerFactory.Registrar { @@ -55,17 +89,16 @@ public Map getPTransformRunnerFactories() { } } - public static class Factory - implements PTransformRunnerFactory { + public static class Factory implements PTransformRunnerFactory { @Override - public DataSamplingFnRunner createRunnerForPTransform(Context context) - throws IOException { + public DataSamplingFnRunner createRunnerForPTransform(Context context) throws IOException { DataSamplingFnRunner runner = new DataSamplingFnRunner(); Optional maybeDataSampler = context.getDataSampler(); if (!maybeDataSampler.isPresent()) { - LOG.warn("Trying to sample output but DataSampler is not present. Is " + - "\"enable_data_sampling\" set?"); + LOG.warn( + "Trying to sample output but DataSampler is not present. Is " + + "\"enable_data_sampling\" set?"); return runner; } @@ -74,13 +107,14 @@ public DataSamplingFnRunner createRunnerForPTransform(Context context) String inputPCollectionId = Iterables.getOnlyElement(context.getPTransform().getInputsMap().values()); - RunnerApi.PCollection inputPCollection = context - .getPCollections() - .get(inputPCollectionId); + RunnerApi.PCollection inputPCollection = context.getPCollections().get(inputPCollectionId); if (inputPCollection == null) { - LOG.warn("Expected input PCollection \"" + inputPCollectionId + "\" does not exist in " + - "PCollections map."); + LOG.warn( + "Expected input PCollection \"" + + inputPCollectionId + + "\" does not exist in " + + "PCollections map."); return runner; } @@ -94,16 +128,18 @@ public DataSamplingFnRunner createRunnerForPTransform(Context context) .putAllWindowingStrategies(context.getWindowingStrategies()) .build()) .withPipeline(Pipeline.create()); - Coder inputCoder = (Coder)rehydratedComponents.getCoder(inputCoderId); + Coder inputCoder = (Coder) rehydratedComponents.getCoder(inputCoderId); - DataSampler.OutputSampler outputSampler = dataSampler.sampleOutput( - inputPCollectionId, - inputCoder); + OutputSampler outputSampler = + dataSampler.sampleOutput( + context.getProcessBundleDescriptorId(), inputPCollectionId, inputCoder); String pCollectionId = Iterables.getOnlyElement(context.getPTransform().getInputsMap().values()); - context.addPCollectionConsumer(pCollectionId, - (FnDataReceiver>) input -> outputSampler.sample(input.getValue())); + context.addPCollectionConsumer( + pCollectionId, + (FnDataReceiver>) + input -> outputSampler.sample(input.getValue())); return runner; } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java new file mode 100644 index 000000000000..8ce036ab28e3 --- /dev/null +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -0,0 +1,93 @@ +/* + * 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.fn.harness.debug; + +import java.io.ByteArrayOutputStream; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class OutputSampler { + private final Coder coder; + private final List buffer = new ArrayList<>(); + private static final Logger LOG = LoggerFactory.getLogger(OutputSampler.class); + + // Maximum number of elements in buffer. + private int maxElements = 10; + + // Sampling rate. + private int sampleEveryN = 1000; + + // Total number of samples taken. + private long numSamples = 0; + + // Index into the buffer of where to overwrite samples. + private int resampleIndex = 0; + + public OutputSampler(Coder coder) { + this.coder = coder; + } + + public OutputSampler(Coder coder, int maxElements, int sampleEveryN) { + this(coder); + this.maxElements = maxElements; + this.sampleEveryN = sampleEveryN; + } + + public void sample(T element) { + // Only sample the first 10 elements then after every `sampleEveryN`th element. + numSamples += 1; + if (numSamples > 10 && numSamples % sampleEveryN != 0) { + return; + } + + // Fill buffer until maxElements. + if (buffer.size() < maxElements) { + buffer.add(element); + } else { + // Then rewrite sampled elements as a circular buffer. + buffer.set(resampleIndex, element); + resampleIndex = (resampleIndex + 1) % maxElements; + } + } + + public List samples() { + List ret = new ArrayList<>(); + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + for (T el : buffer) { + try { + coder.encode(el, stream); + ret.add(stream.toByteArray()); + } catch (Exception exception) { + LOG.warn("Could not encode element \"" + el + "\" to bytes: " + exception); + } finally { + stream.reset(); + } + } + + clear(); + return ret; + } + + private void clear() { + buffer.clear(); + resampleIndex = 0; + } +} diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java index f84676df7d2f..db00cb4812fb 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java @@ -155,6 +155,8 @@ public interface Builder { Builder beamFnStateClient(BeamFnStateClient value); + Builder processBundleDescriptorId(String value); + Builder pTransformId(String value); Builder pTransform(RunnerApi.PTransform value); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerModifierTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerModifierTest.java deleted file mode 100644 index d2e9699ba337..000000000000 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerModifierTest.java +++ /dev/null @@ -1,91 +0,0 @@ -package org.apache.beam.fn.harness.debug; - -import org.apache.beam.fn.harness.PTransformRunnerFactoryTestContext; -import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.CoderTranslation; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -import java.io.ByteArrayOutputStream; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; - -@RunWith(JUnit4.class) -public class DataSamplerModifierTest { - - @Test - public void testCreatingAndProcessingWithSampling() throws Exception { - // Create the DataSampling PTransform. - String pTransformId = "pTransformId"; - - RunnerApi.FunctionSpec functionSpec = - RunnerApi.FunctionSpec.newBuilder() - .setUrn(DataSamplingFnRunner.URN) - .build(); - RunnerApi.PTransform pTransform = - RunnerApi.PTransform.newBuilder() - .setSpec(functionSpec) - .putInputs("input", "inputTarget") - .build(); - - // Populate fake input PCollections. - Map pCollectionMap = new HashMap<>(); - pCollectionMap.put( - "inputTarget", - RunnerApi.PCollection.newBuilder() - .setUniqueName("inputTarget") - .setCoderId("coder-id") - .build()); - - // Populate the PTransform context that includes the DataSampler. - DataSampler dataSampler = new DataSampler(); - RunnerApi.Coder coder = CoderTranslation.toProto(StringUtf8Coder.of()).getCoder(); - PTransformRunnerFactoryTestContext context = - PTransformRunnerFactoryTestContext.builder(pTransformId, pTransform) - .processBundleInstructionId("instruction-id") - .pCollections(pCollectionMap) - .coders(Collections.singletonMap("coder-id", coder)) - .dataSampler(dataSampler) - .build(); - - // Create the runner which samples the input PCollection. - new DataSamplingFnRunner.Factory<>().createRunnerForPTransform(context); - assertThat( - context.getPCollectionConsumers().keySet(), - contains("inputTarget")); - - // Send in a test value that should be sampled. - context.getPCollectionConsumer("inputTarget").accept(valueInGlobalWindow("Hello, World!")); - - // Rehydrate the given utf-8 string coder. - RehydratedComponents rehydratedComponents = - RehydratedComponents.forComponents( - RunnerApi.Components.newBuilder() - .putAllCoders(context.getCoders()) - .putAllPcollections(context.getPCollections()) - .putAllWindowingStrategies(context.getWindowingStrategies()) - .build()) - .withPipeline(Pipeline.create()); - Coder rehydratedCoder = (Coder)rehydratedComponents.getCoder("coder-id"); - - Map> samples = dataSampler.samples(); - assertThat(samples.keySet(), contains("inputTarget")); - - // Ensure that the value was sampled. - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - rehydratedCoder.encode("Hello, World!", outputStream); - byte[] encodedValue = outputStream.toByteArray(); - assertThat(samples.get("inputTarget"), contains(encodedValue)); - } -} diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index 82a96dd28af9..39d7c0328ca7 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -1,36 +1,169 @@ +/* + * 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.fn.harness.debug; -import org.apache.beam.fn.harness.PTransformRunnerFactoryTestContext; -import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.CoderTranslation; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.Coder; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.*; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayOutputStream; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; - @RunWith(JUnit4.class) public class DataSamplerTest { - @Test - public void testCreatingAndProcessingWithSampling() throws Exception { - DataSampler sampler = new DataSampler(); + public byte[] encodeInt(Integer i) throws IOException { + VarIntCoder coder = VarIntCoder.of(); + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + coder.encode(i, stream); + return stream.toByteArray(); + } + + public byte[] encodeString(String s) throws IOException { + StringUtf8Coder coder = StringUtf8Coder.of(); + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + coder.encode(s, stream); + return stream.toByteArray(); + } + + @Test + public void testSingleOutput() throws Exception { + DataSampler sampler = new DataSampler(); + + VarIntCoder coder = VarIntCoder.of(); + sampler.sampleOutput("descriptor-id", "pcollection-id", coder).sample(1); + + Map> samples = sampler.samples(); + assertThat(samples.get("pcollection-id"), contains(encodeInt(1))); + } + + @Test + public void testMultipleOutputs() throws Exception { + DataSampler sampler = new DataSampler(); + + VarIntCoder coder = VarIntCoder.of(); + sampler.sampleOutput("descriptor-id", "pcollection-id-1", coder).sample(1); + sampler.sampleOutput("descriptor-id", "pcollection-id-2", coder).sample(2); + + Map> samples = sampler.samples(); + assertThat(samples.get("pcollection-id-1"), contains(encodeInt(1))); + assertThat(samples.get("pcollection-id-2"), contains(encodeInt(2))); + } + + @Test + public void testMultipleDescriptors() throws Exception { + DataSampler sampler = new DataSampler(); + + VarIntCoder coder = VarIntCoder.of(); + sampler.sampleOutput("descriptor-id-1", "pcollection-id", coder).sample(1); + sampler.sampleOutput("descriptor-id-2", "pcollection-id", coder).sample(2); + + Map> samples = sampler.samples(); + assertThat(samples.get("pcollection-id"), contains(encodeInt(1), encodeInt(2))); + } + + @Test + public void testFiltersSingleDescriptorId() throws Exception { + DataSampler sampler = new DataSampler(10, 10); + + StringUtf8Coder coder = StringUtf8Coder.of(); + sampler.sampleOutput("a", "1", coder).sample("a1"); + sampler.sampleOutput("a", "2", coder).sample("a2"); + sampler.sampleOutput("b", "1", coder).sample("b1"); + sampler.sampleOutput("b", "2", coder).sample("b2"); + + Map> samples = + sampler.samplesForDescriptors(new HashSet<>(Collections.singletonList("a"))); + assertThat(samples.get("1"), contains(encodeString("a1"))); + assertThat(samples.get("2"), contains(encodeString("a2"))); + } + + @Test + public void testFiltersMultipleDescriptorId() throws Exception { + DataSampler sampler = new DataSampler(10, 10); + + StringUtf8Coder coder = StringUtf8Coder.of(); + sampler.sampleOutput("a", "1", coder).sample("a1"); + sampler.sampleOutput("a", "2", coder).sample("a2"); + sampler.sampleOutput("b", "1", coder).sample("b1"); + sampler.sampleOutput("b", "2", coder).sample("b2"); + + Map> samples = sampler.samplesForDescriptors(ImmutableSet.of("a", "b")); + assertThat(samples.get("1"), contains(encodeString("a1"), encodeString("b1"))); + assertThat(samples.get("2"), contains(encodeString("a2"), encodeString("b2"))); + } + + @Test + public void testFiltersSinglePCollectionId() throws Exception { + DataSampler sampler = new DataSampler(10, 10); + + StringUtf8Coder coder = StringUtf8Coder.of(); + sampler.sampleOutput("a", "1", coder).sample("a1"); + sampler.sampleOutput("a", "2", coder).sample("a2"); + sampler.sampleOutput("b", "1", coder).sample("b1"); + sampler.sampleOutput("b", "2", coder).sample("b2"); + + Map> samples = + sampler.samplesForPCollections(new HashSet<>(Collections.singletonList("1"))); + assertThat(samples.get("1"), containsInAnyOrder(encodeString("a1"), encodeString("b1"))); + } + + Map> singletonSample(String pcollectionId, byte[] element) { + Map> ret = new HashMap<>(); + List list = new ArrayList<>(); + list.add(element); + ret.put(pcollectionId, list); + return ret; + } + + void generateStringSamples(DataSampler sampler) { + StringUtf8Coder coder = StringUtf8Coder.of(); + sampler.sampleOutput("a", "1", coder).sample("a1"); + sampler.sampleOutput("a", "2", coder).sample("a2"); + sampler.sampleOutput("b", "1", coder).sample("b1"); + sampler.sampleOutput("b", "2", coder).sample("b2"); + } + + @Test + public void testFiltersDescriptorAndPCollectionIds() throws Exception { + List descriptorIds = ImmutableList.of("a", "b"); + List pcollectionIds = ImmutableList.of("1", "2"); - VarIntCoder coder = VarIntCoder.of(); - sampler.sampleOutput("descriptor-id", "pcollection-id", coder).sample(12345); + for (String descriptorId : descriptorIds) { + for (String pcollectionId : pcollectionIds) { + DataSampler sampler = new DataSampler(10, 10); + generateStringSamples(sampler); + Map> actual = + sampler.samplesFor(ImmutableSet.of(descriptorId), ImmutableSet.of(pcollectionId)); - System.out.println(sampler.samples()); + System.out.print("Testing: " + descriptorId + pcollectionId + "..."); + assertThat(actual.size(), equalTo(1)); + assertThat(actual.get(pcollectionId), contains(encodeString(descriptorId + pcollectionId))); + System.out.println("ok"); + } } + } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java new file mode 100644 index 000000000000..a46d8d1a8508 --- /dev/null +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java @@ -0,0 +1,72 @@ +/* + * 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.fn.harness.debug; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasEntry; + +import com.google.common.collect.Iterables; +import org.apache.beam.model.fnexecution.v1.BeamFnApi; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class DataSamplingDescriptorModifierTest { + @Test + public void testSimple() throws Exception { + DataSamplingDescriptorModifier modifier = new DataSamplingDescriptorModifier(); + + final String PCOLLECTION_ID = "pcollection-id"; + final String CODER_ID = "coder-id"; + BeamFnApi.ProcessBundleDescriptor descriptor = + BeamFnApi.ProcessBundleDescriptor.newBuilder() + .putPcollections( + PCOLLECTION_ID, + RunnerApi.PCollection.newBuilder() + .setUniqueName(PCOLLECTION_ID) + .setCoderId(CODER_ID) + .build()) + .putCoders( + CODER_ID, + RunnerApi.Coder.newBuilder() + .setSpec(RunnerApi.FunctionSpec.newBuilder().setUrn(DataSamplingFnRunner.URN)) + .build()) + .build(); + + BeamFnApi.ProcessBundleDescriptor modified = modifier.ModifyProcessBundleDescriptor(descriptor); + assertThat(modified.getTransformsCount(), equalTo(1)); + + RunnerApi.PTransform samplingTransform = + Iterables.getOnlyElement(modified.getTransformsMap().values()); + assertThat( + samplingTransform.getUniqueName(), + equalTo("synthetic-data-sampling-transform-" + PCOLLECTION_ID)); + assertThat(samplingTransform.getSpec().getUrn(), equalTo(DataSamplingFnRunner.URN)); + assertThat(samplingTransform.getInputsMap(), hasEntry("main", PCOLLECTION_ID)); + + ByteString encodedPayload = samplingTransform.getSpec().getPayload(); + DataSamplingFnRunner.Payload payload = + DataSamplingFnRunner.Payload.decode(encodedPayload.toByteArray()); + assertThat(payload.pcollectionId, equalTo(PCOLLECTION_ID)); + assertThat(payload.coderId, equalTo(CODER_ID)); + } +} diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java index 3da59da7cddf..24644a503c17 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java @@ -17,29 +17,24 @@ */ package org.apache.beam.fn.harness.debug; -import org.apache.beam.fn.harness.FlattenRunner; +import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.*; + +import java.io.ByteArrayOutputStream; +import java.util.*; import org.apache.beam.fn.harness.PTransformRunnerFactoryTestContext; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.runners.core.construction.CoderTranslation; -import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.runners.core.construction.RehydratedComponents; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.fn.data.FnDataReceiver; -import org.apache.beam.sdk.util.WindowedValue; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayOutputStream; -import java.util.*; - -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.*; - @RunWith(JUnit4.class) public class DataSamplingFnRunnerTest { @@ -49,54 +44,50 @@ public void testCreatingAndProcessingWithSampling() throws Exception { String pTransformId = "pTransformId"; RunnerApi.FunctionSpec functionSpec = - RunnerApi.FunctionSpec.newBuilder() - .setUrn(DataSamplingFnRunner.URN) - .build(); + RunnerApi.FunctionSpec.newBuilder().setUrn(DataSamplingFnRunner.URN).build(); RunnerApi.PTransform pTransform = - RunnerApi.PTransform.newBuilder() - .setSpec(functionSpec) - .putInputs("input", "inputTarget") - .build(); + RunnerApi.PTransform.newBuilder() + .setSpec(functionSpec) + .putInputs("input", "inputTarget") + .build(); // Populate fake input PCollections. Map pCollectionMap = new HashMap<>(); pCollectionMap.put( - "inputTarget", - RunnerApi.PCollection.newBuilder() - .setUniqueName("inputTarget") - .setCoderId("coder-id") - .build()); + "inputTarget", + RunnerApi.PCollection.newBuilder() + .setUniqueName("inputTarget") + .setCoderId("coder-id") + .build()); // Populate the PTransform context that includes the DataSampler. DataSampler dataSampler = new DataSampler(); RunnerApi.Coder coder = CoderTranslation.toProto(StringUtf8Coder.of()).getCoder(); PTransformRunnerFactoryTestContext context = - PTransformRunnerFactoryTestContext.builder(pTransformId, pTransform) - .processBundleInstructionId("instruction-id") - .pCollections(pCollectionMap) - .coders(Collections.singletonMap("coder-id", coder)) - .dataSampler(dataSampler) - .build(); + PTransformRunnerFactoryTestContext.builder(pTransformId, pTransform) + .processBundleInstructionId("instruction-id") + .pCollections(pCollectionMap) + .coders(Collections.singletonMap("coder-id", coder)) + .dataSampler(dataSampler) + .build(); // Create the runner which samples the input PCollection. new DataSamplingFnRunner.Factory<>().createRunnerForPTransform(context); - assertThat( - context.getPCollectionConsumers().keySet(), - contains("inputTarget")); + assertThat(context.getPCollectionConsumers().keySet(), contains("inputTarget")); // Send in a test value that should be sampled. context.getPCollectionConsumer("inputTarget").accept(valueInGlobalWindow("Hello, World!")); // Rehydrate the given utf-8 string coder. RehydratedComponents rehydratedComponents = - RehydratedComponents.forComponents( - RunnerApi.Components.newBuilder() - .putAllCoders(context.getCoders()) - .putAllPcollections(context.getPCollections()) - .putAllWindowingStrategies(context.getWindowingStrategies()) - .build()) - .withPipeline(Pipeline.create()); - Coder rehydratedCoder = (Coder)rehydratedComponents.getCoder("coder-id"); + RehydratedComponents.forComponents( + RunnerApi.Components.newBuilder() + .putAllCoders(context.getCoders()) + .putAllPcollections(context.getPCollections()) + .putAllWindowingStrategies(context.getWindowingStrategies()) + .build()) + .withPipeline(Pipeline.create()); + Coder rehydratedCoder = (Coder) rehydratedComponents.getCoder("coder-id"); Map> samples = dataSampler.samples(); assertThat(samples.keySet(), contains("inputTarget")); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java new file mode 100644 index 000000000000..cac01a3d3560 --- /dev/null +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -0,0 +1,85 @@ +/* + * 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.fn.harness.debug; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class OutputSamplerTest { + public byte[] encodeInt(Integer i) throws IOException { + VarIntCoder coder = VarIntCoder.of(); + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + coder.encode(i, stream); + return stream.toByteArray(); + } + + @Test + public void testSamplesFirstN() throws Exception { + DataSampler sampler = new DataSampler(10, 10); + + VarIntCoder coder = VarIntCoder.of(); + OutputSampler outputSampler = + sampler.sampleOutput("descriptor-id", "pcollection-id", coder); + + for (int i = 0; i < 15; ++i) { + outputSampler.sample(i); + } + + List expected = new ArrayList<>(); + for (int i = 0; i < 10; ++i) { + expected.add(encodeInt(i)); + } + + Map> samples = sampler.samples(); + assertThat(samples.get("pcollection-id"), containsInAnyOrder(expected.toArray())); + } + + @Test + public void testActsLikeCircularBuffer() throws Exception { + DataSampler sampler = new DataSampler(5, 20); + + VarIntCoder coder = VarIntCoder.of(); + OutputSampler outputSampler = + sampler.sampleOutput("descriptor-id", "pcollection-id", coder); + + for (int i = 0; i < 100; ++i) { + outputSampler.sample(i); + } + + List expected = new ArrayList<>(); + expected.add(encodeInt(19)); + expected.add(encodeInt(39)); + expected.add(encodeInt(59)); + expected.add(encodeInt(79)); + expected.add(encodeInt(99)); + + Map> samples = sampler.samples(); + assertThat(samples.get("pcollection-id"), containsInAnyOrder(expected.toArray())); + } +} From 7395a2e661ddcfd55dc7f3529cfc989739c6501e Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 31 Jan 2023 17:37:18 -0800 Subject: [PATCH 06/42] Finish Java data sampling impl with tests, adding comments --- .../org/apache/beam/fn/harness/FnHarness.java | 9 +- .../beam/fn/harness/debug/DataSampler.java | 101 ++++++++++++++++-- .../fn/harness/debug/DataSamplerTest.java | 94 ++++++++++++++-- .../debug/DataSamplingFnRunnerTest.java | 2 +- .../fn/harness/debug/OutputSamplerTest.java | 4 +- 5 files changed, 185 insertions(+), 25 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index e649d21682eb..53659d98584d 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -21,6 +21,7 @@ import java.util.Collections; import java.util.EnumMap; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.function.Function; @@ -256,9 +257,12 @@ public static void main( // Add any graph modifications. List modifierList = new ArrayList<>(); - List experimentList = options.as(ExperimentalOptions.class).getExperiments(); + Optional> experimentList = + Optional.ofNullable(options.as(ExperimentalOptions.class).getExperiments()); - if (experimentList != null && experimentList.contains(ENABLE_DATA_SAMPLING_EXPERIMENT)) { + // If data sampling is enabled, then modify the graph to add any DataSampling Operations. + if (experimentList.isPresent() + && experimentList.get().contains(ENABLE_DATA_SAMPLING_EXPERIMENT)) { modifierList.add(new DataSamplingDescriptorModifier()); } @@ -357,6 +361,7 @@ private BeamFnApi.ProcessBundleDescriptor loadDescriptor(String id) { handlers.put( InstructionRequest.RequestCase.HARNESS_MONITORING_INFOS, processWideHandler::harnessMonitoringInfos); + handlers.put(InstructionRequest.RequestCase.SAMPLE, dataSampler::handleDataSampleRequest); JvmInitializers.runBeforeProcessing(options); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index 77af639989a1..4de8ceabbc69 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -17,33 +17,65 @@ */ package org.apache.beam.fn.harness.debug; +import avro.shaded.com.google.common.collect.ImmutableSet; import java.util.ArrayList; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.beam.model.fnexecution.v1.BeamFnApi; +import org.apache.beam.model.fnexecution.v1.BeamFnApi.SampleDataResponse.ElementList; +import org.apache.beam.model.fnexecution.v1.BeamFnApi.SampledElement; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; +/** + * The DataSampler is a global (per SDK Harness) object that facilitates taking and returning + * samples to the Runner Harness. The class is thread-safe with respect to executing + * ProcessBundleDescriptors. Meaning, different threads executing different PBDs can sample + * simultaneously, even if computing the same logical PCollection. + */ public class DataSampler { + /** Creates a DataSampler to sample every 10 elements while keeping a maximum of 10 in memory. */ public DataSampler() {} + /** + * @param maxSamples Sets the maximum number of samples held in memory at once. + * @param sampleEveryN Sets how often to sample. + */ public DataSampler(int maxSamples, int sampleEveryN) { this.maxSamples = maxSamples; this.sampleEveryN = sampleEveryN; } - public static Set EMPTY = new HashSet<>(); - // Maximum number of elements in buffer. private int maxSamples = 10; // Sampling rate. private int sampleEveryN = 1000; - private final Map>> outputSamplers = new HashMap<>(); - + // The fully-qualified type is: Map[ProcessBundleDescriptorId, [PCollectionId, OutputSampler]]. + // The DataSampler object lives on the same level of the FnHarness. This means that many threads + // can and will + // access this simultaneously. However, ProcessBundleDescriptors are unique per thread, so only + // synchronization + // is needed on the outermost map. + private final Map>> outputSamplers = + new ConcurrentHashMap<>(); + + /** + * Creates and returns a class to sample the given PCollection in the given + * ProcessBundleDescriptor. Uses the given coder encode samples as bytes when responding to a + * SampleDataRequest. + * + * @param processBundleDescriptorId The PBD to sample from. + * @param pcollectionId The PCollection to take intermittent samples from. + * @param coder The coder associated with the PCollection. Coder may be from a nested context. + * @return the OutputSampler corresponding to the unique PBD and PCollection. + * @param The type of element contained in the PCollection. + */ public OutputSampler sampleOutput( String processBundleDescriptorId, String pcollectionId, Coder coder) { outputSamplers.putIfAbsent(processBundleDescriptorId, new HashMap<>()); @@ -54,12 +86,50 @@ public OutputSampler sampleOutput( return (OutputSampler) samplers.get(pcollectionId); } - public Map> samples() { - return samplesFor(EMPTY, EMPTY); + /** + * Returns all collected samples. Thread-safe. + * + * @param request The instruction request from the FnApi. Filters based on the given + * SampleDataRequest. + * @return Returns all collected samples. + */ + public BeamFnApi.InstructionResponse.Builder handleDataSampleRequest( + BeamFnApi.InstructionRequest request) { + BeamFnApi.SampleDataRequest sampleDataRequest = request.getSample(); + + Map> responseSamples = + samplesFor( + ImmutableSet.copyOf(sampleDataRequest.getProcessBundleDescriptorIdsList()), + ImmutableSet.copyOf(sampleDataRequest.getPcollectionIdsList())); + + BeamFnApi.SampleDataResponse.Builder response = BeamFnApi.SampleDataResponse.newBuilder(); + for (String pcollectionId : responseSamples.keySet()) { + ElementList.Builder elementList = ElementList.newBuilder(); + for (byte[] sample : responseSamples.get(pcollectionId)) { + elementList.addElements( + SampledElement.newBuilder().setElement(ByteString.copyFrom(sample)).build()); + } + response.putElementSamples(pcollectionId, elementList.build()); + } + + return BeamFnApi.InstructionResponse.newBuilder().setSample(response); } + /** + * Returns a map from PCollection to its samples. Samples are filtered on + * ProcessBundleDescriptorIds and PCollections. Thread-safe. + * + * @param descriptors PCollections under each PBD id will be unioned. If empty, allows all + * descriptors. + * @param pcollections Filters all PCollections on this set. If empty, allows all PCollections. + * @return a map from PCollection to its samples. + */ public Map> samplesFor(Set descriptors, Set pcollections) { Map> samples = new HashMap<>(); + + // Safe to iterate as the ConcurrentHashMap will return each element at most once and will not + // throw + // ConcurrentModificationException. outputSamplers.forEach( (descriptorId, samplers) -> { if (!descriptors.isEmpty() && !descriptors.contains(descriptorId)) { @@ -80,11 +150,24 @@ public Map> samplesFor(Set descriptors, Set return samples; } + /** @return samples from all PBDs and all PCollections. */ + public Map> allSamples() { + return samplesFor(ImmutableSet.of(), ImmutableSet.of()); + } + + /** + * @param descriptors PBDs to filter on. + * @return samples only from the given descriptors. + */ public Map> samplesForDescriptors(Set descriptors) { - return samplesFor(descriptors, EMPTY); + return samplesFor(descriptors, ImmutableSet.of()); } + /** + * @param pcollections PCollection ids to filter on. + * @return samples only from the given PCollections. + */ public Map> samplesForPCollections(Set pcollections) { - return samplesFor(EMPTY, pcollections); + return samplesFor(ImmutableSet.of(), pcollections); } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index 39d7c0328ca7..07358fdab7d0 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -27,8 +27,10 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.*; +import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -49,6 +51,11 @@ public byte[] encodeString(String s) throws IOException { return stream.toByteArray(); } + /** + * Smoke test that a samples show in the output map. + * + * @throws Exception + */ @Test public void testSingleOutput() throws Exception { DataSampler sampler = new DataSampler(); @@ -56,10 +63,15 @@ public void testSingleOutput() throws Exception { VarIntCoder coder = VarIntCoder.of(); sampler.sampleOutput("descriptor-id", "pcollection-id", coder).sample(1); - Map> samples = sampler.samples(); + Map> samples = sampler.allSamples(); assertThat(samples.get("pcollection-id"), contains(encodeInt(1))); } + /** + * Test that sampling multiple PCollections under the same descriptor is OK. + * + * @throws Exception + */ @Test public void testMultipleOutputs() throws Exception { DataSampler sampler = new DataSampler(); @@ -68,11 +80,16 @@ public void testMultipleOutputs() throws Exception { sampler.sampleOutput("descriptor-id", "pcollection-id-1", coder).sample(1); sampler.sampleOutput("descriptor-id", "pcollection-id-2", coder).sample(2); - Map> samples = sampler.samples(); + Map> samples = sampler.allSamples(); assertThat(samples.get("pcollection-id-1"), contains(encodeInt(1))); assertThat(samples.get("pcollection-id-2"), contains(encodeInt(2))); } + /** + * Test that the response contains samples from the same PCollection across descriptors. + * + * @throws Exception + */ @Test public void testMultipleDescriptors() throws Exception { DataSampler sampler = new DataSampler(); @@ -81,10 +98,15 @@ public void testMultipleDescriptors() throws Exception { sampler.sampleOutput("descriptor-id-1", "pcollection-id", coder).sample(1); sampler.sampleOutput("descriptor-id-2", "pcollection-id", coder).sample(2); - Map> samples = sampler.samples(); + Map> samples = sampler.allSamples(); assertThat(samples.get("pcollection-id"), contains(encodeInt(1), encodeInt(2))); } + /** + * Test that samples can be filtered based on ProcessBundleDescriptor id. + * + * @throws Exception + */ @Test public void testFiltersSingleDescriptorId() throws Exception { DataSampler sampler = new DataSampler(10, 10); @@ -101,6 +123,11 @@ public void testFiltersSingleDescriptorId() throws Exception { assertThat(samples.get("2"), contains(encodeString("a2"))); } + /** + * Test that samples are unioned based on ProcessBundleDescriptor id. + * + * @throws Exception + */ @Test public void testFiltersMultipleDescriptorId() throws Exception { DataSampler sampler = new DataSampler(10, 10); @@ -116,6 +143,11 @@ public void testFiltersMultipleDescriptorId() throws Exception { assertThat(samples.get("2"), contains(encodeString("a2"), encodeString("b2"))); } + /** + * Test that samples can be filtered based on PCollection id. + * + * @throws Exception + */ @Test public void testFiltersSinglePCollectionId() throws Exception { DataSampler sampler = new DataSampler(10, 10); @@ -131,14 +163,6 @@ public void testFiltersSinglePCollectionId() throws Exception { assertThat(samples.get("1"), containsInAnyOrder(encodeString("a1"), encodeString("b1"))); } - Map> singletonSample(String pcollectionId, byte[] element) { - Map> ret = new HashMap<>(); - List list = new ArrayList<>(); - list.add(element); - ret.put(pcollectionId, list); - return ret; - } - void generateStringSamples(DataSampler sampler) { StringUtf8Coder coder = StringUtf8Coder.of(); sampler.sampleOutput("a", "1", coder).sample("a1"); @@ -147,11 +171,17 @@ void generateStringSamples(DataSampler sampler) { sampler.sampleOutput("b", "2", coder).sample("b2"); } + /** + * Test that samples can be filtered both on PCollection and ProcessBundleDescriptor id. + * + * @throws Exception + */ @Test public void testFiltersDescriptorAndPCollectionIds() throws Exception { List descriptorIds = ImmutableList.of("a", "b"); List pcollectionIds = ImmutableList.of("1", "2"); + // Try all combinations for descriptor and PCollection ids. for (String descriptorId : descriptorIds) { for (String pcollectionId : pcollectionIds) { DataSampler sampler = new DataSampler(10, 10); @@ -166,4 +196,46 @@ public void testFiltersDescriptorAndPCollectionIds() throws Exception { } } } + + /** + * Test that the DataSampler can respond with the correct samples with filters. + * + * @throws Exception + */ + @Test + public void testMakesCorrectResponse() throws Exception { + DataSampler dataSampler = new DataSampler(); + generateStringSamples(dataSampler); + + // SampleDataRequest that filters on PCollection=1 and PBD ids = "a" or "b". + BeamFnApi.InstructionRequest request = + BeamFnApi.InstructionRequest.newBuilder() + .setSample( + BeamFnApi.SampleDataRequest.newBuilder() + .addPcollectionIds("1") + .addProcessBundleDescriptorIds("a") + .addProcessBundleDescriptorIds("b") + .build()) + .build(); + BeamFnApi.InstructionResponse actual = dataSampler.handleDataSampleRequest(request).build(); + BeamFnApi.InstructionResponse expected = + BeamFnApi.InstructionResponse.newBuilder() + .setSample( + BeamFnApi.SampleDataResponse.newBuilder() + .putElementSamples( + "1", + BeamFnApi.SampleDataResponse.ElementList.newBuilder() + .addElements( + BeamFnApi.SampledElement.newBuilder() + .setElement(ByteString.copyFrom(encodeString("a1"))) + .build()) + .addElements( + BeamFnApi.SampledElement.newBuilder() + .setElement(ByteString.copyFrom(encodeString("b1"))) + .build()) + .build()) + .build()) + .build(); + assertThat(actual, equalTo(expected)); + } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java index 24644a503c17..0aea3e6e6017 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java @@ -89,7 +89,7 @@ public void testCreatingAndProcessingWithSampling() throws Exception { .withPipeline(Pipeline.create()); Coder rehydratedCoder = (Coder) rehydratedComponents.getCoder("coder-id"); - Map> samples = dataSampler.samples(); + Map> samples = dataSampler.allSamples(); assertThat(samples.keySet(), contains("inputTarget")); // Ensure that the value was sampled. diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java index cac01a3d3560..f97840e6506a 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -56,7 +56,7 @@ public void testSamplesFirstN() throws Exception { expected.add(encodeInt(i)); } - Map> samples = sampler.samples(); + Map> samples = sampler.allSamples(); assertThat(samples.get("pcollection-id"), containsInAnyOrder(expected.toArray())); } @@ -79,7 +79,7 @@ public void testActsLikeCircularBuffer() throws Exception { expected.add(encodeInt(79)); expected.add(encodeInt(99)); - Map> samples = sampler.samples(); + Map> samples = sampler.allSamples(); assertThat(samples.get("pcollection-id"), containsInAnyOrder(expected.toArray())); } } From 4c1253fbd11dd82f2364817482a88ba0607a64e7 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 31 Jan 2023 18:05:51 -0800 Subject: [PATCH 07/42] more comments, remove Payload --- .../beam/fn/harness/debug/DataSampler.java | 2 +- .../debug/DataSamplingDescriptorModifier.java | 39 ++++++------ .../harness/debug/DataSamplingFnRunner.java | 61 ++++--------------- .../fn/harness/debug/DataSamplerTest.java | 4 +- .../DataSamplingDescriptorModifierTest.java | 9 +-- 5 files changed, 34 insertions(+), 81 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index 4de8ceabbc69..b7fdf06e9d0d 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -17,7 +17,6 @@ */ package org.apache.beam.fn.harness.debug; -import avro.shaded.com.google.common.collect.ImmutableSet; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -29,6 +28,7 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi.SampledElement; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; /** * The DataSampler is a global (per SDK Harness) object that facilitates taking and returning diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java index cc0293699c59..3b5db451ad3c 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java @@ -20,36 +20,33 @@ import org.apache.beam.fn.harness.ProcessBundleDescriptorModifier; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; +/** + * Modifies the given ProcessBundleDescriptor by adding a DataSampling operation as a consumer to every + * PCollection. + */ public class DataSamplingDescriptorModifier implements ProcessBundleDescriptorModifier { @Override public BeamFnApi.ProcessBundleDescriptor ModifyProcessBundleDescriptor( - BeamFnApi.ProcessBundleDescriptor pbd) throws GraphModificationException { + BeamFnApi.ProcessBundleDescriptor pbd) { BeamFnApi.ProcessBundleDescriptor.Builder builder = pbd.toBuilder(); + + // Get all PCollections to modify. for (String pcollectionId : pbd.getPcollectionsMap().keySet()) { RunnerApi.PCollection pcollection = pbd.getPcollectionsMap().get(pcollectionId); String coderId = pcollection.getCoderId(); String transformId = "synthetic-data-sampling-transform-" + pcollectionId; - try { - builder.putTransforms( - transformId, - RunnerApi.PTransform.newBuilder() - .setUniqueName(transformId) - .setSpec( - RunnerApi.FunctionSpec.newBuilder() - .setUrn(DataSamplingFnRunner.URN) - .setPayload( - ByteString.copyFrom( - DataSamplingFnRunner.Payload.encode(pcollectionId, coderId)))) - .putInputs("main", pcollectionId) - .build()); - } catch (Exception exception) { - throw new GraphModificationException( - "Failed to modify graph: could not encode payload for synthetic data " - + "sampling operation.", - exception); - } + + // Create a new DataSampling PTransform that consumes that given PCollection. + builder.putTransforms( + transformId, + RunnerApi.PTransform.newBuilder() + .setUniqueName(transformId) + .setSpec( + RunnerApi.FunctionSpec.newBuilder() + .setUrn(DataSamplingFnRunner.URN)) + .putInputs("main", pcollectionId) + .build()); } return builder.build(); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java index 06a4274b5f0a..f9a42e3154ce 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java @@ -18,8 +18,6 @@ package org.apache.beam.fn.harness.debug; import com.google.auto.service.AutoService; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Map; import java.util.Optional; @@ -28,11 +26,8 @@ import org.apache.beam.runners.core.construction.RehydratedComponents; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; import org.slf4j.Logger; @@ -41,45 +36,15 @@ @SuppressWarnings({ "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) + +/** + * This class is in charge of retrieving and sampling in-flight elements. This uses an internal only URN to identify it. + */ public class DataSamplingFnRunner { public static final String URN = "beam:internal:sampling:v1"; private static final Logger LOG = LoggerFactory.getLogger(DataSamplingFnRunner.class); - public static class Payload { - public String pcollectionId; - public String coderId; - - public static Coder> coder = - KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); - - public static Payload of(String pcollectionId, String coderId) { - return new Payload(pcollectionId, coderId); - } - - private Payload(String pcollectionId, String coderId) { - this.pcollectionId = pcollectionId; - this.coderId = coderId; - } - - public static byte[] encode(String pcollectionId, String coderId) throws IOException { - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - coder.encode(KV.of(pcollectionId, coderId), outputStream); - return outputStream.toByteArray(); - } - - public static Payload decode(byte[] payload) throws IOException { - ByteArrayInputStream inputStream = new ByteArrayInputStream(payload); - KV decodedPayload = coder.decode(inputStream); - - if (decodedPayload == null) { - throw new IOException("Could not decode payload for DataSamplingFnRunner."); - } - - return new Payload(decodedPayload.getKey(), decodedPayload.getValue()); - } - } - @AutoService(PTransformRunnerFactory.Registrar.class) public static class Registrar implements PTransformRunnerFactory.Registrar { @@ -94,32 +59,31 @@ public static class Factory implements PTransformRunnerFactory maybeDataSampler = context.getDataSampler(); if (!maybeDataSampler.isPresent()) { LOG.warn( - "Trying to sample output but DataSampler is not present. Is " - + "\"enable_data_sampling\" set?"); + "Trying to sample output but DataSampler is not present. Is \"enable_data_sampling\" set?"); return runner; } DataSampler dataSampler = maybeDataSampler.get(); + // This should only have one input, the PCollection to sample String inputPCollectionId = Iterables.getOnlyElement(context.getPTransform().getInputsMap().values()); RunnerApi.PCollection inputPCollection = context.getPCollections().get(inputPCollectionId); - if (inputPCollection == null) { LOG.warn( "Expected input PCollection \"" + inputPCollectionId - + "\" does not exist in " - + "PCollections map."); + + "\" does not exist in PCollections map."); return runner; } + // This coder is used to encode sampled elements to sample back to the Runner Harness. String inputCoderId = inputPCollection.getCoderId(); - RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents( RunnerApi.Components.newBuilder() @@ -128,16 +92,15 @@ public DataSamplingFnRunner createRunnerForPTransform(Context context) throws IO .putAllWindowingStrategies(context.getWindowingStrategies()) .build()) .withPipeline(Pipeline.create()); - Coder inputCoder = (Coder) rehydratedComponents.getCoder(inputCoderId); + Coder inputCoder = (Coder) rehydratedComponents.getCoder(inputCoderId); OutputSampler outputSampler = dataSampler.sampleOutput( context.getProcessBundleDescriptorId(), inputPCollectionId, inputCoder); - String pCollectionId = - Iterables.getOnlyElement(context.getPTransform().getInputsMap().values()); + // Add the runner to consume the PCollection as an input. context.addPCollectionConsumer( - pCollectionId, + inputPCollectionId, (FnDataReceiver>) input -> outputSampler.sample(input.getValue())); return runner; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index 07358fdab7d0..de383f24efeb 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -22,8 +22,6 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.*; @@ -31,6 +29,8 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java index a46d8d1a8508..981264a40239 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java @@ -21,10 +21,9 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasEntry; -import com.google.common.collect.Iterables; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -62,11 +61,5 @@ public void testSimple() throws Exception { equalTo("synthetic-data-sampling-transform-" + PCOLLECTION_ID)); assertThat(samplingTransform.getSpec().getUrn(), equalTo(DataSamplingFnRunner.URN)); assertThat(samplingTransform.getInputsMap(), hasEntry("main", PCOLLECTION_ID)); - - ByteString encodedPayload = samplingTransform.getSpec().getPayload(); - DataSamplingFnRunner.Payload payload = - DataSamplingFnRunner.Payload.decode(encodedPayload.toByteArray()); - assertThat(payload.pcollectionId, equalTo(PCOLLECTION_ID)); - assertThat(payload.coderId, equalTo(CODER_ID)); } } From 3d882544707a826fb130880ace5cd66424bd7695 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 31 Jan 2023 18:26:44 -0800 Subject: [PATCH 08/42] more comments --- .../debug/DataSamplingDescriptorModifier.java | 5 +- .../beam/fn/harness/debug/OutputSampler.java | 13 +++++ .../DataSamplingDescriptorModifierTest.java | 52 +++++++++++++------ .../fn/harness/debug/OutputSamplerTest.java | 12 +++++ 4 files changed, 63 insertions(+), 19 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java index 3b5db451ad3c..6ffb0e585b79 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java @@ -33,11 +33,8 @@ public BeamFnApi.ProcessBundleDescriptor ModifyProcessBundleDescriptor( // Get all PCollections to modify. for (String pcollectionId : pbd.getPcollectionsMap().keySet()) { - RunnerApi.PCollection pcollection = pbd.getPcollectionsMap().get(pcollectionId); - String coderId = pcollection.getCoderId(); - String transformId = "synthetic-data-sampling-transform-" + pcollectionId; - // Create a new DataSampling PTransform that consumes that given PCollection. + String transformId = "synthetic-data-sampling-transform-" + pcollectionId; builder.putTransforms( transformId, RunnerApi.PTransform.newBuilder() diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index 8ce036ab28e3..e46f59b94c48 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -24,6 +24,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * This class holds samples for a single PCollection until queried by the parent DataSampler. This class is meant to hold + * only a limited number of elements in memory. So old values are constantly being overridden in a circular buffer. + * @param the element type of the PCollection. + */ public class OutputSampler { private final Coder coder; private final List buffer = new ArrayList<>(); @@ -51,6 +56,10 @@ public OutputSampler(Coder coder, int maxElements, int sampleEveryN) { this.sampleEveryN = sampleEveryN; } + /** + * Samples every 1000th element or if it is part of the first 10 in the (local) PCollection. + * @param element the element to sample. + */ public void sample(T element) { // Only sample the first 10 elements then after every `sampleEveryN`th element. numSamples += 1; @@ -68,6 +77,10 @@ public void sample(T element) { } } + /** + * Clears samples at end of call. This is to help mitigate memory use. + * @return samples taken since last call. + */ public List samples() { List ret = new ArrayList<>(); ByteArrayOutputStream stream = new ByteArrayOutputStream(); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java index 981264a40239..7a4da27f06a9 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java @@ -30,36 +30,58 @@ @RunWith(JUnit4.class) public class DataSamplingDescriptorModifierTest { + + /** + * Tests that given a ProcessBundleDescriptor, the correct graph modification is done to create a DataSampling PTransform. + */ @Test - public void testSimple() throws Exception { + public void testThatDataSamplingTransformIsMade() { DataSamplingDescriptorModifier modifier = new DataSamplingDescriptorModifier(); - final String PCOLLECTION_ID = "pcollection-id"; - final String CODER_ID = "coder-id"; + final String PCOLLECTION_ID_A = "pcollection-id-a"; + final String PCOLLECTION_ID_B = "pcollection-id-b"; + final String CODER_ID_A = "coder-id-a"; + final String CODER_ID_B = "coder-id-b"; BeamFnApi.ProcessBundleDescriptor descriptor = BeamFnApi.ProcessBundleDescriptor.newBuilder() .putPcollections( - PCOLLECTION_ID, + PCOLLECTION_ID_A, RunnerApi.PCollection.newBuilder() - .setUniqueName(PCOLLECTION_ID) - .setCoderId(CODER_ID) + .setUniqueName(PCOLLECTION_ID_A) + .setCoderId(CODER_ID_A) + .build()) + .putPcollections( + PCOLLECTION_ID_B, + RunnerApi.PCollection.newBuilder() + .setUniqueName(PCOLLECTION_ID_B) + .setCoderId(CODER_ID_B) + .build()) + .putCoders( + CODER_ID_A, + RunnerApi.Coder.newBuilder() + .setSpec(RunnerApi.FunctionSpec.newBuilder().setUrn(DataSamplingFnRunner.URN)) .build()) .putCoders( - CODER_ID, + CODER_ID_B, RunnerApi.Coder.newBuilder() .setSpec(RunnerApi.FunctionSpec.newBuilder().setUrn(DataSamplingFnRunner.URN)) .build()) .build(); + final String PTRANSFORM_ID_A = "synthetic-data-sampling-transform-" + PCOLLECTION_ID_A; + final String PTRANSFORM_ID_B = "synthetic-data-sampling-transform-" + PCOLLECTION_ID_B; + BeamFnApi.ProcessBundleDescriptor modified = modifier.ModifyProcessBundleDescriptor(descriptor); - assertThat(modified.getTransformsCount(), equalTo(1)); + assertThat(modified.getTransformsCount(), equalTo(2)); + + RunnerApi.PTransform samplingTransformA = modified.getTransformsMap().get(PTRANSFORM_ID_A); + assertThat(samplingTransformA.getUniqueName(), equalTo(PTRANSFORM_ID_A)); + assertThat(samplingTransformA.getSpec().getUrn(), equalTo(DataSamplingFnRunner.URN)); + assertThat(samplingTransformA.getInputsMap(), hasEntry("main", PCOLLECTION_ID_A)); - RunnerApi.PTransform samplingTransform = - Iterables.getOnlyElement(modified.getTransformsMap().values()); - assertThat( - samplingTransform.getUniqueName(), - equalTo("synthetic-data-sampling-transform-" + PCOLLECTION_ID)); - assertThat(samplingTransform.getSpec().getUrn(), equalTo(DataSamplingFnRunner.URN)); - assertThat(samplingTransform.getInputsMap(), hasEntry("main", PCOLLECTION_ID)); + RunnerApi.PTransform samplingTransformB = modified.getTransformsMap().get(PTRANSFORM_ID_B); + assertThat(samplingTransformB.getUniqueName(), equalTo(PTRANSFORM_ID_B)); + assertThat(samplingTransformB.getSpec().getUrn(), equalTo(DataSamplingFnRunner.URN)); + assertThat(samplingTransformB.getInputsMap(), hasEntry("main", PCOLLECTION_ID_B)); } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java index f97840e6506a..cdb6125d9eb3 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -39,6 +39,10 @@ public byte[] encodeInt(Integer i) throws IOException { return stream.toByteArray(); } + /** + * Test that the first N are always sampled. + * @throws Exception when encoding fails (shouldn't happen). + */ @Test public void testSamplesFirstN() throws Exception { DataSampler sampler = new DataSampler(10, 10); @@ -47,10 +51,12 @@ public void testSamplesFirstN() throws Exception { OutputSampler outputSampler = sampler.sampleOutput("descriptor-id", "pcollection-id", coder); + // Purposely go over maxSamples and sampleEveryN. This helps to increase confidence. for (int i = 0; i < 15; ++i) { outputSampler.sample(i); } + // The expected list is only 0..9 inclusive. List expected = new ArrayList<>(); for (int i = 0; i < 10; ++i) { expected.add(encodeInt(i)); @@ -60,6 +66,10 @@ public void testSamplesFirstN() throws Exception { assertThat(samples.get("pcollection-id"), containsInAnyOrder(expected.toArray())); } + /** + * Test that the previous values are overwritten and only the most recent `maxSamples` are kept. + * @throws Exception when encoding fails (shouldn't happen). + */ @Test public void testActsLikeCircularBuffer() throws Exception { DataSampler sampler = new DataSampler(5, 20); @@ -72,6 +82,8 @@ public void testActsLikeCircularBuffer() throws Exception { outputSampler.sample(i); } + // The first 10 are always sampled, but with maxSamples = 5, the first ten are downsampled to 4..9 inclusive. Then, + // the 20th element is sampled (19) and every 20 after. List expected = new ArrayList<>(); expected.add(encodeInt(19)); expected.add(encodeInt(39)); From 5bbef91f0171724b822babc9d4237c855bca4e7f Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 31 Jan 2023 18:28:39 -0800 Subject: [PATCH 09/42] spotless --- .../fn/harness/debug/DataSamplingDescriptorModifier.java | 8 +++----- .../beam/fn/harness/debug/DataSamplingFnRunner.java | 6 ++++-- .../org/apache/beam/fn/harness/debug/OutputSampler.java | 8 ++++++-- .../harness/debug/DataSamplingDescriptorModifierTest.java | 4 ++-- .../apache/beam/fn/harness/debug/OutputSamplerTest.java | 5 ++++- 5 files changed, 19 insertions(+), 12 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java index 6ffb0e585b79..832ec3e15eb0 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java @@ -22,8 +22,8 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; /** - * Modifies the given ProcessBundleDescriptor by adding a DataSampling operation as a consumer to every - * PCollection. + * Modifies the given ProcessBundleDescriptor by adding a DataSampling operation as a consumer to + * every PCollection. */ public class DataSamplingDescriptorModifier implements ProcessBundleDescriptorModifier { @Override @@ -39,9 +39,7 @@ public BeamFnApi.ProcessBundleDescriptor ModifyProcessBundleDescriptor( transformId, RunnerApi.PTransform.newBuilder() .setUniqueName(transformId) - .setSpec( - RunnerApi.FunctionSpec.newBuilder() - .setUrn(DataSamplingFnRunner.URN)) + .setSpec(RunnerApi.FunctionSpec.newBuilder().setUrn(DataSamplingFnRunner.URN)) .putInputs("main", pcollectionId) .build()); } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java index f9a42e3154ce..5d3e29148b58 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java @@ -38,7 +38,8 @@ }) /** - * This class is in charge of retrieving and sampling in-flight elements. This uses an internal only URN to identify it. + * This class is in charge of retrieving and sampling in-flight elements. This uses an internal only + * URN to identify it. */ public class DataSamplingFnRunner { public static final String URN = "beam:internal:sampling:v1"; @@ -59,7 +60,8 @@ public static class Factory implements PTransformRunnerFactory maybeDataSampler = context.getDataSampler(); if (!maybeDataSampler.isPresent()) { LOG.warn( diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index e46f59b94c48..672c2041db69 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -25,8 +25,10 @@ import org.slf4j.LoggerFactory; /** - * This class holds samples for a single PCollection until queried by the parent DataSampler. This class is meant to hold - * only a limited number of elements in memory. So old values are constantly being overridden in a circular buffer. + * This class holds samples for a single PCollection until queried by the parent DataSampler. This + * class is meant to hold only a limited number of elements in memory. So old values are constantly + * being overridden in a circular buffer. + * * @param the element type of the PCollection. */ public class OutputSampler { @@ -58,6 +60,7 @@ public OutputSampler(Coder coder, int maxElements, int sampleEveryN) { /** * Samples every 1000th element or if it is part of the first 10 in the (local) PCollection. + * * @param element the element to sample. */ public void sample(T element) { @@ -79,6 +82,7 @@ public void sample(T element) { /** * Clears samples at end of call. This is to help mitigate memory use. + * * @return samples taken since last call. */ public List samples() { diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java index 7a4da27f06a9..5aa49eefb69a 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java @@ -23,7 +23,6 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -32,7 +31,8 @@ public class DataSamplingDescriptorModifierTest { /** - * Tests that given a ProcessBundleDescriptor, the correct graph modification is done to create a DataSampling PTransform. + * Tests that given a ProcessBundleDescriptor, the correct graph modification is done to create a + * DataSampling PTransform. */ @Test public void testThatDataSamplingTransformIsMade() { diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java index cdb6125d9eb3..00a52df9039d 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -41,6 +41,7 @@ public byte[] encodeInt(Integer i) throws IOException { /** * Test that the first N are always sampled. + * * @throws Exception when encoding fails (shouldn't happen). */ @Test @@ -68,6 +69,7 @@ public void testSamplesFirstN() throws Exception { /** * Test that the previous values are overwritten and only the most recent `maxSamples` are kept. + * * @throws Exception when encoding fails (shouldn't happen). */ @Test @@ -82,7 +84,8 @@ public void testActsLikeCircularBuffer() throws Exception { outputSampler.sample(i); } - // The first 10 are always sampled, but with maxSamples = 5, the first ten are downsampled to 4..9 inclusive. Then, + // The first 10 are always sampled, but with maxSamples = 5, the first ten are downsampled to + // 4..9 inclusive. Then, // the 20th element is sampled (19) and every 20 after. List expected = new ArrayList<>(); expected.add(encodeInt(19)); From 6993b83d34784d7365b946afd06a78ab7f50d04d Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Fri, 3 Feb 2023 14:21:59 -0800 Subject: [PATCH 10/42] Encode in the nested context --- .../java/org/apache/beam/fn/harness/debug/OutputSampler.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index 672c2041db69..4ec6be3f8b80 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -90,7 +90,10 @@ public List samples() { ByteArrayOutputStream stream = new ByteArrayOutputStream(); for (T el : buffer) { try { - coder.encode(el, stream); + // This is deprecated, but until this is fully removed, this specifically needs the nested + // context. This is because the SDK will need to decode the sampled elements with the + // ToStringFn. + coder.encode(el, stream, Coder.Context.NESTED); ret.add(stream.toByteArray()); } catch (Exception exception) { LOG.warn("Could not encode element \"" + el + "\" to bytes: " + exception); From 769902f7e0efc3a5cbde3824dca8b2af6b9c5d33 Mon Sep 17 00:00:00 2001 From: Sam sam Date: Tue, 7 Feb 2023 15:27:17 -0800 Subject: [PATCH 11/42] Update sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java Co-authored-by: Lukasz Cwik --- .../apache/beam/fn/harness/control/ProcessBundleHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index 0e30f715e4ad..cf21f5e51de6 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -156,7 +156,7 @@ public class ProcessBundleHandler { @VisibleForTesting final BundleProcessorCache bundleProcessorCache; private final Set runnerCapabilities; - private DataSampler dataSampler; + private final DataSampler dataSampler; public ProcessBundleHandler( PipelineOptions options, From 5c06d4ebda7aa236a1ff4eae56cd4bfd36a1712e Mon Sep 17 00:00:00 2001 From: Sam sam Date: Tue, 7 Feb 2023 15:29:37 -0800 Subject: [PATCH 12/42] Apply suggestions from code review Co-authored-by: Lukasz Cwik --- .../java/org/apache/beam/fn/harness/debug/DataSampler.java | 6 +++--- .../org/apache/beam/fn/harness/debug/OutputSampler.java | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index b7fdf06e9d0d..2834a43411ed 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -51,10 +51,10 @@ public DataSampler(int maxSamples, int sampleEveryN) { } // Maximum number of elements in buffer. - private int maxSamples = 10; + private final int maxSamples = 10; // Sampling rate. - private int sampleEveryN = 1000; + private final int sampleEveryN = 1000; // The fully-qualified type is: Map[ProcessBundleDescriptorId, [PCollectionId, OutputSampler]]. // The DataSampler object lives on the same level of the FnHarness. This means that many threads @@ -142,7 +142,7 @@ public Map> samplesFor(Set descriptors, Set return; } - samples.putIfAbsent(pcollectionId, new ArrayList<>()); + samples.putIfAbsent(pcollectionId, Collections.EMPTY_LIST); samples.get(pcollectionId).addAll(outputSampler.samples()); }); }); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index 4ec6be3f8b80..45f4cf095522 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -33,7 +33,7 @@ */ public class OutputSampler { private final Coder coder; - private final List buffer = new ArrayList<>(); + private final List buffer = new ArrayList<>(maxElements); private static final Logger LOG = LoggerFactory.getLogger(OutputSampler.class); // Maximum number of elements in buffer. From 99087e8577220ca8d2fc0518cf3c5f4fa90c85dc Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Wed, 8 Feb 2023 18:22:06 -0800 Subject: [PATCH 13/42] address pr comments --- .../beam/fn/harness/debug/DataSampler.java | 53 ++--- .../beam/fn/harness/debug/OutputSampler.java | 29 ++- .../fn/harness/debug/DataSamplerTest.java | 195 +++++++++--------- .../debug/DataSamplingFnRunnerTest.java | 24 ++- .../fn/harness/debug/OutputSamplerTest.java | 19 +- 5 files changed, 163 insertions(+), 157 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index 2834a43411ed..34321c055500 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -21,14 +21,12 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.fnexecution.v1.BeamFnApi.SampleDataResponse.ElementList; import org.apache.beam.model.fnexecution.v1.BeamFnApi.SampledElement; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; /** * The DataSampler is a global (per SDK Harness) object that facilitates taking and returning @@ -38,8 +36,13 @@ */ public class DataSampler { - /** Creates a DataSampler to sample every 10 elements while keeping a maximum of 10 in memory. */ - public DataSampler() {} + /** + * Creates a DataSampler to sample every 1000 elements while keeping a maximum of 10 in memory. + */ + public DataSampler() { + this.maxSamples = 10; + this.sampleEveryN = 1000; + } /** * @param maxSamples Sets the maximum number of samples held in memory at once. @@ -51,17 +54,15 @@ public DataSampler(int maxSamples, int sampleEveryN) { } // Maximum number of elements in buffer. - private final int maxSamples = 10; + private final int maxSamples; // Sampling rate. - private final int sampleEveryN = 1000; + private final int sampleEveryN; // The fully-qualified type is: Map[ProcessBundleDescriptorId, [PCollectionId, OutputSampler]]. // The DataSampler object lives on the same level of the FnHarness. This means that many threads - // can and will - // access this simultaneously. However, ProcessBundleDescriptors are unique per thread, so only - // synchronization - // is needed on the outermost map. + // can and will access this simultaneously. However, ProcessBundleDescriptors are unique per + // thread, so only synchronization is needed on the outermost map. private final Map>> outputSamplers = new ConcurrentHashMap<>(); @@ -73,8 +74,8 @@ public DataSampler(int maxSamples, int sampleEveryN) { * @param processBundleDescriptorId The PBD to sample from. * @param pcollectionId The PCollection to take intermittent samples from. * @param coder The coder associated with the PCollection. Coder may be from a nested context. - * @return the OutputSampler corresponding to the unique PBD and PCollection. * @param The type of element contained in the PCollection. + * @return the OutputSampler corresponding to the unique PBD and PCollection. */ public OutputSampler sampleOutput( String processBundleDescriptorId, String pcollectionId, Coder coder) { @@ -99,8 +100,8 @@ public BeamFnApi.InstructionResponse.Builder handleDataSampleRequest( Map> responseSamples = samplesFor( - ImmutableSet.copyOf(sampleDataRequest.getProcessBundleDescriptorIdsList()), - ImmutableSet.copyOf(sampleDataRequest.getPcollectionIdsList())); + sampleDataRequest.getProcessBundleDescriptorIdsList(), + sampleDataRequest.getPcollectionIdsList()); BeamFnApi.SampleDataResponse.Builder response = BeamFnApi.SampleDataResponse.newBuilder(); for (String pcollectionId : responseSamples.keySet()) { @@ -124,7 +125,8 @@ public BeamFnApi.InstructionResponse.Builder handleDataSampleRequest( * @param pcollections Filters all PCollections on this set. If empty, allows all PCollections. * @return a map from PCollection to its samples. */ - public Map> samplesFor(Set descriptors, Set pcollections) { + private Map> samplesFor( + List descriptors, List pcollections) { Map> samples = new HashMap<>(); // Safe to iterate as the ConcurrentHashMap will return each element at most once and will not @@ -142,32 +144,11 @@ public Map> samplesFor(Set descriptors, Set return; } - samples.putIfAbsent(pcollectionId, Collections.EMPTY_LIST); + samples.putIfAbsent(pcollectionId, new ArrayList<>()); samples.get(pcollectionId).addAll(outputSampler.samples()); }); }); return samples; } - - /** @return samples from all PBDs and all PCollections. */ - public Map> allSamples() { - return samplesFor(ImmutableSet.of(), ImmutableSet.of()); - } - - /** - * @param descriptors PBDs to filter on. - * @return samples only from the given descriptors. - */ - public Map> samplesForDescriptors(Set descriptors) { - return samplesFor(descriptors, ImmutableSet.of()); - } - - /** - * @param pcollections PCollection ids to filter on. - * @return samples only from the given PCollections. - */ - public Map> samplesForPCollections(Set pcollections) { - return samplesFor(ImmutableSet.of(), pcollections); - } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index 45f4cf095522..bfa301018d5b 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -33,14 +33,16 @@ */ public class OutputSampler { private final Coder coder; - private final List buffer = new ArrayList<>(maxElements); private static final Logger LOG = LoggerFactory.getLogger(OutputSampler.class); + // Temporarily holds elements until the SDK receives a sample data request. + private final List buffer; + // Maximum number of elements in buffer. - private int maxElements = 10; + private final int maxElements; // Sampling rate. - private int sampleEveryN = 1000; + private final int sampleEveryN; // Total number of samples taken. private long numSamples = 0; @@ -48,14 +50,11 @@ public class OutputSampler { // Index into the buffer of where to overwrite samples. private int resampleIndex = 0; - public OutputSampler(Coder coder) { - this.coder = coder; - } - public OutputSampler(Coder coder, int maxElements, int sampleEveryN) { - this(coder); + this.coder = coder; this.maxElements = maxElements; this.sampleEveryN = sampleEveryN; + this.buffer = new ArrayList<>(this.maxElements); } /** @@ -63,7 +62,7 @@ public OutputSampler(Coder coder, int maxElements, int sampleEveryN) { * * @param element the element to sample. */ - public void sample(T element) { + public synchronized void sample(T element) { // Only sample the first 10 elements then after every `sampleEveryN`th element. numSamples += 1; if (numSamples > 10 && numSamples % sampleEveryN != 0) { @@ -87,8 +86,17 @@ public void sample(T element) { */ public List samples() { List ret = new ArrayList<>(); + + // Serializing can take a lot of CPU time for larger or complex elements. Copy the array here + // so as to not slow down the main processing hot path. + List copiedBuffer; + synchronized (this) { + copiedBuffer = new ArrayList<>(buffer); + clear(); + } + ByteArrayOutputStream stream = new ByteArrayOutputStream(); - for (T el : buffer) { + for (T el : copiedBuffer) { try { // This is deprecated, but until this is fully removed, this specifically needs the nested // context. This is because the SDK will need to decode the sampled elements with the @@ -102,7 +110,6 @@ public List samples() { } } - clear(); return ret; } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index de383f24efeb..97f9cd720a83 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -18,9 +18,10 @@ package org.apache.beam.fn.harness.debug; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -30,27 +31,88 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @RunWith(JUnit4.class) public class DataSamplerTest { - public byte[] encodeInt(Integer i) throws IOException { + byte[] encodeInt(Integer i) throws IOException { VarIntCoder coder = VarIntCoder.of(); ByteArrayOutputStream stream = new ByteArrayOutputStream(); coder.encode(i, stream); return stream.toByteArray(); } - public byte[] encodeString(String s) throws IOException { + byte[] encodeString(String s) throws IOException { StringUtf8Coder coder = StringUtf8Coder.of(); ByteArrayOutputStream stream = new ByteArrayOutputStream(); coder.encode(s, stream); return stream.toByteArray(); } + BeamFnApi.InstructionResponse getAllSamples(DataSampler dataSampler) { + BeamFnApi.InstructionRequest request = + BeamFnApi.InstructionRequest.newBuilder() + .setSample(BeamFnApi.SampleDataRequest.newBuilder().build()) + .build(); + return dataSampler.handleDataSampleRequest(request).build(); + } + + BeamFnApi.InstructionResponse getSamplesForPCollection( + DataSampler dataSampler, String pcollection) { + BeamFnApi.InstructionRequest request = + BeamFnApi.InstructionRequest.newBuilder() + .setSample( + BeamFnApi.SampleDataRequest.newBuilder().addPcollectionIds(pcollection).build()) + .build(); + return dataSampler.handleDataSampleRequest(request).build(); + } + + BeamFnApi.InstructionResponse getSamplesForDescriptors( + DataSampler dataSampler, List descriptors) { + BeamFnApi.InstructionRequest request = + BeamFnApi.InstructionRequest.newBuilder() + .setSample( + BeamFnApi.SampleDataRequest.newBuilder() + .addAllProcessBundleDescriptorIds(descriptors) + .build()) + .build(); + return dataSampler.handleDataSampleRequest(request).build(); + } + + BeamFnApi.InstructionResponse getSamplesFor( + DataSampler dataSampler, String descriptor, String pcollection) { + BeamFnApi.InstructionRequest request = + BeamFnApi.InstructionRequest.newBuilder() + .setSample( + BeamFnApi.SampleDataRequest.newBuilder() + .addProcessBundleDescriptorIds(descriptor) + .addPcollectionIds(pcollection) + .build()) + .build(); + return dataSampler.handleDataSampleRequest(request).build(); + } + + void assertHasSamples( + BeamFnApi.InstructionResponse response, String pcollection, Iterable elements) { + Map elementSamplesMap = + response.getSample().getElementSamplesMap(); + + assertFalse(elementSamplesMap.isEmpty()); + + BeamFnApi.SampleDataResponse.ElementList elementList = elementSamplesMap.get(pcollection); + assertNotNull(elementList); + + List expectedSamples = new ArrayList<>(); + for (byte[] el : elements) { + expectedSamples.add( + BeamFnApi.SampledElement.newBuilder().setElement(ByteString.copyFrom(el)).build()); + } + + assertTrue(elementList.getElementsList().containsAll(expectedSamples)); + } + /** * Smoke test that a samples show in the output map. * @@ -63,8 +125,8 @@ public void testSingleOutput() throws Exception { VarIntCoder coder = VarIntCoder.of(); sampler.sampleOutput("descriptor-id", "pcollection-id", coder).sample(1); - Map> samples = sampler.allSamples(); - assertThat(samples.get("pcollection-id"), contains(encodeInt(1))); + BeamFnApi.InstructionResponse samples = getAllSamples(sampler); + assertHasSamples(samples, "pcollection-id", Collections.singleton(encodeInt(1))); } /** @@ -80,9 +142,9 @@ public void testMultipleOutputs() throws Exception { sampler.sampleOutput("descriptor-id", "pcollection-id-1", coder).sample(1); sampler.sampleOutput("descriptor-id", "pcollection-id-2", coder).sample(2); - Map> samples = sampler.allSamples(); - assertThat(samples.get("pcollection-id-1"), contains(encodeInt(1))); - assertThat(samples.get("pcollection-id-2"), contains(encodeInt(2))); + BeamFnApi.InstructionResponse samples = getAllSamples(sampler); + assertHasSamples(samples, "pcollection-id-1", Collections.singleton(encodeInt(1))); + assertHasSamples(samples, "pcollection-id-2", Collections.singleton(encodeInt(2))); } /** @@ -98,8 +160,16 @@ public void testMultipleDescriptors() throws Exception { sampler.sampleOutput("descriptor-id-1", "pcollection-id", coder).sample(1); sampler.sampleOutput("descriptor-id-2", "pcollection-id", coder).sample(2); - Map> samples = sampler.allSamples(); - assertThat(samples.get("pcollection-id"), contains(encodeInt(1), encodeInt(2))); + BeamFnApi.InstructionResponse samples = getAllSamples(sampler); + assertHasSamples(samples, "pcollection-id", ImmutableList.of(encodeInt(1), encodeInt(2))); + } + + void generateStringSamples(DataSampler sampler) { + StringUtf8Coder coder = StringUtf8Coder.of(); + sampler.sampleOutput("a", "1", coder).sample("a1"); + sampler.sampleOutput("a", "2", coder).sample("a2"); + sampler.sampleOutput("b", "1", coder).sample("b1"); + sampler.sampleOutput("b", "2", coder).sample("b2"); } /** @@ -110,17 +180,12 @@ public void testMultipleDescriptors() throws Exception { @Test public void testFiltersSingleDescriptorId() throws Exception { DataSampler sampler = new DataSampler(10, 10); + generateStringSamples(sampler); - StringUtf8Coder coder = StringUtf8Coder.of(); - sampler.sampleOutput("a", "1", coder).sample("a1"); - sampler.sampleOutput("a", "2", coder).sample("a2"); - sampler.sampleOutput("b", "1", coder).sample("b1"); - sampler.sampleOutput("b", "2", coder).sample("b2"); - - Map> samples = - sampler.samplesForDescriptors(new HashSet<>(Collections.singletonList("a"))); - assertThat(samples.get("1"), contains(encodeString("a1"))); - assertThat(samples.get("2"), contains(encodeString("a2"))); + BeamFnApi.InstructionResponse samples = + getSamplesForDescriptors(sampler, ImmutableList.of("a")); + assertHasSamples(samples, "1", Collections.singleton(encodeString("a1"))); + assertHasSamples(samples, "2", Collections.singleton(encodeString("a2"))); } /** @@ -131,16 +196,12 @@ public void testFiltersSingleDescriptorId() throws Exception { @Test public void testFiltersMultipleDescriptorId() throws Exception { DataSampler sampler = new DataSampler(10, 10); + generateStringSamples(sampler); - StringUtf8Coder coder = StringUtf8Coder.of(); - sampler.sampleOutput("a", "1", coder).sample("a1"); - sampler.sampleOutput("a", "2", coder).sample("a2"); - sampler.sampleOutput("b", "1", coder).sample("b1"); - sampler.sampleOutput("b", "2", coder).sample("b2"); - - Map> samples = sampler.samplesForDescriptors(ImmutableSet.of("a", "b")); - assertThat(samples.get("1"), contains(encodeString("a1"), encodeString("b1"))); - assertThat(samples.get("2"), contains(encodeString("a2"), encodeString("b2"))); + BeamFnApi.InstructionResponse samples = + getSamplesForDescriptors(sampler, ImmutableList.of("a", "b")); + assertHasSamples(samples, "1", ImmutableList.of(encodeString("a1"), encodeString("b1"))); + assertHasSamples(samples, "2", ImmutableList.of(encodeString("a2"), encodeString("b2"))); } /** @@ -151,24 +212,10 @@ public void testFiltersMultipleDescriptorId() throws Exception { @Test public void testFiltersSinglePCollectionId() throws Exception { DataSampler sampler = new DataSampler(10, 10); + generateStringSamples(sampler); - StringUtf8Coder coder = StringUtf8Coder.of(); - sampler.sampleOutput("a", "1", coder).sample("a1"); - sampler.sampleOutput("a", "2", coder).sample("a2"); - sampler.sampleOutput("b", "1", coder).sample("b1"); - sampler.sampleOutput("b", "2", coder).sample("b2"); - - Map> samples = - sampler.samplesForPCollections(new HashSet<>(Collections.singletonList("1"))); - assertThat(samples.get("1"), containsInAnyOrder(encodeString("a1"), encodeString("b1"))); - } - - void generateStringSamples(DataSampler sampler) { - StringUtf8Coder coder = StringUtf8Coder.of(); - sampler.sampleOutput("a", "1", coder).sample("a1"); - sampler.sampleOutput("a", "2", coder).sample("a2"); - sampler.sampleOutput("b", "1", coder).sample("b1"); - sampler.sampleOutput("b", "2", coder).sample("b2"); + BeamFnApi.InstructionResponse samples = getSamplesForPCollection(sampler, "1"); + assertHasSamples(samples, "1", ImmutableList.of(encodeString("a1"), encodeString("b1"))); } /** @@ -186,56 +233,16 @@ public void testFiltersDescriptorAndPCollectionIds() throws Exception { for (String pcollectionId : pcollectionIds) { DataSampler sampler = new DataSampler(10, 10); generateStringSamples(sampler); - Map> actual = - sampler.samplesFor(ImmutableSet.of(descriptorId), ImmutableSet.of(pcollectionId)); + BeamFnApi.InstructionResponse samples = getSamplesFor(sampler, descriptorId, pcollectionId); System.out.print("Testing: " + descriptorId + pcollectionId + "..."); - assertThat(actual.size(), equalTo(1)); - assertThat(actual.get(pcollectionId), contains(encodeString(descriptorId + pcollectionId))); + assertThat(samples.getSample().getElementSamplesMap().size(), equalTo(1)); + assertHasSamples( + samples, + pcollectionId, + Collections.singleton(encodeString(descriptorId + pcollectionId))); System.out.println("ok"); } } } - - /** - * Test that the DataSampler can respond with the correct samples with filters. - * - * @throws Exception - */ - @Test - public void testMakesCorrectResponse() throws Exception { - DataSampler dataSampler = new DataSampler(); - generateStringSamples(dataSampler); - - // SampleDataRequest that filters on PCollection=1 and PBD ids = "a" or "b". - BeamFnApi.InstructionRequest request = - BeamFnApi.InstructionRequest.newBuilder() - .setSample( - BeamFnApi.SampleDataRequest.newBuilder() - .addPcollectionIds("1") - .addProcessBundleDescriptorIds("a") - .addProcessBundleDescriptorIds("b") - .build()) - .build(); - BeamFnApi.InstructionResponse actual = dataSampler.handleDataSampleRequest(request).build(); - BeamFnApi.InstructionResponse expected = - BeamFnApi.InstructionResponse.newBuilder() - .setSample( - BeamFnApi.SampleDataResponse.newBuilder() - .putElementSamples( - "1", - BeamFnApi.SampleDataResponse.ElementList.newBuilder() - .addElements( - BeamFnApi.SampledElement.newBuilder() - .setElement(ByteString.copyFrom(encodeString("a1"))) - .build()) - .addElements( - BeamFnApi.SampledElement.newBuilder() - .setElement(ByteString.copyFrom(encodeString("b1"))) - .build()) - .build()) - .build()) - .build(); - assertThat(actual, equalTo(expected)); - } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java index 0aea3e6e6017..286e3087f88e 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java @@ -20,10 +20,12 @@ import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.*; +import static org.junit.Assert.assertTrue; import java.io.ByteArrayOutputStream; import java.util.*; import org.apache.beam.fn.harness.PTransformRunnerFactoryTestContext; +import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.runners.core.construction.CoderTranslation; @@ -31,6 +33,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -65,6 +68,7 @@ public void testCreatingAndProcessingWithSampling() throws Exception { RunnerApi.Coder coder = CoderTranslation.toProto(StringUtf8Coder.of()).getCoder(); PTransformRunnerFactoryTestContext context = PTransformRunnerFactoryTestContext.builder(pTransformId, pTransform) + .processBundleDescriptorId("descriptor-id") .processBundleInstructionId("instruction-id") .pCollections(pCollectionMap) .coders(Collections.singletonMap("coder-id", coder)) @@ -89,13 +93,27 @@ public void testCreatingAndProcessingWithSampling() throws Exception { .withPipeline(Pipeline.create()); Coder rehydratedCoder = (Coder) rehydratedComponents.getCoder("coder-id"); - Map> samples = dataSampler.allSamples(); - assertThat(samples.keySet(), contains("inputTarget")); + BeamFnApi.InstructionRequest request = + BeamFnApi.InstructionRequest.newBuilder() + .setSample(BeamFnApi.SampleDataRequest.newBuilder().build()) + .build(); + BeamFnApi.InstructionResponse samples = dataSampler.handleDataSampleRequest(request).build(); + assertThat(samples.getSample().getElementSamplesMap().keySet(), contains("inputTarget")); // Ensure that the value was sampled. ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); rehydratedCoder.encode("Hello, World!", outputStream); byte[] encodedValue = outputStream.toByteArray(); - assertThat(samples.get("inputTarget"), contains(encodedValue)); + + assertTrue( + samples + .getSample() + .getElementSamplesMap() + .get("inputTarget") + .getElementsList() + .contains( + BeamFnApi.SampledElement.newBuilder() + .setElement(ByteString.copyFrom(encodedValue)) + .build())); } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java index 00a52df9039d..f3e0a5a06211 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -24,7 +24,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Map; import org.apache.beam.sdk.coders.VarIntCoder; import org.junit.Test; import org.junit.runner.RunWith; @@ -46,11 +45,8 @@ public byte[] encodeInt(Integer i) throws IOException { */ @Test public void testSamplesFirstN() throws Exception { - DataSampler sampler = new DataSampler(10, 10); - VarIntCoder coder = VarIntCoder.of(); - OutputSampler outputSampler = - sampler.sampleOutput("descriptor-id", "pcollection-id", coder); + OutputSampler outputSampler = new OutputSampler<>(coder, 10, 10); // Purposely go over maxSamples and sampleEveryN. This helps to increase confidence. for (int i = 0; i < 15; ++i) { @@ -63,8 +59,8 @@ public void testSamplesFirstN() throws Exception { expected.add(encodeInt(i)); } - Map> samples = sampler.allSamples(); - assertThat(samples.get("pcollection-id"), containsInAnyOrder(expected.toArray())); + List samples = outputSampler.samples(); + assertThat(samples, containsInAnyOrder(expected.toArray())); } /** @@ -74,11 +70,8 @@ public void testSamplesFirstN() throws Exception { */ @Test public void testActsLikeCircularBuffer() throws Exception { - DataSampler sampler = new DataSampler(5, 20); - VarIntCoder coder = VarIntCoder.of(); - OutputSampler outputSampler = - sampler.sampleOutput("descriptor-id", "pcollection-id", coder); + OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20); for (int i = 0; i < 100; ++i) { outputSampler.sample(i); @@ -94,7 +87,7 @@ public void testActsLikeCircularBuffer() throws Exception { expected.add(encodeInt(79)); expected.add(encodeInt(99)); - Map> samples = sampler.allSamples(); - assertThat(samples.get("pcollection-id"), containsInAnyOrder(expected.toArray())); + List samples = outputSampler.samples(); + assertThat(samples, containsInAnyOrder(expected.toArray())); } } From 5609e4a7a307c928599cc38bf1fc4775d964a2f3 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Thu, 9 Feb 2023 11:36:04 -0800 Subject: [PATCH 14/42] give default pbd id to test context --- .../beam/fn/harness/PTransformRunnerFactoryTestContext.java | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java index db00cb4812fb..3cca0a172c16 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java @@ -107,6 +107,7 @@ public CompletableFuture handle(StateRequest.Builder requestBuild () -> { throw new UnsupportedOperationException("Unexpected call during test."); }) + .processBundleDescriptorId("") .cacheTokensSupplier(() -> Collections.emptyList()) .bundleCacheSupplier(() -> Caches.noop()) .processWideCache(Caches.noop()) From 694c929635c683f0a67dfa70c3d1a91b1df0e450 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Thu, 9 Feb 2023 11:43:41 -0800 Subject: [PATCH 15/42] address spotlesscheck --- .../beam/fn/harness/debug/package-info.java | 20 ++++++++++ .../fn/harness/debug/DataSamplerTest.java | 6 ++- .../DataSamplingDescriptorModifierTest.java | 40 +++++++++---------- .../debug/DataSamplingFnRunnerTest.java | 7 +++- 4 files changed, 50 insertions(+), 23 deletions(-) create mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/package-info.java diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/package-info.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/package-info.java new file mode 100644 index 000000000000..978bcd346d47 --- /dev/null +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/package-info.java @@ -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. + */ + +/** Classes and utilities related to debugging features. */ +package org.apache.beam.fn.harness.debug; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index 97f9cd720a83..b3030cfb3b05 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -25,7 +25,11 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java index 5aa49eefb69a..0b606c59dc9e 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java @@ -38,50 +38,50 @@ public class DataSamplingDescriptorModifierTest { public void testThatDataSamplingTransformIsMade() { DataSamplingDescriptorModifier modifier = new DataSamplingDescriptorModifier(); - final String PCOLLECTION_ID_A = "pcollection-id-a"; - final String PCOLLECTION_ID_B = "pcollection-id-b"; - final String CODER_ID_A = "coder-id-a"; - final String CODER_ID_B = "coder-id-b"; + final String pcollectionIdA = "pcollection-id-a"; + final String pcollectionIdB = "pcollection-id-b"; + final String coderIdA = "coder-id-a"; + final String coderIdB = "coder-id-b"; BeamFnApi.ProcessBundleDescriptor descriptor = BeamFnApi.ProcessBundleDescriptor.newBuilder() .putPcollections( - PCOLLECTION_ID_A, + pcollectionIdA, RunnerApi.PCollection.newBuilder() - .setUniqueName(PCOLLECTION_ID_A) - .setCoderId(CODER_ID_A) + .setUniqueName(pcollectionIdA) + .setCoderId(coderIdA) .build()) .putPcollections( - PCOLLECTION_ID_B, + pcollectionIdB, RunnerApi.PCollection.newBuilder() - .setUniqueName(PCOLLECTION_ID_B) - .setCoderId(CODER_ID_B) + .setUniqueName(pcollectionIdB) + .setCoderId(coderIdB) .build()) .putCoders( - CODER_ID_A, + coderIdA, RunnerApi.Coder.newBuilder() .setSpec(RunnerApi.FunctionSpec.newBuilder().setUrn(DataSamplingFnRunner.URN)) .build()) .putCoders( - CODER_ID_B, + coderIdB, RunnerApi.Coder.newBuilder() .setSpec(RunnerApi.FunctionSpec.newBuilder().setUrn(DataSamplingFnRunner.URN)) .build()) .build(); - final String PTRANSFORM_ID_A = "synthetic-data-sampling-transform-" + PCOLLECTION_ID_A; - final String PTRANSFORM_ID_B = "synthetic-data-sampling-transform-" + PCOLLECTION_ID_B; + final String ptransformIdA = "synthetic-data-sampling-transform-" + pcollectionIdA; + final String ptransformIdB = "synthetic-data-sampling-transform-" + pcollectionIdB; BeamFnApi.ProcessBundleDescriptor modified = modifier.ModifyProcessBundleDescriptor(descriptor); assertThat(modified.getTransformsCount(), equalTo(2)); - RunnerApi.PTransform samplingTransformA = modified.getTransformsMap().get(PTRANSFORM_ID_A); - assertThat(samplingTransformA.getUniqueName(), equalTo(PTRANSFORM_ID_A)); + RunnerApi.PTransform samplingTransformA = modified.getTransformsMap().get(ptransformIdA); + assertThat(samplingTransformA.getUniqueName(), equalTo(ptransformIdA)); assertThat(samplingTransformA.getSpec().getUrn(), equalTo(DataSamplingFnRunner.URN)); - assertThat(samplingTransformA.getInputsMap(), hasEntry("main", PCOLLECTION_ID_A)); + assertThat(samplingTransformA.getInputsMap(), hasEntry("main", pcollectionIdA)); - RunnerApi.PTransform samplingTransformB = modified.getTransformsMap().get(PTRANSFORM_ID_B); - assertThat(samplingTransformB.getUniqueName(), equalTo(PTRANSFORM_ID_B)); + RunnerApi.PTransform samplingTransformB = modified.getTransformsMap().get(ptransformIdB); + assertThat(samplingTransformB.getUniqueName(), equalTo(ptransformIdB)); assertThat(samplingTransformB.getSpec().getUrn(), equalTo(DataSamplingFnRunner.URN)); - assertThat(samplingTransformB.getInputsMap(), hasEntry("main", PCOLLECTION_ID_B)); + assertThat(samplingTransformB.getInputsMap(), hasEntry("main", pcollectionIdB)); } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java index 286e3087f88e..52c56cf91da7 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java @@ -19,11 +19,14 @@ import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.*; +import static org.hamcrest.Matchers.contains; import static org.junit.Assert.assertTrue; import java.io.ByteArrayOutputStream; -import java.util.*; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + import org.apache.beam.fn.harness.PTransformRunnerFactoryTestContext; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.pipeline.v1.RunnerApi; From 519aece99ab43f345341a7b8eeaac9a7222d1d59 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Thu, 9 Feb 2023 11:44:29 -0800 Subject: [PATCH 16/42] spotless apply --- .../java/org/apache/beam/fn/harness/debug/DataSamplerTest.java | 1 - .../apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java | 1 - 2 files changed, 2 deletions(-) diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index b3030cfb3b05..788cae7e2c46 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -29,7 +29,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; - import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java index 52c56cf91da7..501816ae4c23 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java @@ -26,7 +26,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; - import org.apache.beam.fn.harness.PTransformRunnerFactoryTestContext; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.pipeline.v1.RunnerApi; From 8efbb154d1be5cf3eec49089c0c447033a1971ae Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Thu, 9 Feb 2023 13:50:15 -0800 Subject: [PATCH 17/42] style guide spotless apply --- .../java/org/apache/beam/fn/harness/FnHarness.java | 2 +- .../fn/harness/ProcessBundleDescriptorModifier.java | 2 +- .../fn/harness/control/ProcessBundleHandler.java | 12 +++++++++++- .../debug/DataSamplingDescriptorModifier.java | 2 +- .../debug/DataSamplingDescriptorModifierTest.java | 2 +- 5 files changed, 15 insertions(+), 5 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index 53659d98584d..05eca62eea4b 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -290,7 +290,7 @@ private BeamFnApi.ProcessBundleDescriptor loadDescriptor(String id) { for (ProcessBundleDescriptorModifier modifier : modifiers) { try { LOG.debug("Modifying graph with " + modifier); - descriptor = modifier.ModifyProcessBundleDescriptor(descriptor); + descriptor = modifier.modifyProcessBundleDescriptor(descriptor); } catch (ProcessBundleDescriptorModifier.GraphModificationException e) { LOG.warn("Could not modify graph with " + modifier + ": " + e.getMessage()); } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java index 9044abd837f9..db96e0422428 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java @@ -37,6 +37,6 @@ public GraphModificationException(String message, Throwable throwable) { * failure. Can be used to instrument functionality onto an existing ProcessBundleDescriptor. For * instance, this is used to add DataSampling PTransforms to a graph to sample in-flight elements. */ - BeamFnApi.ProcessBundleDescriptor ModifyProcessBundleDescriptor( + BeamFnApi.ProcessBundleDescriptor modifyProcessBundleDescriptor( BeamFnApi.ProcessBundleDescriptor pbd) throws GraphModificationException; } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index cf21f5e51de6..8a772b34e944 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -20,7 +20,17 @@ import com.google.auto.value.AutoValue; import java.io.IOException; import java.time.Duration; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.WeakHashMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Phaser; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java index 832ec3e15eb0..e3ca74cdf987 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java @@ -27,7 +27,7 @@ */ public class DataSamplingDescriptorModifier implements ProcessBundleDescriptorModifier { @Override - public BeamFnApi.ProcessBundleDescriptor ModifyProcessBundleDescriptor( + public BeamFnApi.ProcessBundleDescriptor modifyProcessBundleDescriptor( BeamFnApi.ProcessBundleDescriptor pbd) { BeamFnApi.ProcessBundleDescriptor.Builder builder = pbd.toBuilder(); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java index 0b606c59dc9e..7847665e9440 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java @@ -71,7 +71,7 @@ public void testThatDataSamplingTransformIsMade() { final String ptransformIdA = "synthetic-data-sampling-transform-" + pcollectionIdA; final String ptransformIdB = "synthetic-data-sampling-transform-" + pcollectionIdB; - BeamFnApi.ProcessBundleDescriptor modified = modifier.ModifyProcessBundleDescriptor(descriptor); + BeamFnApi.ProcessBundleDescriptor modified = modifier.modifyProcessBundleDescriptor(descriptor); assertThat(modified.getTransformsCount(), equalTo(2)); RunnerApi.PTransform samplingTransformA = modified.getTransformsMap().get(ptransformIdA); From cd3732dbe291845b7f74d24b0e37070bab3cf62d Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Thu, 9 Feb 2023 14:47:31 -0800 Subject: [PATCH 18/42] add serviceloader --- .../org/apache/beam/fn/harness/control/ProcessBundleHandler.java | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index 8a772b34e944..b619341d0cbc 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.ServiceLoader; import java.util.Set; import java.util.WeakHashMap; import java.util.concurrent.CompletableFuture; From 415e3f0ab8b34b3ca237a3c40c18b0b4eb64cf90 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Fri, 10 Feb 2023 00:02:14 -0800 Subject: [PATCH 19/42] change datasampling to modify the consumers and not graph for sampling --- .../org/apache/beam/fn/harness/FnHarness.java | 39 ++--- .../fn/harness/PTransformRunnerFactory.java | 12 -- .../ProcessBundleDescriptorModifier.java | 42 ----- .../harness/control/ProcessBundleHandler.java | 19 +-- .../data/PCollectionConsumerRegistry.java | 52 +++++- .../beam/fn/harness/debug/DataSampler.java | 45 ++---- .../debug/DataSamplingDescriptorModifier.java | 49 ------ .../harness/debug/DataSamplingFnRunner.java | 111 ------------- .../beam/fn/harness/debug/OutputSampler.java | 153 ++++++++++-------- .../PTransformRunnerFactoryTestContext.java | 6 - .../fn/harness/debug/DataSamplerTest.java | 107 +++--------- .../DataSamplingDescriptorModifierTest.java | 87 ---------- .../debug/DataSamplingFnRunnerTest.java | 121 -------------- .../fn/harness/debug/OutputSamplerTest.java | 6 +- 14 files changed, 192 insertions(+), 657 deletions(-) delete mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java delete mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java delete mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java delete mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java delete mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index 05eca62eea4b..fcc6f7be987f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -17,7 +17,6 @@ */ package org.apache.beam.fn.harness; -import java.util.ArrayList; import java.util.Collections; import java.util.EnumMap; import java.util.List; @@ -33,7 +32,6 @@ import org.apache.beam.fn.harness.control.ProcessBundleHandler; import org.apache.beam.fn.harness.data.BeamFnDataGrpcClient; import org.apache.beam.fn.harness.debug.DataSampler; -import org.apache.beam.fn.harness.debug.DataSamplingDescriptorModifier; import org.apache.beam.fn.harness.logging.BeamFnLoggingClient; import org.apache.beam.fn.harness.state.BeamFnStateGrpcClientCache; import org.apache.beam.fn.harness.status.BeamFnStatusClient; @@ -96,7 +94,7 @@ public class FnHarness { private static final String RUNNER_CAPABILITIES = "RUNNER_CAPABILITIES"; private static final String ENABLE_DATA_SAMPLING_EXPERIMENT = "enable_data_sampling"; private static final Logger LOG = LoggerFactory.getLogger(FnHarness.class); - private static final DataSampler dataSampler = new DataSampler(); + private static DataSampler dataSampler = new DataSampler(); private static Endpoints.ApiServiceDescriptor getApiServiceDescriptor(String descriptor) throws TextFormat.ParseException { @@ -255,16 +253,12 @@ public static void main( FinalizeBundleHandler finalizeBundleHandler = new FinalizeBundleHandler(executorService); - // Add any graph modifications. - List modifierList = new ArrayList<>(); + // Create the sampler, if the experiment is enabled. Optional> experimentList = Optional.ofNullable(options.as(ExperimentalOptions.class).getExperiments()); - - // If data sampling is enabled, then modify the graph to add any DataSampling Operations. - if (experimentList.isPresent() - && experimentList.get().contains(ENABLE_DATA_SAMPLING_EXPERIMENT)) { - modifierList.add(new DataSamplingDescriptorModifier()); - } + boolean shouldSample = + experimentList.isPresent() + && experimentList.get().contains(ENABLE_DATA_SAMPLING_EXPERIMENT); // Retrieves the ProcessBundleDescriptor from cache. Requests the PBD from the Runner if it // doesn't exist. Additionally, runs any graph modifications. @@ -274,29 +268,16 @@ public static void main( private final Cache cache = Caches.subCache(processWideCache, PROCESS_BUNDLE_DESCRIPTORS); - private final List modifiers = modifierList; - @Override public BeamFnApi.ProcessBundleDescriptor apply(String id) { return cache.computeIfAbsent(id, this::loadDescriptor); } private BeamFnApi.ProcessBundleDescriptor loadDescriptor(String id) { - ProcessBundleDescriptor descriptor = - blockingControlStub.getProcessBundleDescriptor( - BeamFnApi.GetProcessBundleDescriptorRequest.newBuilder() - .setProcessBundleDescriptorId(id) - .build()); - for (ProcessBundleDescriptorModifier modifier : modifiers) { - try { - LOG.debug("Modifying graph with " + modifier); - descriptor = modifier.modifyProcessBundleDescriptor(descriptor); - } catch (ProcessBundleDescriptorModifier.GraphModificationException e) { - LOG.warn("Could not modify graph with " + modifier + ": " + e.getMessage()); - } - } - - return descriptor; + return blockingControlStub.getProcessBundleDescriptor( + BeamFnApi.GetProcessBundleDescriptorRequest.newBuilder() + .setProcessBundleDescriptorId(id) + .build()); } }; @@ -313,7 +294,7 @@ private BeamFnApi.ProcessBundleDescriptor loadDescriptor(String id) { metricsShortIds, executionStateSampler, processWideCache, - dataSampler); + shouldSample ? dataSampler : null); logging.setProcessBundleHandler(processBundleHandler); BeamFnStatusClient beamFnStatusClient = null; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java index 4ddfcdfae4a1..e7b29550afdf 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java @@ -20,13 +20,11 @@ import java.io.IOException; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.function.Supplier; import org.apache.beam.fn.harness.control.BundleProgressReporter; import org.apache.beam.fn.harness.control.BundleSplitListener; import org.apache.beam.fn.harness.data.BeamFnDataClient; -import org.apache.beam.fn.harness.debug.DataSampler; import org.apache.beam.fn.harness.state.BeamFnStateClient; import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleRequest; import org.apache.beam.model.pipeline.v1.Endpoints; @@ -61,9 +59,6 @@ interface Context { /** A client for handling state requests. */ BeamFnStateClient getBeamFnStateClient(); - /** The id of the parent ProcessBundleDescriptor. */ - String getProcessBundleDescriptorId(); - /** The id of the PTransform. */ String getPTransformId(); @@ -168,13 +163,6 @@ void addIncomingTimerEndpoint( * instant provides the timeout on how long the finalization callback is valid for. */ DoFn.BundleFinalizer getBundleFinalizer(); - - /** - * A DataSampler can be used to sample in-flight elements. This is used to plumb a global - * DataSampler to DataSampler operations in order to perform said sampling. Only present when - * using the "enable_data_sampling" experiment. - */ - Optional getDataSampler(); } /** diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java deleted file mode 100644 index db96e0422428..000000000000 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/ProcessBundleDescriptorModifier.java +++ /dev/null @@ -1,42 +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.fn.harness; - -import org.apache.beam.model.fnexecution.v1.BeamFnApi; - -public interface ProcessBundleDescriptorModifier { - class GraphModificationException extends Exception { - public GraphModificationException() {} - - public GraphModificationException(String message) { - super(message); - } - - public GraphModificationException(String message, Throwable throwable) { - super(throwable); - } - } - - /** - * Modifies the given ProcessBundleDescriptor in-place. Throws a `GraphModificationException` on - * failure. Can be used to instrument functionality onto an existing ProcessBundleDescriptor. For - * instance, this is used to add DataSampling PTransforms to a graph to sample in-flight elements. - */ - BeamFnApi.ProcessBundleDescriptor modifyProcessBundleDescriptor( - BeamFnApi.ProcessBundleDescriptor pbd) throws GraphModificationException; -} diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index b619341d0cbc..6bbb477ae005 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -28,7 +28,6 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.ServiceLoader; import java.util.Set; import java.util.WeakHashMap; @@ -167,7 +166,7 @@ public class ProcessBundleHandler { @VisibleForTesting final BundleProcessorCache bundleProcessorCache; private final Set runnerCapabilities; - private final DataSampler dataSampler; + private final @Nullable DataSampler dataSampler; public ProcessBundleHandler( PipelineOptions options, @@ -179,7 +178,7 @@ public ProcessBundleHandler( ShortIdMap shortIds, ExecutionStateSampler executionStateSampler, Cache processWideCache, - DataSampler dataSampler) { + @Nullable DataSampler dataSampler) { this( options, runnerCapabilities, @@ -208,7 +207,7 @@ public ProcessBundleHandler( Map urnToPTransformRunnerFactoryMap, Cache processWideCache, BundleProcessorCache bundleProcessorCache, - DataSampler dataSampler) { + @Nullable DataSampler dataSampler) { this.options = options; this.fnApiRegistry = fnApiRegistry; this.beamFnDataClient = beamFnDataClient; @@ -328,11 +327,6 @@ public BeamFnStateClient getBeamFnStateClient() { return beamFnStateClient; } - @Override - public String getProcessBundleDescriptorId() { - return processBundleDescriptor.getId(); - } - @Override public String getPTransformId() { return pTransformId; @@ -494,11 +488,6 @@ public BundleSplitListener getSplitListener() { public BundleFinalizer getBundleFinalizer() { return bundleFinalizer; } - - @Override - public Optional getDataSampler() { - return Optional.ofNullable(dataSampler); - } }); if (runner instanceof BeamFnDataReadRunner) { channelRoots.add((BeamFnDataReadRunner) runner); @@ -789,7 +778,7 @@ private BundleProcessor createBundleProcessor( bundleProgressReporterAndRegistrar.register(stateTracker); PCollectionConsumerRegistry pCollectionConsumerRegistry = new PCollectionConsumerRegistry( - stateTracker, shortIds, bundleProgressReporterAndRegistrar, bundleDescriptor); + stateTracker, shortIds, bundleProgressReporterAndRegistrar, bundleDescriptor, dataSampler); HashSet processedPTransformIds = new HashSet<>(); PTransformFunctionRegistry startFunctionRegistry = diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java index 45298a68d98c..39cd422c91cc 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.Random; +import javax.annotation.Nullable; import org.apache.beam.fn.harness.HandlesSplits; import org.apache.beam.fn.harness.control.BundleProgressReporter; import org.apache.beam.fn.harness.control.ExecutionStateSampler.ExecutionState; @@ -31,6 +32,8 @@ import org.apache.beam.fn.harness.control.Metrics; import org.apache.beam.fn.harness.control.Metrics.BundleCounter; import org.apache.beam.fn.harness.control.Metrics.BundleDistribution; +import org.apache.beam.fn.harness.debug.DataSampler; +import org.apache.beam.fn.harness.debug.OutputSampler; import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleDescriptor; import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; import org.apache.beam.model.pipeline.v1.RunnerApi; @@ -87,12 +90,27 @@ public static ConsumerAndMetadata forConsumer( private final BundleProgressReporter.Registrar bundleProgressReporterRegistrar; private final ProcessBundleDescriptor processBundleDescriptor; private final RehydratedComponents rehydratedComponents; + private final @Nullable DataSampler dataSampler; public PCollectionConsumerRegistry( ExecutionStateTracker stateTracker, ShortIdMap shortIdMap, BundleProgressReporter.Registrar bundleProgressReporterRegistrar, ProcessBundleDescriptor processBundleDescriptor) { + this( + stateTracker, + shortIdMap, + bundleProgressReporterRegistrar, + processBundleDescriptor, + null); + } + + public PCollectionConsumerRegistry( + ExecutionStateTracker stateTracker, + ShortIdMap shortIdMap, + BundleProgressReporter.Registrar bundleProgressReporterRegistrar, + ProcessBundleDescriptor processBundleDescriptor, + @Nullable DataSampler dataSampler) { this.stateTracker = stateTracker; this.shortIdMap = shortIdMap; this.pCollectionIdsToConsumers = new HashMap<>(); @@ -106,6 +124,7 @@ public PCollectionConsumerRegistry( .putAllPcollections(processBundleDescriptor.getPcollectionsMap()) .putAllWindowingStrategies(processBundleDescriptor.getWindowingStrategiesMap()) .build()); + this.dataSampler = dataSampler; } /** @@ -201,16 +220,16 @@ public FnDataReceiver> getMultiplexingConsumer(String pCollecti if (consumerAndMetadatas.size() == 1) { ConsumerAndMetadata consumerAndMetadata = consumerAndMetadatas.get(0); if (consumerAndMetadata.getConsumer() instanceof HandlesSplits) { - return new SplittingMetricTrackingFnDataReceiver(pcId, coder, consumerAndMetadata); + return new SplittingMetricTrackingFnDataReceiver(pcId, coder, consumerAndMetadata, dataSampler); } - return new MetricTrackingFnDataReceiver(pcId, coder, consumerAndMetadata); + return new MetricTrackingFnDataReceiver(pcId, coder, consumerAndMetadata, dataSampler); } else { /* TODO(SDF), Consider supporting splitting each consumer individually. This would never come up in the existing SDF expansion, but might be useful to support fused SDF nodes. This would require dedicated delivery of the split results to each of the consumers separately. */ return new MultiplexingMetricTrackingFnDataReceiver( - pcId, coder, ImmutableList.copyOf(consumerAndMetadatas)); + pcId, coder, ImmutableList.copyOf(consumerAndMetadatas), dataSampler); } }); } @@ -228,9 +247,11 @@ private class MetricTrackingFnDataReceiver implements FnDataReceiver sampledByteSizeDistribution; private final Coder coder; + private OutputSampler outputSampler; public MetricTrackingFnDataReceiver( - String pCollectionId, Coder coder, ConsumerAndMetadata consumerAndMetadata) { + String pCollectionId, Coder coder, ConsumerAndMetadata consumerAndMetadata, + @Nullable DataSampler dataSampler) { this.delegate = consumerAndMetadata.getConsumer(); this.executionState = consumerAndMetadata.getExecutionState(); @@ -266,6 +287,9 @@ public MetricTrackingFnDataReceiver( bundleProgressReporterRegistrar.register(sampledByteSizeUnderlyingDistribution); this.coder = coder; + if (dataSampler != null) { + this.outputSampler = dataSampler.sampleOutput(pCollectionId, coder); + } } @Override @@ -276,6 +300,10 @@ public void accept(WindowedValue input) throws Exception { // we have window optimization. this.sampledByteSizeDistribution.tryUpdate(input.getValue(), this.coder); + if (outputSampler != null) { + outputSampler.sample(input.getValue()); + } + // Use the ExecutionStateTracker and enter an appropriate state to track the // Process Bundle Execution time metric and also ensure user counters can get an appropriate // metrics container. @@ -302,9 +330,11 @@ private class MultiplexingMetricTrackingFnDataReceiver private final BundleCounter elementCountCounter; private final SampleByteSizeDistribution sampledByteSizeDistribution; private final Coder coder; + private @Nullable OutputSampler outputSampler = null; public MultiplexingMetricTrackingFnDataReceiver( - String pCollectionId, Coder coder, List consumerAndMetadatas) { + String pCollectionId, Coder coder, List consumerAndMetadatas, + @Nullable DataSampler dataSampler) { this.consumerAndMetadatas = consumerAndMetadatas; HashMap labels = new HashMap<>(); @@ -339,6 +369,9 @@ public MultiplexingMetricTrackingFnDataReceiver( bundleProgressReporterRegistrar.register(sampledByteSizeUnderlyingDistribution); this.coder = coder; + if (dataSampler != null) { + this.outputSampler = dataSampler.sampleOutput(pCollectionId, coder); + } } @Override @@ -349,6 +382,10 @@ public void accept(WindowedValue input) throws Exception { // when we have window optimization. this.sampledByteSizeDistribution.tryUpdate(input.getValue(), coder); + if (outputSampler != null) { + outputSampler.sample(input.getValue()); + } + // Use the ExecutionStateTracker and enter an appropriate state to track the // Process Bundle Execution time metric and also ensure user counters can get an appropriate // metrics container. @@ -377,8 +414,9 @@ private class SplittingMetricTrackingFnDataReceiver extends MetricTrackingFnD private final HandlesSplits delegate; public SplittingMetricTrackingFnDataReceiver( - String pCollection, Coder coder, ConsumerAndMetadata consumerAndMetadata) { - super(pCollection, coder, consumerAndMetadata); + String pCollection, Coder coder, ConsumerAndMetadata consumerAndMetadata, + @Nullable DataSampler dataSampler) { + super(pCollection, coder, consumerAndMetadata, dataSampler); this.delegate = (HandlesSplits) consumerAndMetadata.getConsumer(); } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index 34321c055500..7ce1df0dce2f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -59,11 +59,9 @@ public DataSampler(int maxSamples, int sampleEveryN) { // Sampling rate. private final int sampleEveryN; - // The fully-qualified type is: Map[ProcessBundleDescriptorId, [PCollectionId, OutputSampler]]. - // The DataSampler object lives on the same level of the FnHarness. This means that many threads - // can and will access this simultaneously. However, ProcessBundleDescriptors are unique per - // thread, so only synchronization is needed on the outermost map. - private final Map>> outputSamplers = + // The fully-qualified type is: Map[PCollectionId, OutputSampler.SamplerState]. In order to sample + // on a PCollection-basis and not per-bundle, this keeps track of shared samples between states. + private final Map> outputSamplers = new ConcurrentHashMap<>(); /** @@ -71,20 +69,15 @@ public DataSampler(int maxSamples, int sampleEveryN) { * ProcessBundleDescriptor. Uses the given coder encode samples as bytes when responding to a * SampleDataRequest. * - * @param processBundleDescriptorId The PBD to sample from. * @param pcollectionId The PCollection to take intermittent samples from. * @param coder The coder associated with the PCollection. Coder may be from a nested context. * @param The type of element contained in the PCollection. * @return the OutputSampler corresponding to the unique PBD and PCollection. */ - public OutputSampler sampleOutput( - String processBundleDescriptorId, String pcollectionId, Coder coder) { - outputSamplers.putIfAbsent(processBundleDescriptorId, new HashMap<>()); - Map> samplers = outputSamplers.get(processBundleDescriptorId); - samplers.putIfAbsent( - pcollectionId, new OutputSampler(coder, this.maxSamples, this.sampleEveryN)); - - return (OutputSampler) samplers.get(pcollectionId); + public OutputSampler sampleOutput(String pcollectionId, Coder coder) { + outputSamplers.putIfAbsent( + pcollectionId, new OutputSampler.SampleState<>(coder, this.maxSamples, this.sampleEveryN)); + return new OutputSampler<>((OutputSampler.SampleState) outputSamplers.get(pcollectionId)); } /** @@ -99,9 +92,7 @@ public BeamFnApi.InstructionResponse.Builder handleDataSampleRequest( BeamFnApi.SampleDataRequest sampleDataRequest = request.getSample(); Map> responseSamples = - samplesFor( - sampleDataRequest.getProcessBundleDescriptorIdsList(), - sampleDataRequest.getPcollectionIdsList()); + samplesFor(sampleDataRequest.getPcollectionIdsList()); BeamFnApi.SampleDataResponse.Builder response = BeamFnApi.SampleDataResponse.newBuilder(); for (String pcollectionId : responseSamples.keySet()) { @@ -120,33 +111,23 @@ public BeamFnApi.InstructionResponse.Builder handleDataSampleRequest( * Returns a map from PCollection to its samples. Samples are filtered on * ProcessBundleDescriptorIds and PCollections. Thread-safe. * - * @param descriptors PCollections under each PBD id will be unioned. If empty, allows all - * descriptors. * @param pcollections Filters all PCollections on this set. If empty, allows all PCollections. * @return a map from PCollection to its samples. */ - private Map> samplesFor( - List descriptors, List pcollections) { + private Map> samplesFor(List pcollections) { Map> samples = new HashMap<>(); // Safe to iterate as the ConcurrentHashMap will return each element at most once and will not // throw // ConcurrentModificationException. outputSamplers.forEach( - (descriptorId, samplers) -> { - if (!descriptors.isEmpty() && !descriptors.contains(descriptorId)) { + (pcollectionId, outputSampler) -> { + if (!pcollections.isEmpty() && !pcollections.contains(pcollectionId)) { return; } - samplers.forEach( - (pcollectionId, outputSampler) -> { - if (!pcollections.isEmpty() && !pcollections.contains(pcollectionId)) { - return; - } - - samples.putIfAbsent(pcollectionId, new ArrayList<>()); - samples.get(pcollectionId).addAll(outputSampler.samples()); - }); + samples.putIfAbsent(pcollectionId, new ArrayList<>()); + samples.get(pcollectionId).addAll(outputSampler.samples()); }); return samples; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java deleted file mode 100644 index e3ca74cdf987..000000000000 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java +++ /dev/null @@ -1,49 +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.fn.harness.debug; - -import org.apache.beam.fn.harness.ProcessBundleDescriptorModifier; -import org.apache.beam.model.fnexecution.v1.BeamFnApi; -import org.apache.beam.model.pipeline.v1.RunnerApi; - -/** - * Modifies the given ProcessBundleDescriptor by adding a DataSampling operation as a consumer to - * every PCollection. - */ -public class DataSamplingDescriptorModifier implements ProcessBundleDescriptorModifier { - @Override - public BeamFnApi.ProcessBundleDescriptor modifyProcessBundleDescriptor( - BeamFnApi.ProcessBundleDescriptor pbd) { - BeamFnApi.ProcessBundleDescriptor.Builder builder = pbd.toBuilder(); - - // Get all PCollections to modify. - for (String pcollectionId : pbd.getPcollectionsMap().keySet()) { - // Create a new DataSampling PTransform that consumes that given PCollection. - String transformId = "synthetic-data-sampling-transform-" + pcollectionId; - builder.putTransforms( - transformId, - RunnerApi.PTransform.newBuilder() - .setUniqueName(transformId) - .setSpec(RunnerApi.FunctionSpec.newBuilder().setUrn(DataSamplingFnRunner.URN)) - .putInputs("main", pcollectionId) - .build()); - } - - return builder.build(); - } -} diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java deleted file mode 100644 index 5d3e29148b58..000000000000 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunner.java +++ /dev/null @@ -1,111 +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.fn.harness.debug; - -import com.google.auto.service.AutoService; -import java.io.IOException; -import java.util.Map; -import java.util.Optional; -import org.apache.beam.fn.harness.PTransformRunnerFactory; -import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.fn.data.FnDataReceiver; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@SuppressWarnings({ - "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) -}) - -/** - * This class is in charge of retrieving and sampling in-flight elements. This uses an internal only - * URN to identify it. - */ -public class DataSamplingFnRunner { - public static final String URN = "beam:internal:sampling:v1"; - - private static final Logger LOG = LoggerFactory.getLogger(DataSamplingFnRunner.class); - - @AutoService(PTransformRunnerFactory.Registrar.class) - public static class Registrar implements PTransformRunnerFactory.Registrar { - - @Override - public Map getPTransformRunnerFactories() { - return ImmutableMap.of(URN, new Factory()); - } - } - - public static class Factory implements PTransformRunnerFactory { - @Override - public DataSamplingFnRunner createRunnerForPTransform(Context context) throws IOException { - DataSamplingFnRunner runner = new DataSamplingFnRunner(); - - // This is only present when the "enable_data_sampling" experiment is used. Returns a NOOP - // runner if Datasampling is disabled. - Optional maybeDataSampler = context.getDataSampler(); - if (!maybeDataSampler.isPresent()) { - LOG.warn( - "Trying to sample output but DataSampler is not present. Is \"enable_data_sampling\" set?"); - return runner; - } - - DataSampler dataSampler = maybeDataSampler.get(); - - // This should only have one input, the PCollection to sample - String inputPCollectionId = - Iterables.getOnlyElement(context.getPTransform().getInputsMap().values()); - - RunnerApi.PCollection inputPCollection = context.getPCollections().get(inputPCollectionId); - if (inputPCollection == null) { - LOG.warn( - "Expected input PCollection \"" - + inputPCollectionId - + "\" does not exist in PCollections map."); - return runner; - } - - // This coder is used to encode sampled elements to sample back to the Runner Harness. - String inputCoderId = inputPCollection.getCoderId(); - RehydratedComponents rehydratedComponents = - RehydratedComponents.forComponents( - RunnerApi.Components.newBuilder() - .putAllCoders(context.getCoders()) - .putAllPcollections(context.getPCollections()) - .putAllWindowingStrategies(context.getWindowingStrategies()) - .build()) - .withPipeline(Pipeline.create()); - - Coder inputCoder = (Coder) rehydratedComponents.getCoder(inputCoderId); - OutputSampler outputSampler = - dataSampler.sampleOutput( - context.getProcessBundleDescriptorId(), inputPCollectionId, inputCoder); - - // Add the runner to consume the PCollection as an input. - context.addPCollectionConsumer( - inputPCollectionId, - (FnDataReceiver>) - input -> outputSampler.sample(input.getValue())); - return runner; - } - } -} diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index bfa301018d5b..494af338b9db 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -32,29 +32,99 @@ * @param the element type of the PCollection. */ public class OutputSampler { - private final Coder coder; - private static final Logger LOG = LoggerFactory.getLogger(OutputSampler.class); + public static class SampleState { + // Temporarily holds elements until the SDK receives a sample data request. + private final List buffer; + + // Maximum number of elements in buffer. + private final int maxElements; + + // Sampling rate. + private final int sampleEveryN; + + // Total number of samples taken. + private long numSamples = 0; + + // Index into the buffer of where to overwrite samples. + private int resampleIndex = 0; + + private final Coder coder; + + public SampleState(Coder coder, int maxElements, int sampleEveryN) { + this.coder = coder; + this.maxElements = maxElements; + this.sampleEveryN = sampleEveryN; + this.buffer = new ArrayList<>(this.maxElements); + } - // Temporarily holds elements until the SDK receives a sample data request. - private final List buffer; + /** + * Samples every 1000th element or if it is part of the first 10 in the (local) PCollection. + * + * @param element the element to sample. + */ + public synchronized void sample(T element) { + // Only sample the first 10 elements then after every `sampleEveryN`th element. + numSamples += 1; + if (numSamples > 10 && numSamples % sampleEveryN != 0) { + return; + } + + // Fill buffer until maxElements. + if (buffer.size() < maxElements) { + buffer.add(element); + } else { + // Then rewrite sampled elements as a circular buffer. + buffer.set(resampleIndex, element); + resampleIndex = (resampleIndex + 1) % maxElements; + } + } - // Maximum number of elements in buffer. - private final int maxElements; + /** + * Clears samples at end of call. This is to help mitigate memory use. + * + * @return samples taken since last call. + */ + public List samples() { + List ret = new ArrayList<>(); - // Sampling rate. - private final int sampleEveryN; + // Serializing can take a lot of CPU time for larger or complex elements. Copy the array here + // so as to not slow down the main processing hot path. + List copiedBuffer; + synchronized (this) { + copiedBuffer = new ArrayList<>(buffer); + clear(); + } - // Total number of samples taken. - private long numSamples = 0; + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + for (T el : copiedBuffer) { + try { + // This is deprecated, but until this is fully removed, this specifically needs the nested + // context. This is because the SDK will need to decode the sampled elements with the + // ToStringFn. + coder.encode(el, stream, Coder.Context.NESTED); + ret.add(stream.toByteArray()); + } catch (Exception exception) { + LOG.warn("Could not encode element \"" + el + "\" to bytes: " + exception); + } finally { + stream.reset(); + } + } - // Index into the buffer of where to overwrite samples. - private int resampleIndex = 0; + return ret; + } - public OutputSampler(Coder coder, int maxElements, int sampleEveryN) { - this.coder = coder; - this.maxElements = maxElements; - this.sampleEveryN = sampleEveryN; - this.buffer = new ArrayList<>(this.maxElements); + private void clear() { + buffer.clear(); + resampleIndex = 0; + } + } + + private static final Logger LOG = LoggerFactory.getLogger(OutputSampler.class); + + private final SampleState sampleState; + + public OutputSampler(SampleState sampleState) { + this.sampleState = sampleState; } /** @@ -62,21 +132,8 @@ public OutputSampler(Coder coder, int maxElements, int sampleEveryN) { * * @param element the element to sample. */ - public synchronized void sample(T element) { - // Only sample the first 10 elements then after every `sampleEveryN`th element. - numSamples += 1; - if (numSamples > 10 && numSamples % sampleEveryN != 0) { - return; - } - - // Fill buffer until maxElements. - if (buffer.size() < maxElements) { - buffer.add(element); - } else { - // Then rewrite sampled elements as a circular buffer. - buffer.set(resampleIndex, element); - resampleIndex = (resampleIndex + 1) % maxElements; - } + public void sample(T element) { + this.sampleState.sample(element); } /** @@ -85,36 +142,6 @@ public synchronized void sample(T element) { * @return samples taken since last call. */ public List samples() { - List ret = new ArrayList<>(); - - // Serializing can take a lot of CPU time for larger or complex elements. Copy the array here - // so as to not slow down the main processing hot path. - List copiedBuffer; - synchronized (this) { - copiedBuffer = new ArrayList<>(buffer); - clear(); - } - - ByteArrayOutputStream stream = new ByteArrayOutputStream(); - for (T el : copiedBuffer) { - try { - // This is deprecated, but until this is fully removed, this specifically needs the nested - // context. This is because the SDK will need to decode the sampled elements with the - // ToStringFn. - coder.encode(el, stream, Coder.Context.NESTED); - ret.add(stream.toByteArray()); - } catch (Exception exception) { - LOG.warn("Could not encode element \"" + el + "\" to bytes: " + exception); - } finally { - stream.reset(); - } - } - - return ret; - } - - private void clear() { - buffer.clear(); - resampleIndex = 0; + return this.sampleState.samples(); } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java index 3cca0a172c16..57fe54156692 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java @@ -31,7 +31,6 @@ import org.apache.beam.fn.harness.control.BundleProgressReporter; import org.apache.beam.fn.harness.control.BundleSplitListener; import org.apache.beam.fn.harness.data.BeamFnDataClient; -import org.apache.beam.fn.harness.debug.DataSampler; import org.apache.beam.fn.harness.state.BeamFnStateClient; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.fnexecution.v1.BeamFnApi.BundleApplication; @@ -107,7 +106,6 @@ public CompletableFuture handle(StateRequest.Builder requestBuild () -> { throw new UnsupportedOperationException("Unexpected call during test."); }) - .processBundleDescriptorId("") .cacheTokensSupplier(() -> Collections.emptyList()) .bundleCacheSupplier(() -> Caches.noop()) .processWideCache(Caches.noop()) @@ -156,8 +154,6 @@ public interface Builder { Builder beamFnStateClient(BeamFnStateClient value); - Builder processBundleDescriptorId(String value); - Builder pTransformId(String value); Builder pTransform(RunnerApi.PTransform value); @@ -210,8 +206,6 @@ default Builder processBundleInstructionId(String value) { Builder timerApiServiceDescriptor(ApiServiceDescriptor value); - Builder dataSampler(DataSampler dataSampler); - PTransformRunnerFactoryTestContext build(); } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index 788cae7e2c46..7a107d1fa0fa 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -72,27 +72,12 @@ BeamFnApi.InstructionResponse getSamplesForPCollection( return dataSampler.handleDataSampleRequest(request).build(); } - BeamFnApi.InstructionResponse getSamplesForDescriptors( - DataSampler dataSampler, List descriptors) { + BeamFnApi.InstructionResponse getSamplesForPCollections( + DataSampler dataSampler, Iterable pcollections) { BeamFnApi.InstructionRequest request = BeamFnApi.InstructionRequest.newBuilder() .setSample( - BeamFnApi.SampleDataRequest.newBuilder() - .addAllProcessBundleDescriptorIds(descriptors) - .build()) - .build(); - return dataSampler.handleDataSampleRequest(request).build(); - } - - BeamFnApi.InstructionResponse getSamplesFor( - DataSampler dataSampler, String descriptor, String pcollection) { - BeamFnApi.InstructionRequest request = - BeamFnApi.InstructionRequest.newBuilder() - .setSample( - BeamFnApi.SampleDataRequest.newBuilder() - .addProcessBundleDescriptorIds(descriptor) - .addPcollectionIds(pcollection) - .build()) + BeamFnApi.SampleDataRequest.newBuilder().addAllPcollectionIds(pcollections).build()) .build(); return dataSampler.handleDataSampleRequest(request).build(); } @@ -126,7 +111,7 @@ public void testSingleOutput() throws Exception { DataSampler sampler = new DataSampler(); VarIntCoder coder = VarIntCoder.of(); - sampler.sampleOutput("descriptor-id", "pcollection-id", coder).sample(1); + sampler.sampleOutput("pcollection-id", coder).sample(1); BeamFnApi.InstructionResponse samples = getAllSamples(sampler); assertHasSamples(samples, "pcollection-id", Collections.singleton(encodeInt(1))); @@ -142,8 +127,8 @@ public void testMultipleOutputs() throws Exception { DataSampler sampler = new DataSampler(); VarIntCoder coder = VarIntCoder.of(); - sampler.sampleOutput("descriptor-id", "pcollection-id-1", coder).sample(1); - sampler.sampleOutput("descriptor-id", "pcollection-id-2", coder).sample(2); + sampler.sampleOutput("pcollection-id-1", coder).sample(1); + sampler.sampleOutput("pcollection-id-2", coder).sample(2); BeamFnApi.InstructionResponse samples = getAllSamples(sampler); assertHasSamples(samples, "pcollection-id-1", Collections.singleton(encodeInt(1))); @@ -156,12 +141,12 @@ public void testMultipleOutputs() throws Exception { * @throws Exception */ @Test - public void testMultipleDescriptors() throws Exception { + public void testMultipleSamePCollections() throws Exception { DataSampler sampler = new DataSampler(); VarIntCoder coder = VarIntCoder.of(); - sampler.sampleOutput("descriptor-id-1", "pcollection-id", coder).sample(1); - sampler.sampleOutput("descriptor-id-2", "pcollection-id", coder).sample(2); + sampler.sampleOutput("pcollection-id", coder).sample(1); + sampler.sampleOutput("pcollection-id", coder).sample(2); BeamFnApi.InstructionResponse samples = getAllSamples(sampler); assertHasSamples(samples, "pcollection-id", ImmutableList.of(encodeInt(1), encodeInt(2))); @@ -169,83 +154,43 @@ public void testMultipleDescriptors() throws Exception { void generateStringSamples(DataSampler sampler) { StringUtf8Coder coder = StringUtf8Coder.of(); - sampler.sampleOutput("a", "1", coder).sample("a1"); - sampler.sampleOutput("a", "2", coder).sample("a2"); - sampler.sampleOutput("b", "1", coder).sample("b1"); - sampler.sampleOutput("b", "2", coder).sample("b2"); + sampler.sampleOutput("a", coder).sample("a1"); + sampler.sampleOutput("a", coder).sample("a2"); + sampler.sampleOutput("b", coder).sample("b1"); + sampler.sampleOutput("b", coder).sample("b2"); + sampler.sampleOutput("c", coder).sample("c1"); + sampler.sampleOutput("c", coder).sample("c2"); } /** - * Test that samples can be filtered based on ProcessBundleDescriptor id. + * Test that samples can be filtered based on PCollection id. * * @throws Exception */ @Test - public void testFiltersSingleDescriptorId() throws Exception { + public void testFiltersSinglePCollectionId() throws Exception { DataSampler sampler = new DataSampler(10, 10); generateStringSamples(sampler); - BeamFnApi.InstructionResponse samples = - getSamplesForDescriptors(sampler, ImmutableList.of("a")); - assertHasSamples(samples, "1", Collections.singleton(encodeString("a1"))); - assertHasSamples(samples, "2", Collections.singleton(encodeString("a2"))); + BeamFnApi.InstructionResponse samples = getSamplesForPCollection(sampler, "a"); + assertHasSamples(samples, "a", ImmutableList.of(encodeString("a1"), encodeString("a2"))); } /** - * Test that samples are unioned based on ProcessBundleDescriptor id. + * Test that samples can be filtered both on PCollection and ProcessBundleDescriptor id. * * @throws Exception */ @Test - public void testFiltersMultipleDescriptorId() throws Exception { - DataSampler sampler = new DataSampler(10, 10); - generateStringSamples(sampler); + public void testFiltersMultiplePCollectionIds() throws Exception { + List pcollectionIds = ImmutableList.of("a", "c"); - BeamFnApi.InstructionResponse samples = - getSamplesForDescriptors(sampler, ImmutableList.of("a", "b")); - assertHasSamples(samples, "1", ImmutableList.of(encodeString("a1"), encodeString("b1"))); - assertHasSamples(samples, "2", ImmutableList.of(encodeString("a2"), encodeString("b2"))); - } - - /** - * Test that samples can be filtered based on PCollection id. - * - * @throws Exception - */ - @Test - public void testFiltersSinglePCollectionId() throws Exception { DataSampler sampler = new DataSampler(10, 10); generateStringSamples(sampler); - BeamFnApi.InstructionResponse samples = getSamplesForPCollection(sampler, "1"); - assertHasSamples(samples, "1", ImmutableList.of(encodeString("a1"), encodeString("b1"))); - } - - /** - * Test that samples can be filtered both on PCollection and ProcessBundleDescriptor id. - * - * @throws Exception - */ - @Test - public void testFiltersDescriptorAndPCollectionIds() throws Exception { - List descriptorIds = ImmutableList.of("a", "b"); - List pcollectionIds = ImmutableList.of("1", "2"); - - // Try all combinations for descriptor and PCollection ids. - for (String descriptorId : descriptorIds) { - for (String pcollectionId : pcollectionIds) { - DataSampler sampler = new DataSampler(10, 10); - generateStringSamples(sampler); - - BeamFnApi.InstructionResponse samples = getSamplesFor(sampler, descriptorId, pcollectionId); - System.out.print("Testing: " + descriptorId + pcollectionId + "..."); - assertThat(samples.getSample().getElementSamplesMap().size(), equalTo(1)); - assertHasSamples( - samples, - pcollectionId, - Collections.singleton(encodeString(descriptorId + pcollectionId))); - System.out.println("ok"); - } - } + BeamFnApi.InstructionResponse samples = getSamplesForPCollections(sampler, pcollectionIds); + assertThat(samples.getSample().getElementSamplesMap().size(), equalTo(2)); + assertHasSamples(samples, "a", ImmutableList.of(encodeString("a1"), encodeString("a2"))); + assertHasSamples(samples, "c", ImmutableList.of(encodeString("c1"), encodeString("c2"))); } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java deleted file mode 100644 index 7847665e9440..000000000000 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifierTest.java +++ /dev/null @@ -1,87 +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.fn.harness.debug; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasEntry; - -import org.apache.beam.model.fnexecution.v1.BeamFnApi; -import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -@RunWith(JUnit4.class) -public class DataSamplingDescriptorModifierTest { - - /** - * Tests that given a ProcessBundleDescriptor, the correct graph modification is done to create a - * DataSampling PTransform. - */ - @Test - public void testThatDataSamplingTransformIsMade() { - DataSamplingDescriptorModifier modifier = new DataSamplingDescriptorModifier(); - - final String pcollectionIdA = "pcollection-id-a"; - final String pcollectionIdB = "pcollection-id-b"; - final String coderIdA = "coder-id-a"; - final String coderIdB = "coder-id-b"; - BeamFnApi.ProcessBundleDescriptor descriptor = - BeamFnApi.ProcessBundleDescriptor.newBuilder() - .putPcollections( - pcollectionIdA, - RunnerApi.PCollection.newBuilder() - .setUniqueName(pcollectionIdA) - .setCoderId(coderIdA) - .build()) - .putPcollections( - pcollectionIdB, - RunnerApi.PCollection.newBuilder() - .setUniqueName(pcollectionIdB) - .setCoderId(coderIdB) - .build()) - .putCoders( - coderIdA, - RunnerApi.Coder.newBuilder() - .setSpec(RunnerApi.FunctionSpec.newBuilder().setUrn(DataSamplingFnRunner.URN)) - .build()) - .putCoders( - coderIdB, - RunnerApi.Coder.newBuilder() - .setSpec(RunnerApi.FunctionSpec.newBuilder().setUrn(DataSamplingFnRunner.URN)) - .build()) - .build(); - - final String ptransformIdA = "synthetic-data-sampling-transform-" + pcollectionIdA; - final String ptransformIdB = "synthetic-data-sampling-transform-" + pcollectionIdB; - - BeamFnApi.ProcessBundleDescriptor modified = modifier.modifyProcessBundleDescriptor(descriptor); - assertThat(modified.getTransformsCount(), equalTo(2)); - - RunnerApi.PTransform samplingTransformA = modified.getTransformsMap().get(ptransformIdA); - assertThat(samplingTransformA.getUniqueName(), equalTo(ptransformIdA)); - assertThat(samplingTransformA.getSpec().getUrn(), equalTo(DataSamplingFnRunner.URN)); - assertThat(samplingTransformA.getInputsMap(), hasEntry("main", pcollectionIdA)); - - RunnerApi.PTransform samplingTransformB = modified.getTransformsMap().get(ptransformIdB); - assertThat(samplingTransformB.getUniqueName(), equalTo(ptransformIdB)); - assertThat(samplingTransformB.getSpec().getUrn(), equalTo(DataSamplingFnRunner.URN)); - assertThat(samplingTransformB.getInputsMap(), hasEntry("main", pcollectionIdB)); - } -} diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java deleted file mode 100644 index 501816ae4c23..000000000000 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplingFnRunnerTest.java +++ /dev/null @@ -1,121 +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.fn.harness.debug; - -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; -import static org.junit.Assert.assertTrue; - -import java.io.ByteArrayOutputStream; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import org.apache.beam.fn.harness.PTransformRunnerFactoryTestContext; -import org.apache.beam.model.fnexecution.v1.BeamFnApi; -import org.apache.beam.model.pipeline.v1.RunnerApi; -import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; -import org.apache.beam.runners.core.construction.CoderTranslation; -import org.apache.beam.runners.core.construction.RehydratedComponents; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -@RunWith(JUnit4.class) -public class DataSamplingFnRunnerTest { - - @Test - public void testCreatingAndProcessingWithSampling() throws Exception { - // Create the DataSampling PTransform. - String pTransformId = "pTransformId"; - - RunnerApi.FunctionSpec functionSpec = - RunnerApi.FunctionSpec.newBuilder().setUrn(DataSamplingFnRunner.URN).build(); - RunnerApi.PTransform pTransform = - RunnerApi.PTransform.newBuilder() - .setSpec(functionSpec) - .putInputs("input", "inputTarget") - .build(); - - // Populate fake input PCollections. - Map pCollectionMap = new HashMap<>(); - pCollectionMap.put( - "inputTarget", - RunnerApi.PCollection.newBuilder() - .setUniqueName("inputTarget") - .setCoderId("coder-id") - .build()); - - // Populate the PTransform context that includes the DataSampler. - DataSampler dataSampler = new DataSampler(); - RunnerApi.Coder coder = CoderTranslation.toProto(StringUtf8Coder.of()).getCoder(); - PTransformRunnerFactoryTestContext context = - PTransformRunnerFactoryTestContext.builder(pTransformId, pTransform) - .processBundleDescriptorId("descriptor-id") - .processBundleInstructionId("instruction-id") - .pCollections(pCollectionMap) - .coders(Collections.singletonMap("coder-id", coder)) - .dataSampler(dataSampler) - .build(); - - // Create the runner which samples the input PCollection. - new DataSamplingFnRunner.Factory<>().createRunnerForPTransform(context); - assertThat(context.getPCollectionConsumers().keySet(), contains("inputTarget")); - - // Send in a test value that should be sampled. - context.getPCollectionConsumer("inputTarget").accept(valueInGlobalWindow("Hello, World!")); - - // Rehydrate the given utf-8 string coder. - RehydratedComponents rehydratedComponents = - RehydratedComponents.forComponents( - RunnerApi.Components.newBuilder() - .putAllCoders(context.getCoders()) - .putAllPcollections(context.getPCollections()) - .putAllWindowingStrategies(context.getWindowingStrategies()) - .build()) - .withPipeline(Pipeline.create()); - Coder rehydratedCoder = (Coder) rehydratedComponents.getCoder("coder-id"); - - BeamFnApi.InstructionRequest request = - BeamFnApi.InstructionRequest.newBuilder() - .setSample(BeamFnApi.SampleDataRequest.newBuilder().build()) - .build(); - BeamFnApi.InstructionResponse samples = dataSampler.handleDataSampleRequest(request).build(); - assertThat(samples.getSample().getElementSamplesMap().keySet(), contains("inputTarget")); - - // Ensure that the value was sampled. - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - rehydratedCoder.encode("Hello, World!", outputStream); - byte[] encodedValue = outputStream.toByteArray(); - - assertTrue( - samples - .getSample() - .getElementSamplesMap() - .get("inputTarget") - .getElementsList() - .contains( - BeamFnApi.SampledElement.newBuilder() - .setElement(ByteString.copyFrom(encodedValue)) - .build())); - } -} diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java index f3e0a5a06211..fe10a55ec525 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -46,7 +46,8 @@ public byte[] encodeInt(Integer i) throws IOException { @Test public void testSamplesFirstN() throws Exception { VarIntCoder coder = VarIntCoder.of(); - OutputSampler outputSampler = new OutputSampler<>(coder, 10, 10); + OutputSampler.SampleState sampleState = new OutputSampler.SampleState<>(coder, 10, 10); + OutputSampler outputSampler = new OutputSampler<>(sampleState); // Purposely go over maxSamples and sampleEveryN. This helps to increase confidence. for (int i = 0; i < 15; ++i) { @@ -71,7 +72,8 @@ public void testSamplesFirstN() throws Exception { @Test public void testActsLikeCircularBuffer() throws Exception { VarIntCoder coder = VarIntCoder.of(); - OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20); + OutputSampler.SampleState sampleState = new OutputSampler.SampleState<>(coder, 5, 20); + OutputSampler outputSampler = new OutputSampler<>(sampleState); for (int i = 0; i < 100; ++i) { outputSampler.sample(i); From 553fc5e0d98a7c82da75fad14ef8c6fc54d2cfa0 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Fri, 10 Feb 2023 00:28:17 -0800 Subject: [PATCH 20/42] remove redundant SamplerState obj --- .../org/apache/beam/fn/harness/FnHarness.java | 14 +- .../beam/fn/harness/debug/DataSampler.java | 9 +- .../beam/fn/harness/debug/OutputSampler.java | 152 ++++++++---------- .../fn/harness/debug/OutputSamplerTest.java | 6 +- 4 files changed, 76 insertions(+), 105 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index fcc6f7be987f..5a1f350b293e 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -92,7 +92,7 @@ public class FnHarness { private static final String STATUS_API_SERVICE_DESCRIPTOR = "STATUS_API_SERVICE_DESCRIPTOR"; private static final String PIPELINE_OPTIONS = "PIPELINE_OPTIONS"; private static final String RUNNER_CAPABILITIES = "RUNNER_CAPABILITIES"; - private static final String ENABLE_DATA_SAMPLING_EXPERIMENT = "enable_data_sampling"; +// private static final String ENABLE_DATA_SAMPLING_EXPERIMENT = "enable_data_sampling"; private static final Logger LOG = LoggerFactory.getLogger(FnHarness.class); private static DataSampler dataSampler = new DataSampler(); @@ -254,11 +254,11 @@ public static void main( FinalizeBundleHandler finalizeBundleHandler = new FinalizeBundleHandler(executorService); // Create the sampler, if the experiment is enabled. - Optional> experimentList = - Optional.ofNullable(options.as(ExperimentalOptions.class).getExperiments()); - boolean shouldSample = - experimentList.isPresent() - && experimentList.get().contains(ENABLE_DATA_SAMPLING_EXPERIMENT); +// Optional> experimentList = +// Optional.ofNullable(options.as(ExperimentalOptions.class).getExperiments()); +// boolean shouldSample = +// experimentList.isPresent() +// && experimentList.get().contains(ENABLE_DATA_SAMPLING_EXPERIMENT); // Retrieves the ProcessBundleDescriptor from cache. Requests the PBD from the Runner if it // doesn't exist. Additionally, runs any graph modifications. @@ -294,7 +294,7 @@ private BeamFnApi.ProcessBundleDescriptor loadDescriptor(String id) { metricsShortIds, executionStateSampler, processWideCache, - shouldSample ? dataSampler : null); + dataSampler); logging.setProcessBundleHandler(processBundleHandler); BeamFnStatusClient beamFnStatusClient = null; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index 7ce1df0dce2f..bc99e5e70e5a 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -59,10 +59,9 @@ public DataSampler(int maxSamples, int sampleEveryN) { // Sampling rate. private final int sampleEveryN; - // The fully-qualified type is: Map[PCollectionId, OutputSampler.SamplerState]. In order to sample + // The fully-qualified type is: Map[PCollectionId, OutputSampler]. In order to sample // on a PCollection-basis and not per-bundle, this keeps track of shared samples between states. - private final Map> outputSamplers = - new ConcurrentHashMap<>(); + private final Map> outputSamplers = new ConcurrentHashMap<>(); /** * Creates and returns a class to sample the given PCollection in the given @@ -76,8 +75,8 @@ public DataSampler(int maxSamples, int sampleEveryN) { */ public OutputSampler sampleOutput(String pcollectionId, Coder coder) { outputSamplers.putIfAbsent( - pcollectionId, new OutputSampler.SampleState<>(coder, this.maxSamples, this.sampleEveryN)); - return new OutputSampler<>((OutputSampler.SampleState) outputSamplers.get(pcollectionId)); + pcollectionId, new OutputSampler<>(coder, this.maxSamples, this.sampleEveryN)); + return (OutputSampler) outputSamplers.get(pcollectionId); } /** diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index 494af338b9db..91572ef3d408 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -32,99 +32,30 @@ * @param the element type of the PCollection. */ public class OutputSampler { - public static class SampleState { - // Temporarily holds elements until the SDK receives a sample data request. - private final List buffer; - - // Maximum number of elements in buffer. - private final int maxElements; - - // Sampling rate. - private final int sampleEveryN; + private static final Logger LOG = LoggerFactory.getLogger(OutputSampler.class); - // Total number of samples taken. - private long numSamples = 0; + // Temporarily holds elements until the SDK receives a sample data request. + private final List buffer; - // Index into the buffer of where to overwrite samples. - private int resampleIndex = 0; + // Maximum number of elements in buffer. + private final int maxElements; - private final Coder coder; + // Sampling rate. + private final int sampleEveryN; - public SampleState(Coder coder, int maxElements, int sampleEveryN) { - this.coder = coder; - this.maxElements = maxElements; - this.sampleEveryN = sampleEveryN; - this.buffer = new ArrayList<>(this.maxElements); - } + // Total number of samples taken. + private long numSamples = 0; - /** - * Samples every 1000th element or if it is part of the first 10 in the (local) PCollection. - * - * @param element the element to sample. - */ - public synchronized void sample(T element) { - // Only sample the first 10 elements then after every `sampleEveryN`th element. - numSamples += 1; - if (numSamples > 10 && numSamples % sampleEveryN != 0) { - return; - } + // Index into the buffer of where to overwrite samples. + private int resampleIndex = 0; - // Fill buffer until maxElements. - if (buffer.size() < maxElements) { - buffer.add(element); - } else { - // Then rewrite sampled elements as a circular buffer. - buffer.set(resampleIndex, element); - resampleIndex = (resampleIndex + 1) % maxElements; - } - } + private final Coder coder; - /** - * Clears samples at end of call. This is to help mitigate memory use. - * - * @return samples taken since last call. - */ - public List samples() { - List ret = new ArrayList<>(); - - // Serializing can take a lot of CPU time for larger or complex elements. Copy the array here - // so as to not slow down the main processing hot path. - List copiedBuffer; - synchronized (this) { - copiedBuffer = new ArrayList<>(buffer); - clear(); - } - - ByteArrayOutputStream stream = new ByteArrayOutputStream(); - for (T el : copiedBuffer) { - try { - // This is deprecated, but until this is fully removed, this specifically needs the nested - // context. This is because the SDK will need to decode the sampled elements with the - // ToStringFn. - coder.encode(el, stream, Coder.Context.NESTED); - ret.add(stream.toByteArray()); - } catch (Exception exception) { - LOG.warn("Could not encode element \"" + el + "\" to bytes: " + exception); - } finally { - stream.reset(); - } - } - - return ret; - } - - private void clear() { - buffer.clear(); - resampleIndex = 0; - } - } - - private static final Logger LOG = LoggerFactory.getLogger(OutputSampler.class); - - private final SampleState sampleState; - - public OutputSampler(SampleState sampleState) { - this.sampleState = sampleState; + public OutputSampler(Coder coder, int maxElements, int sampleEveryN) { + this.coder = coder; + this.maxElements = maxElements; + this.sampleEveryN = sampleEveryN; + this.buffer = new ArrayList<>(this.maxElements); } /** @@ -132,8 +63,21 @@ public OutputSampler(SampleState sampleState) { * * @param element the element to sample. */ - public void sample(T element) { - this.sampleState.sample(element); + public synchronized void sample(T element) { + // Only sample the first 10 elements then after every `sampleEveryN`th element. + numSamples += 1; + if (numSamples > 10 && numSamples % sampleEveryN != 0) { + return; + } + + // Fill buffer until maxElements. + if (buffer.size() < maxElements) { + buffer.add(element); + } else { + // Then rewrite sampled elements as a circular buffer. + buffer.set(resampleIndex, element); + resampleIndex = (resampleIndex + 1) % maxElements; + } } /** @@ -142,6 +86,36 @@ public void sample(T element) { * @return samples taken since last call. */ public List samples() { - return this.sampleState.samples(); + List ret = new ArrayList<>(); + + // Serializing can take a lot of CPU time for larger or complex elements. Copy the array here + // so as to not slow down the main processing hot path. + List copiedBuffer; + synchronized (this) { + copiedBuffer = new ArrayList<>(buffer); + clear(); + } + + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + for (T el : copiedBuffer) { + try { + // This is deprecated, but until this is fully removed, this specifically needs the nested + // context. This is because the SDK will need to decode the sampled elements with the + // ToStringFn. + coder.encode(el, stream, Coder.Context.NESTED); + ret.add(stream.toByteArray()); + } catch (Exception exception) { + LOG.warn("Could not encode element \"" + el + "\" to bytes: " + exception); + } finally { + stream.reset(); + } + } + + return ret; + } + + private void clear() { + buffer.clear(); + resampleIndex = 0; } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java index fe10a55ec525..f3e0a5a06211 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -46,8 +46,7 @@ public byte[] encodeInt(Integer i) throws IOException { @Test public void testSamplesFirstN() throws Exception { VarIntCoder coder = VarIntCoder.of(); - OutputSampler.SampleState sampleState = new OutputSampler.SampleState<>(coder, 10, 10); - OutputSampler outputSampler = new OutputSampler<>(sampleState); + OutputSampler outputSampler = new OutputSampler<>(coder, 10, 10); // Purposely go over maxSamples and sampleEveryN. This helps to increase confidence. for (int i = 0; i < 15; ++i) { @@ -72,8 +71,7 @@ public void testSamplesFirstN() throws Exception { @Test public void testActsLikeCircularBuffer() throws Exception { VarIntCoder coder = VarIntCoder.of(); - OutputSampler.SampleState sampleState = new OutputSampler.SampleState<>(coder, 5, 20); - OutputSampler outputSampler = new OutputSampler<>(sampleState); + OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20); for (int i = 0; i < 100; ++i) { outputSampler.sample(i); From e260eb4f1e84ca0b310e301998c71d14313ab224 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Fri, 10 Feb 2023 00:28:59 -0800 Subject: [PATCH 21/42] spotless --- .../java/org/apache/beam/fn/harness/FnHarness.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index 5a1f350b293e..24ac46a7558b 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -19,8 +19,6 @@ import java.util.Collections; import java.util.EnumMap; -import java.util.List; -import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.function.Function; @@ -92,7 +90,7 @@ public class FnHarness { private static final String STATUS_API_SERVICE_DESCRIPTOR = "STATUS_API_SERVICE_DESCRIPTOR"; private static final String PIPELINE_OPTIONS = "PIPELINE_OPTIONS"; private static final String RUNNER_CAPABILITIES = "RUNNER_CAPABILITIES"; -// private static final String ENABLE_DATA_SAMPLING_EXPERIMENT = "enable_data_sampling"; + // private static final String ENABLE_DATA_SAMPLING_EXPERIMENT = "enable_data_sampling"; private static final Logger LOG = LoggerFactory.getLogger(FnHarness.class); private static DataSampler dataSampler = new DataSampler(); @@ -254,11 +252,11 @@ public static void main( FinalizeBundleHandler finalizeBundleHandler = new FinalizeBundleHandler(executorService); // Create the sampler, if the experiment is enabled. -// Optional> experimentList = -// Optional.ofNullable(options.as(ExperimentalOptions.class).getExperiments()); -// boolean shouldSample = -// experimentList.isPresent() -// && experimentList.get().contains(ENABLE_DATA_SAMPLING_EXPERIMENT); + // Optional> experimentList = + // Optional.ofNullable(options.as(ExperimentalOptions.class).getExperiments()); + // boolean shouldSample = + // experimentList.isPresent() + // && experimentList.get().contains(ENABLE_DATA_SAMPLING_EXPERIMENT); // Retrieves the ProcessBundleDescriptor from cache. Requests the PBD from the Runner if it // doesn't exist. Additionally, runs any graph modifications. From 4f29308031dd7e9d40ad5a2cb8a51c8961d2449e Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Fri, 10 Feb 2023 13:09:44 -0800 Subject: [PATCH 22/42] replace mutex with atomics in output sampler to reduce contention --- .../org/apache/beam/fn/harness/FnHarness.java | 16 +++++----- .../beam/fn/harness/debug/OutputSampler.java | 30 ++++++++++++------- 2 files changed, 28 insertions(+), 18 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index 24ac46a7558b..4f9e9948f28c 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -19,6 +19,8 @@ import java.util.Collections; import java.util.EnumMap; +import java.util.Optional; +import java.util.List; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.function.Function; @@ -90,7 +92,7 @@ public class FnHarness { private static final String STATUS_API_SERVICE_DESCRIPTOR = "STATUS_API_SERVICE_DESCRIPTOR"; private static final String PIPELINE_OPTIONS = "PIPELINE_OPTIONS"; private static final String RUNNER_CAPABILITIES = "RUNNER_CAPABILITIES"; - // private static final String ENABLE_DATA_SAMPLING_EXPERIMENT = "enable_data_sampling"; + private static final String ENABLE_DATA_SAMPLING_EXPERIMENT = "enable_data_sampling"; private static final Logger LOG = LoggerFactory.getLogger(FnHarness.class); private static DataSampler dataSampler = new DataSampler(); @@ -252,11 +254,11 @@ public static void main( FinalizeBundleHandler finalizeBundleHandler = new FinalizeBundleHandler(executorService); // Create the sampler, if the experiment is enabled. - // Optional> experimentList = - // Optional.ofNullable(options.as(ExperimentalOptions.class).getExperiments()); - // boolean shouldSample = - // experimentList.isPresent() - // && experimentList.get().contains(ENABLE_DATA_SAMPLING_EXPERIMENT); + Optional> experimentList = + Optional.ofNullable(options.as(ExperimentalOptions.class).getExperiments()); + boolean shouldSample = + experimentList.isPresent() + && experimentList.get().contains(ENABLE_DATA_SAMPLING_EXPERIMENT); // Retrieves the ProcessBundleDescriptor from cache. Requests the PBD from the Runner if it // doesn't exist. Additionally, runs any graph modifications. @@ -292,7 +294,7 @@ private BeamFnApi.ProcessBundleDescriptor loadDescriptor(String id) { metricsShortIds, executionStateSampler, processWideCache, - dataSampler); + shouldSample ? dataSampler : null); logging.setProcessBundleHandler(processBundleHandler); BeamFnStatusClient beamFnStatusClient = null; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index 91572ef3d408..cd9dd1fcd2c2 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -20,6 +20,7 @@ import java.io.ByteArrayOutputStream; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.atomic.AtomicLong; import org.apache.beam.sdk.coders.Coder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,7 +45,7 @@ public class OutputSampler { private final int sampleEveryN; // Total number of samples taken. - private long numSamples = 0; + private final AtomicLong numSamples = new AtomicLong(); // Index into the buffer of where to overwrite samples. private int resampleIndex = 0; @@ -63,20 +64,27 @@ public OutputSampler(Coder coder, int maxElements, int sampleEveryN) { * * @param element the element to sample. */ - public synchronized void sample(T element) { + public void sample(T element) { // Only sample the first 10 elements then after every `sampleEveryN`th element. - numSamples += 1; - if (numSamples > 10 && numSamples % sampleEveryN != 0) { + long samples = numSamples.get(); + + // This has eventual consistency. If there are many threads lazy setting, this will be set to + // the slowest thread accessing the atomic. But over time, it will still increase. This is ok + // because this is a debugging feature and doesn't need strict atomics. + numSamples.lazySet(samples + 1); + if (samples > 10 && samples % sampleEveryN != 0) { return; } - // Fill buffer until maxElements. - if (buffer.size() < maxElements) { - buffer.add(element); - } else { - // Then rewrite sampled elements as a circular buffer. - buffer.set(resampleIndex, element); - resampleIndex = (resampleIndex + 1) % maxElements; + synchronized(this) { + // Fill buffer until maxElements. + if (buffer.size() < maxElements) { + buffer.add(element); + } else { + // Then rewrite sampled elements as a circular buffer. + buffer.set(resampleIndex, element); + resampleIndex = (resampleIndex + 1) % maxElements; + } } } From 8cbc6c83db7b26a916dbe18ad8edbe9d48f7f1f6 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Fri, 10 Feb 2023 14:32:00 -0800 Subject: [PATCH 23/42] spotless and fix OutputSamplerTest --- .../src/main/java/org/apache/beam/fn/harness/FnHarness.java | 2 +- .../org/apache/beam/fn/harness/debug/OutputSampler.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index 4f9e9948f28c..fcc6f7be987f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -19,8 +19,8 @@ import java.util.Collections; import java.util.EnumMap; -import java.util.Optional; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.function.Function; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index cd9dd1fcd2c2..e5e1c82fd3d0 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -66,17 +66,17 @@ public OutputSampler(Coder coder, int maxElements, int sampleEveryN) { */ public void sample(T element) { // Only sample the first 10 elements then after every `sampleEveryN`th element. - long samples = numSamples.get(); + long samples = numSamples.get() + 1; // This has eventual consistency. If there are many threads lazy setting, this will be set to // the slowest thread accessing the atomic. But over time, it will still increase. This is ok // because this is a debugging feature and doesn't need strict atomics. - numSamples.lazySet(samples + 1); + numSamples.lazySet(samples); if (samples > 10 && samples % sampleEveryN != 0) { return; } - synchronized(this) { + synchronized (this) { // Fill buffer until maxElements. if (buffer.size() < maxElements) { buffer.add(element); From f67234f92388fa70832f03996fd5f99b05b494c4 Mon Sep 17 00:00:00 2001 From: Sam sam Date: Mon, 13 Feb 2023 10:46:24 -0800 Subject: [PATCH 24/42] Update sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java Co-authored-by: Lukasz Cwik --- .../beam/fn/harness/data/PCollectionConsumerRegistry.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java index 39cd422c91cc..63a89a509a58 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java @@ -247,7 +247,7 @@ private class MetricTrackingFnDataReceiver implements FnDataReceiver sampledByteSizeDistribution; private final Coder coder; - private OutputSampler outputSampler; + private final @Nullable OutputSampler outputSampler; public MetricTrackingFnDataReceiver( String pCollectionId, Coder coder, ConsumerAndMetadata consumerAndMetadata, From 6815330df118d8fa5eef296d97b576f042caa823 Mon Sep 17 00:00:00 2001 From: Sam sam Date: Mon, 13 Feb 2023 10:46:32 -0800 Subject: [PATCH 25/42] Update sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java Co-authored-by: Lukasz Cwik --- .../beam/fn/harness/data/PCollectionConsumerRegistry.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java index 63a89a509a58..1aa9056e3606 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java @@ -330,7 +330,7 @@ private class MultiplexingMetricTrackingFnDataReceiver private final BundleCounter elementCountCounter; private final SampleByteSizeDistribution sampledByteSizeDistribution; private final Coder coder; - private @Nullable OutputSampler outputSampler = null; + private final @Nullable OutputSampler outputSampler; public MultiplexingMetricTrackingFnDataReceiver( String pCollectionId, Coder coder, List consumerAndMetadatas, From 484872529171336a8c4441f61c1e7d14f5fba40d Mon Sep 17 00:00:00 2001 From: Sam sam Date: Mon, 13 Feb 2023 10:47:06 -0800 Subject: [PATCH 26/42] Update sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java Co-authored-by: Lukasz Cwik --- .../org/apache/beam/fn/harness/debug/OutputSampler.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index e5e1c82fd3d0..1fe79b85bcdd 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -101,7 +101,8 @@ public List samples() { List copiedBuffer; synchronized (this) { copiedBuffer = new ArrayList<>(buffer); - clear(); + buffer.clear(); + resampleIndex = 0; } ByteArrayOutputStream stream = new ByteArrayOutputStream(); @@ -121,9 +122,4 @@ public List samples() { return ret; } - - private void clear() { - buffer.clear(); - resampleIndex = 0; - } } From 6c16576baafc21395f7c04c895d4b227c9f8ce51 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Mon, 13 Feb 2023 11:34:50 -0800 Subject: [PATCH 27/42] always init outputsampler --- .../beam/fn/harness/data/PCollectionConsumerRegistry.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java index 1aa9056e3606..a8ba5d6c5b33 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java @@ -287,7 +287,9 @@ public MetricTrackingFnDataReceiver( bundleProgressReporterRegistrar.register(sampledByteSizeUnderlyingDistribution); this.coder = coder; - if (dataSampler != null) { + if (dataSampler == null) { + this.outputSampler = null; + } else { this.outputSampler = dataSampler.sampleOutput(pCollectionId, coder); } } @@ -369,7 +371,9 @@ public MultiplexingMetricTrackingFnDataReceiver( bundleProgressReporterRegistrar.register(sampledByteSizeUnderlyingDistribution); this.coder = coder; - if (dataSampler != null) { + if (dataSampler == null) { + this.outputSampler = null; + } else { this.outputSampler = dataSampler.sampleOutput(pCollectionId, coder); } } From 822587d50f52afdf667af6a981b0d5ce43872ad2 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Mon, 13 Feb 2023 14:49:30 -0800 Subject: [PATCH 28/42] add final to DataSampler in FnHarness --- .../src/main/java/org/apache/beam/fn/harness/FnHarness.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index fcc6f7be987f..bae290b1cb6c 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -94,7 +94,7 @@ public class FnHarness { private static final String RUNNER_CAPABILITIES = "RUNNER_CAPABILITIES"; private static final String ENABLE_DATA_SAMPLING_EXPERIMENT = "enable_data_sampling"; private static final Logger LOG = LoggerFactory.getLogger(FnHarness.class); - private static DataSampler dataSampler = new DataSampler(); + private static final DataSampler dataSampler = new DataSampler(); private static Endpoints.ApiServiceDescriptor getApiServiceDescriptor(String descriptor) throws TextFormat.ParseException { From fe9ab2a3ca79292193970f0d00ff8db24b7c04fa Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Mon, 13 Feb 2023 15:45:25 -0800 Subject: [PATCH 29/42] spotless apply --- .../org/apache/beam/fn/harness/FnHarness.java | 3 ++- .../harness/control/ProcessBundleHandler.java | 6 ++++- .../data/PCollectionConsumerRegistry.java | 22 ++++++++++--------- .../beam/fn/harness/debug/DataSampler.java | 4 ++-- 4 files changed, 21 insertions(+), 14 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index bae290b1cb6c..ec75c43e050b 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -342,7 +342,8 @@ private BeamFnApi.ProcessBundleDescriptor loadDescriptor(String id) { handlers.put( InstructionRequest.RequestCase.HARNESS_MONITORING_INFOS, processWideHandler::harnessMonitoringInfos); - handlers.put(InstructionRequest.RequestCase.SAMPLE, dataSampler::handleDataSampleRequest); + handlers.put( + InstructionRequest.RequestCase.SAMPLE_DATA, dataSampler::handleDataSampleRequest); JvmInitializers.runBeforeProcessing(options); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index 6bbb477ae005..2ec1411bd3d5 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -778,7 +778,11 @@ private BundleProcessor createBundleProcessor( bundleProgressReporterAndRegistrar.register(stateTracker); PCollectionConsumerRegistry pCollectionConsumerRegistry = new PCollectionConsumerRegistry( - stateTracker, shortIds, bundleProgressReporterAndRegistrar, bundleDescriptor, dataSampler); + stateTracker, + shortIds, + bundleProgressReporterAndRegistrar, + bundleDescriptor, + dataSampler); HashSet processedPTransformIds = new HashSet<>(); PTransformFunctionRegistry startFunctionRegistry = diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java index a8ba5d6c5b33..282096da1d53 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java @@ -97,12 +97,7 @@ public PCollectionConsumerRegistry( ShortIdMap shortIdMap, BundleProgressReporter.Registrar bundleProgressReporterRegistrar, ProcessBundleDescriptor processBundleDescriptor) { - this( - stateTracker, - shortIdMap, - bundleProgressReporterRegistrar, - processBundleDescriptor, - null); + this(stateTracker, shortIdMap, bundleProgressReporterRegistrar, processBundleDescriptor, null); } public PCollectionConsumerRegistry( @@ -220,7 +215,8 @@ public FnDataReceiver> getMultiplexingConsumer(String pCollecti if (consumerAndMetadatas.size() == 1) { ConsumerAndMetadata consumerAndMetadata = consumerAndMetadatas.get(0); if (consumerAndMetadata.getConsumer() instanceof HandlesSplits) { - return new SplittingMetricTrackingFnDataReceiver(pcId, coder, consumerAndMetadata, dataSampler); + return new SplittingMetricTrackingFnDataReceiver( + pcId, coder, consumerAndMetadata, dataSampler); } return new MetricTrackingFnDataReceiver(pcId, coder, consumerAndMetadata, dataSampler); } else { @@ -250,7 +246,9 @@ private class MetricTrackingFnDataReceiver implements FnDataReceiver outputSampler; public MetricTrackingFnDataReceiver( - String pCollectionId, Coder coder, ConsumerAndMetadata consumerAndMetadata, + String pCollectionId, + Coder coder, + ConsumerAndMetadata consumerAndMetadata, @Nullable DataSampler dataSampler) { this.delegate = consumerAndMetadata.getConsumer(); this.executionState = consumerAndMetadata.getExecutionState(); @@ -335,7 +333,9 @@ private class MultiplexingMetricTrackingFnDataReceiver private final @Nullable OutputSampler outputSampler; public MultiplexingMetricTrackingFnDataReceiver( - String pCollectionId, Coder coder, List consumerAndMetadatas, + String pCollectionId, + Coder coder, + List consumerAndMetadatas, @Nullable DataSampler dataSampler) { this.consumerAndMetadatas = consumerAndMetadatas; @@ -418,7 +418,9 @@ private class SplittingMetricTrackingFnDataReceiver extends MetricTrackingFnD private final HandlesSplits delegate; public SplittingMetricTrackingFnDataReceiver( - String pCollection, Coder coder, ConsumerAndMetadata consumerAndMetadata, + String pCollection, + Coder coder, + ConsumerAndMetadata consumerAndMetadata, @Nullable DataSampler dataSampler) { super(pCollection, coder, consumerAndMetadata, dataSampler); this.delegate = (HandlesSplits) consumerAndMetadata.getConsumer(); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index bc99e5e70e5a..aa0638997abb 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -88,7 +88,7 @@ public OutputSampler sampleOutput(String pcollectionId, Coder coder) { */ public BeamFnApi.InstructionResponse.Builder handleDataSampleRequest( BeamFnApi.InstructionRequest request) { - BeamFnApi.SampleDataRequest sampleDataRequest = request.getSample(); + BeamFnApi.SampleDataRequest sampleDataRequest = request.getSampleData(); Map> responseSamples = samplesFor(sampleDataRequest.getPcollectionIdsList()); @@ -103,7 +103,7 @@ public BeamFnApi.InstructionResponse.Builder handleDataSampleRequest( response.putElementSamples(pcollectionId, elementList.build()); } - return BeamFnApi.InstructionResponse.newBuilder().setSample(response); + return BeamFnApi.InstructionResponse.newBuilder().setSampleData(response); } /** From 07d37ea19f154a9a3bcf7d09455b65e3fba6fbea Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Mon, 13 Feb 2023 15:48:57 -0800 Subject: [PATCH 30/42] update from proto names --- .../beam/fn/harness/control/ProcessBundleHandler.java | 1 - .../apache/beam/fn/harness/debug/DataSamplerTest.java | 10 +++++----- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index 2ec1411bd3d5..348b9a761fdf 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -165,7 +165,6 @@ public class ProcessBundleHandler { private final Cache processWideCache; @VisibleForTesting final BundleProcessorCache bundleProcessorCache; private final Set runnerCapabilities; - private final @Nullable DataSampler dataSampler; public ProcessBundleHandler( diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index 7a107d1fa0fa..77c0ad8441bb 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -57,7 +57,7 @@ byte[] encodeString(String s) throws IOException { BeamFnApi.InstructionResponse getAllSamples(DataSampler dataSampler) { BeamFnApi.InstructionRequest request = BeamFnApi.InstructionRequest.newBuilder() - .setSample(BeamFnApi.SampleDataRequest.newBuilder().build()) + .setSampleData(BeamFnApi.SampleDataRequest.newBuilder().build()) .build(); return dataSampler.handleDataSampleRequest(request).build(); } @@ -66,7 +66,7 @@ BeamFnApi.InstructionResponse getSamplesForPCollection( DataSampler dataSampler, String pcollection) { BeamFnApi.InstructionRequest request = BeamFnApi.InstructionRequest.newBuilder() - .setSample( + .setSampleData( BeamFnApi.SampleDataRequest.newBuilder().addPcollectionIds(pcollection).build()) .build(); return dataSampler.handleDataSampleRequest(request).build(); @@ -76,7 +76,7 @@ BeamFnApi.InstructionResponse getSamplesForPCollections( DataSampler dataSampler, Iterable pcollections) { BeamFnApi.InstructionRequest request = BeamFnApi.InstructionRequest.newBuilder() - .setSample( + .setSampleData( BeamFnApi.SampleDataRequest.newBuilder().addAllPcollectionIds(pcollections).build()) .build(); return dataSampler.handleDataSampleRequest(request).build(); @@ -85,7 +85,7 @@ BeamFnApi.InstructionResponse getSamplesForPCollections( void assertHasSamples( BeamFnApi.InstructionResponse response, String pcollection, Iterable elements) { Map elementSamplesMap = - response.getSample().getElementSamplesMap(); + response.getSampleData().getElementSamplesMap(); assertFalse(elementSamplesMap.isEmpty()); @@ -189,7 +189,7 @@ public void testFiltersMultiplePCollectionIds() throws Exception { generateStringSamples(sampler); BeamFnApi.InstructionResponse samples = getSamplesForPCollections(sampler, pcollectionIds); - assertThat(samples.getSample().getElementSamplesMap().size(), equalTo(2)); + assertThat(samples.getSampleData().getElementSamplesMap().size(), equalTo(2)); assertHasSamples(samples, "a", ImmutableList.of(encodeString("a1"), encodeString("a2"))); assertHasSamples(samples, "c", ImmutableList.of(encodeString("c1"), encodeString("c2"))); } From 5e9c4b01af11b5567985efd93b541dec721c1e72 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 14 Feb 2023 14:55:27 -0800 Subject: [PATCH 31/42] spotless bugs --- .../java/org/apache/beam/fn/harness/debug/DataSampler.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index aa0638997abb..f9ca3ed2d48f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -94,9 +94,10 @@ public BeamFnApi.InstructionResponse.Builder handleDataSampleRequest( samplesFor(sampleDataRequest.getPcollectionIdsList()); BeamFnApi.SampleDataResponse.Builder response = BeamFnApi.SampleDataResponse.newBuilder(); - for (String pcollectionId : responseSamples.keySet()) { + for (Map.Entry> entry : responseSamples.entrySet()) { + String pcollectionId = entry.getKey(); ElementList.Builder elementList = ElementList.newBuilder(); - for (byte[] sample : responseSamples.get(pcollectionId)) { + for (byte[] sample : entry.getValue()) { elementList.addElements( SampledElement.newBuilder().setElement(ByteString.copyFrom(sample)).build()); } From f5f97fb5c13fcb243826bb1b2b54ffb68034d1ae Mon Sep 17 00:00:00 2001 From: Sam sam Date: Tue, 14 Feb 2023 15:43:57 -0800 Subject: [PATCH 32/42] Apply suggestions from code review Co-authored-by: Lukasz Cwik --- .../main/java/org/apache/beam/fn/harness/FnHarness.java | 6 +----- .../java/org/apache/beam/fn/harness/debug/DataSampler.java | 7 ++++--- .../org/apache/beam/fn/harness/debug/OutputSampler.java | 6 +++++- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index ec75c43e050b..22f8424fb4dd 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -254,11 +254,7 @@ public static void main( FinalizeBundleHandler finalizeBundleHandler = new FinalizeBundleHandler(executorService); // Create the sampler, if the experiment is enabled. - Optional> experimentList = - Optional.ofNullable(options.as(ExperimentalOptions.class).getExperiments()); - boolean shouldSample = - experimentList.isPresent() - && experimentList.get().contains(ENABLE_DATA_SAMPLING_EXPERIMENT); + boolean shouldSample = ExperimentalOptions.hasExperiment(options, ENABLE_DATA_SAMPLING_EXPERIMENT); // Retrieves the ProcessBundleDescriptor from cache. Requests the PBD from the Runner if it // doesn't exist. Additionally, runs any graph modifications. diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index f9ca3ed2d48f..1d7e3cbbea0c 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -40,8 +40,7 @@ public class DataSampler { * Creates a DataSampler to sample every 1000 elements while keeping a maximum of 10 in memory. */ public DataSampler() { - this.maxSamples = 10; - this.sampleEveryN = 1000; + this(10, 1000); } /** @@ -68,6 +67,8 @@ public DataSampler(int maxSamples, int sampleEveryN) { * ProcessBundleDescriptor. Uses the given coder encode samples as bytes when responding to a * SampleDataRequest. * + *

Invoked by multiple bundle processing threads in parallel when a new bundle processor is being instantiated. + * * @param pcollectionId The PCollection to take intermittent samples from. * @param coder The coder associated with the PCollection. Coder may be from a nested context. * @param The type of element contained in the PCollection. @@ -86,7 +87,7 @@ public OutputSampler sampleOutput(String pcollectionId, Coder coder) { * SampleDataRequest. * @return Returns all collected samples. */ - public BeamFnApi.InstructionResponse.Builder handleDataSampleRequest( + public synchronized BeamFnApi.InstructionResponse.Builder handleDataSampleRequest( BeamFnApi.InstructionRequest request) { BeamFnApi.SampleDataRequest sampleDataRequest = request.getSampleData(); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index 1fe79b85bcdd..fdfe6d7ce09c 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -60,7 +60,9 @@ public OutputSampler(Coder coder, int maxElements, int sampleEveryN) { } /** - * Samples every 1000th element or if it is part of the first 10 in the (local) PCollection. + * Samples every {@code sampleEveryN}th element or if it is part of the first 10 in the (local) PCollection. + * + *

This method is invoked in parallel by multiple bundle processing threads and in parallel to any {@link #samples} being returned to a thread handling a sample request. * * @param element the element to sample. */ @@ -91,6 +93,8 @@ public void sample(T element) { /** * Clears samples at end of call. This is to help mitigate memory use. * + *

This method is invoked by a thread handling a data sampling request in parallel to any calls to {@link #sample}. + * * @return samples taken since last call. */ public List samples() { From c3db7c0625000f0a7fc8fbcfe32f7d017a1444da Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 14 Feb 2023 15:43:41 -0800 Subject: [PATCH 33/42] address pr comments --- .../org/apache/beam/fn/harness/FnHarness.java | 2 +- .../beam/fn/harness/debug/DataSampler.java | 47 ++++--------------- .../beam/fn/harness/debug/OutputSampler.java | 16 ++++--- .../fn/harness/debug/OutputSamplerTest.java | 16 ++++--- 4 files changed, 29 insertions(+), 52 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index 22f8424fb4dd..811c9377a072 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -94,7 +94,6 @@ public class FnHarness { private static final String RUNNER_CAPABILITIES = "RUNNER_CAPABILITIES"; private static final String ENABLE_DATA_SAMPLING_EXPERIMENT = "enable_data_sampling"; private static final Logger LOG = LoggerFactory.getLogger(FnHarness.class); - private static final DataSampler dataSampler = new DataSampler(); private static Endpoints.ApiServiceDescriptor getApiServiceDescriptor(String descriptor) throws TextFormat.ParseException { @@ -226,6 +225,7 @@ public static void main( options.as(ExecutorOptions.class).getScheduledExecutorService(); ExecutionStateSampler executionStateSampler = new ExecutionStateSampler(options, System::currentTimeMillis); + final DataSampler dataSampler = new DataSampler(); // The logging client variable is not used per se, but during its lifetime (until close()) it // intercepts logging and sends it to the logging service. diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index 1d7e3cbbea0c..4132deb34aa6 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -17,16 +17,12 @@ */ package org.apache.beam.fn.harness.debug; -import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.fnexecution.v1.BeamFnApi.SampleDataResponse.ElementList; -import org.apache.beam.model.fnexecution.v1.BeamFnApi.SampledElement; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; /** * The DataSampler is a global (per SDK Harness) object that facilitates taking and returning @@ -75,9 +71,8 @@ public DataSampler(int maxSamples, int sampleEveryN) { * @return the OutputSampler corresponding to the unique PBD and PCollection. */ public OutputSampler sampleOutput(String pcollectionId, Coder coder) { - outputSamplers.putIfAbsent( - pcollectionId, new OutputSampler<>(coder, this.maxSamples, this.sampleEveryN)); - return (OutputSampler) outputSamplers.get(pcollectionId); + return (OutputSampler)outputSamplers.computeIfAbsent( + pcollectionId, k -> new OutputSampler<>(coder, this.maxSamples, this.sampleEveryN)); } /** @@ -91,46 +86,22 @@ public synchronized BeamFnApi.InstructionResponse.Builder handleDataSampleReques BeamFnApi.InstructionRequest request) { BeamFnApi.SampleDataRequest sampleDataRequest = request.getSampleData(); - Map> responseSamples = - samplesFor(sampleDataRequest.getPcollectionIdsList()); - - BeamFnApi.SampleDataResponse.Builder response = BeamFnApi.SampleDataResponse.newBuilder(); - for (Map.Entry> entry : responseSamples.entrySet()) { - String pcollectionId = entry.getKey(); - ElementList.Builder elementList = ElementList.newBuilder(); - for (byte[] sample : entry.getValue()) { - elementList.addElements( - SampledElement.newBuilder().setElement(ByteString.copyFrom(sample)).build()); - } - response.putElementSamples(pcollectionId, elementList.build()); - } - - return BeamFnApi.InstructionResponse.newBuilder().setSampleData(response); - } - - /** - * Returns a map from PCollection to its samples. Samples are filtered on - * ProcessBundleDescriptorIds and PCollections. Thread-safe. - * - * @param pcollections Filters all PCollections on this set. If empty, allows all PCollections. - * @return a map from PCollection to its samples. - */ - private Map> samplesFor(List pcollections) { - Map> samples = new HashMap<>(); + List pcollections = sampleDataRequest.getPcollectionIdsList(); // Safe to iterate as the ConcurrentHashMap will return each element at most once and will not - // throw - // ConcurrentModificationException. + // throw ConcurrentModificationException. + BeamFnApi.SampleDataResponse.Builder response = BeamFnApi.SampleDataResponse.newBuilder(); outputSamplers.forEach( (pcollectionId, outputSampler) -> { if (!pcollections.isEmpty() && !pcollections.contains(pcollectionId)) { return; } - samples.putIfAbsent(pcollectionId, new ArrayList<>()); - samples.get(pcollectionId).addAll(outputSampler.samples()); + response.putElementSamples( + pcollectionId, + ElementList.newBuilder().addAllElements(outputSampler.samples()).build()); }); - return samples; + return BeamFnApi.InstructionResponse.newBuilder().setSampleData(response); } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index fdfe6d7ce09c..48bd001b1e9c 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -17,11 +17,12 @@ */ package org.apache.beam.fn.harness.debug; -import java.io.ByteArrayOutputStream; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicLong; +import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.util.ByteStringOutputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -97,8 +98,8 @@ public void sample(T element) { * * @return samples taken since last call. */ - public List samples() { - List ret = new ArrayList<>(); + public List samples() { + List ret = new ArrayList<>(); // Serializing can take a lot of CPU time for larger or complex elements. Copy the array here // so as to not slow down the main processing hot path. @@ -109,18 +110,19 @@ public List samples() { resampleIndex = 0; } - ByteArrayOutputStream stream = new ByteArrayOutputStream(); + ByteStringOutputStream stream = new ByteStringOutputStream(); for (T el : copiedBuffer) { try { // This is deprecated, but until this is fully removed, this specifically needs the nested // context. This is because the SDK will need to decode the sampled elements with the // ToStringFn. coder.encode(el, stream, Coder.Context.NESTED); - ret.add(stream.toByteArray()); + ret.add( + BeamFnApi.SampledElement.newBuilder() + .setElement(stream.toByteStringAndReset()) + .build()); } catch (Exception exception) { LOG.warn("Could not encode element \"" + el + "\" to bytes: " + exception); - } finally { - stream.reset(); } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java index f3e0a5a06211..e25be2353421 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -24,18 +24,22 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @RunWith(JUnit4.class) public class OutputSamplerTest { - public byte[] encodeInt(Integer i) throws IOException { + public BeamFnApi.SampledElement encodeInt(Integer i) throws IOException { VarIntCoder coder = VarIntCoder.of(); ByteArrayOutputStream stream = new ByteArrayOutputStream(); coder.encode(i, stream); - return stream.toByteArray(); + return BeamFnApi.SampledElement.newBuilder() + .setElement(ByteString.copyFrom(stream.toByteArray())) + .build(); } /** @@ -54,12 +58,12 @@ public void testSamplesFirstN() throws Exception { } // The expected list is only 0..9 inclusive. - List expected = new ArrayList<>(); + List expected = new ArrayList<>(); for (int i = 0; i < 10; ++i) { expected.add(encodeInt(i)); } - List samples = outputSampler.samples(); + List samples = outputSampler.samples(); assertThat(samples, containsInAnyOrder(expected.toArray())); } @@ -80,14 +84,14 @@ public void testActsLikeCircularBuffer() throws Exception { // The first 10 are always sampled, but with maxSamples = 5, the first ten are downsampled to // 4..9 inclusive. Then, // the 20th element is sampled (19) and every 20 after. - List expected = new ArrayList<>(); + List expected = new ArrayList<>(); expected.add(encodeInt(19)); expected.add(encodeInt(39)); expected.add(encodeInt(59)); expected.add(encodeInt(79)); expected.add(encodeInt(99)); - List samples = outputSampler.samples(); + List samples = outputSampler.samples(); assertThat(samples, containsInAnyOrder(expected.toArray())); } } From fce6d69b41acbce958698f7b4afdb5c41a86fb31 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 14 Feb 2023 16:13:01 -0800 Subject: [PATCH 34/42] spotlessapply and add byte[] test --- .../org/apache/beam/fn/harness/FnHarness.java | 5 ++- .../beam/fn/harness/debug/DataSampler.java | 8 +++-- .../beam/fn/harness/debug/OutputSampler.java | 9 ++++-- .../fn/harness/debug/DataSamplerTest.java | 32 +++++++++++++++++-- 4 files changed, 43 insertions(+), 11 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index 811c9377a072..ce78f33c14cb 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -19,8 +19,6 @@ import java.util.Collections; import java.util.EnumMap; -import java.util.List; -import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.function.Function; @@ -254,7 +252,8 @@ public static void main( FinalizeBundleHandler finalizeBundleHandler = new FinalizeBundleHandler(executorService); // Create the sampler, if the experiment is enabled. - boolean shouldSample = ExperimentalOptions.hasExperiment(options, ENABLE_DATA_SAMPLING_EXPERIMENT); + boolean shouldSample = + ExperimentalOptions.hasExperiment(options, ENABLE_DATA_SAMPLING_EXPERIMENT); // Retrieves the ProcessBundleDescriptor from cache. Requests the PBD from the Runner if it // doesn't exist. Additionally, runs any graph modifications. diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index 4132deb34aa6..8c667b05c8fd 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -63,7 +63,8 @@ public DataSampler(int maxSamples, int sampleEveryN) { * ProcessBundleDescriptor. Uses the given coder encode samples as bytes when responding to a * SampleDataRequest. * - *

Invoked by multiple bundle processing threads in parallel when a new bundle processor is being instantiated. + *

Invoked by multiple bundle processing threads in parallel when a new bundle processor is + * being instantiated. * * @param pcollectionId The PCollection to take intermittent samples from. * @param coder The coder associated with the PCollection. Coder may be from a nested context. @@ -71,8 +72,9 @@ public DataSampler(int maxSamples, int sampleEveryN) { * @return the OutputSampler corresponding to the unique PBD and PCollection. */ public OutputSampler sampleOutput(String pcollectionId, Coder coder) { - return (OutputSampler)outputSamplers.computeIfAbsent( - pcollectionId, k -> new OutputSampler<>(coder, this.maxSamples, this.sampleEveryN)); + return (OutputSampler) + outputSamplers.computeIfAbsent( + pcollectionId, k -> new OutputSampler<>(coder, this.maxSamples, this.sampleEveryN)); } /** diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index 48bd001b1e9c..883986453bb7 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -61,9 +61,11 @@ public OutputSampler(Coder coder, int maxElements, int sampleEveryN) { } /** - * Samples every {@code sampleEveryN}th element or if it is part of the first 10 in the (local) PCollection. + * Samples every {@code sampleEveryN}th element or if it is part of the first 10 in the (local) + * PCollection. * - *

This method is invoked in parallel by multiple bundle processing threads and in parallel to any {@link #samples} being returned to a thread handling a sample request. + *

This method is invoked in parallel by multiple bundle processing threads and in parallel to + * any {@link #samples} being returned to a thread handling a sample request. * * @param element the element to sample. */ @@ -94,7 +96,8 @@ public void sample(T element) { /** * Clears samples at end of call. This is to help mitigate memory use. * - *

This method is invoked by a thread handling a data sampling request in parallel to any calls to {@link #sample}. + *

This method is invoked by a thread handling a data sampling request in parallel to any calls + * to {@link #sample}. * * @return samples taken since last call. */ diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index 77c0ad8441bb..416eb1eb4fb1 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -25,11 +25,14 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; import org.apache.beam.model.fnexecution.v1.BeamFnApi; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; @@ -43,14 +46,21 @@ public class DataSamplerTest { byte[] encodeInt(Integer i) throws IOException { VarIntCoder coder = VarIntCoder.of(); ByteArrayOutputStream stream = new ByteArrayOutputStream(); - coder.encode(i, stream); + coder.encode(i, stream, Coder.Context.NESTED); return stream.toByteArray(); } byte[] encodeString(String s) throws IOException { StringUtf8Coder coder = StringUtf8Coder.of(); ByteArrayOutputStream stream = new ByteArrayOutputStream(); - coder.encode(s, stream); + coder.encode(s, stream, Coder.Context.NESTED); + return stream.toByteArray(); + } + + byte[] encodeByteArray(byte[] b) throws IOException { + ByteArrayCoder coder = ByteArrayCoder.of(); + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + coder.encode(b, stream, Coder.Context.NESTED); return stream.toByteArray(); } @@ -117,6 +127,24 @@ public void testSingleOutput() throws Exception { assertHasSamples(samples, "pcollection-id", Collections.singleton(encodeInt(1))); } + /** + * Smoke test that a samples show in the output map. + * + * @throws Exception + */ + @Test + public void testNestedContext() throws Exception { + DataSampler sampler = new DataSampler(); + + String rawString = "hello"; + byte[] byteArray = rawString.getBytes(Charset.forName("ASCII")); + ByteArrayCoder coder = ByteArrayCoder.of(); + sampler.sampleOutput("pcollection-id", coder).sample(byteArray); + + BeamFnApi.InstructionResponse samples = getAllSamples(sampler); + assertHasSamples(samples, "pcollection-id", Collections.singleton(encodeByteArray(byteArray))); + } + /** * Test that sampling multiple PCollections under the same descriptor is OK. * From 69d8bb4e00c03db90f8c757d5d7a6ebed1366e51 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 14 Feb 2023 17:02:22 -0800 Subject: [PATCH 35/42] validate datasampler args --- .../java/org/apache/beam/fn/harness/debug/DataSampler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index 8c667b05c8fd..8a77af714d2e 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -44,8 +44,8 @@ public DataSampler() { * @param sampleEveryN Sets how often to sample. */ public DataSampler(int maxSamples, int sampleEveryN) { - this.maxSamples = maxSamples; - this.sampleEveryN = sampleEveryN; + this.maxSamples = maxSamples <= 0 ? 10 : maxSamples; + this.sampleEveryN = sampleEveryN <= 0 ? 1000 : sampleEveryN; } // Maximum number of elements in buffer. From be2ebe44661b6d15fb9a8438d9428214610a79ef Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 14 Feb 2023 18:35:41 -0800 Subject: [PATCH 36/42] add concurrency tests --- .../beam/fn/harness/debug/DataSampler.java | 14 ++++- .../beam/fn/harness/debug/OutputSampler.java | 24 +++----- .../data/PCollectionConsumerRegistryTest.java | 60 +++++++++++++++++++ .../fn/harness/debug/DataSamplerTest.java | 40 +++++++++++++ .../fn/harness/debug/OutputSamplerTest.java | 36 +++++++++++ 5 files changed, 155 insertions(+), 19 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index 8a77af714d2e..d5c378a1af04 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -17,12 +17,15 @@ */ package org.apache.beam.fn.harness.debug; +import java.io.IOException; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.fnexecution.v1.BeamFnApi.SampleDataResponse.ElementList; import org.apache.beam.sdk.coders.Coder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * The DataSampler is a global (per SDK Harness) object that facilitates taking and returning @@ -31,6 +34,7 @@ * simultaneously, even if computing the same logical PCollection. */ public class DataSampler { + private static final Logger LOG = LoggerFactory.getLogger(DataSampler.class); /** * Creates a DataSampler to sample every 1000 elements while keeping a maximum of 10 in memory. @@ -99,9 +103,13 @@ public synchronized BeamFnApi.InstructionResponse.Builder handleDataSampleReques return; } - response.putElementSamples( - pcollectionId, - ElementList.newBuilder().addAllElements(outputSampler.samples()).build()); + try { + response.putElementSamples( + pcollectionId, + ElementList.newBuilder().addAllElements(outputSampler.samples()).build()); + } catch (IOException e) { + LOG.warn("Could not encode elements from \"" + pcollectionId + "\" to bytes: " + e); + } }); return BeamFnApi.InstructionResponse.newBuilder().setSampleData(response); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index 883986453bb7..806ad0f12fa0 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -17,14 +17,13 @@ */ package org.apache.beam.fn.harness.debug; +import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicLong; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * This class holds samples for a single PCollection until queried by the parent DataSampler. This @@ -34,7 +33,6 @@ * @param the element type of the PCollection. */ public class OutputSampler { - private static final Logger LOG = LoggerFactory.getLogger(OutputSampler.class); // Temporarily holds elements until the SDK receives a sample data request. private final List buffer; @@ -101,7 +99,7 @@ public void sample(T element) { * * @return samples taken since last call. */ - public List samples() { + public List samples() throws IOException { List ret = new ArrayList<>(); // Serializing can take a lot of CPU time for larger or complex elements. Copy the array here @@ -115,18 +113,12 @@ public List samples() { ByteStringOutputStream stream = new ByteStringOutputStream(); for (T el : copiedBuffer) { - try { - // This is deprecated, but until this is fully removed, this specifically needs the nested - // context. This is because the SDK will need to decode the sampled elements with the - // ToStringFn. - coder.encode(el, stream, Coder.Context.NESTED); - ret.add( - BeamFnApi.SampledElement.newBuilder() - .setElement(stream.toByteStringAndReset()) - .build()); - } catch (Exception exception) { - LOG.warn("Could not encode element \"" + el + "\" to bytes: " + exception); - } + // This is deprecated, but until this is fully removed, this specifically needs the nested + // context. This is because the SDK will need to decode the sampled elements with the + // ToStringFn. + coder.encode(el, stream, Coder.Context.NESTED); + ret.add( + BeamFnApi.SampledElement.newBuilder().setElement(stream.toByteStringAndReset()).build()); } return ret; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java index 35bd5697adc0..c24f016b5cc1 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java @@ -21,6 +21,8 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; @@ -39,6 +41,8 @@ import org.apache.beam.fn.harness.control.BundleProgressReporter; import org.apache.beam.fn.harness.control.ExecutionStateSampler; import org.apache.beam.fn.harness.control.ExecutionStateSampler.ExecutionStateTracker; +import org.apache.beam.fn.harness.debug.DataSampler; +import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleDescriptor; import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; @@ -56,6 +60,7 @@ import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterator; @@ -507,6 +512,61 @@ public void testLazyByteSizeEstimation() throws Exception { assertThat(result, containsInAnyOrder(expected.toArray())); } + /** + * Test that element samples are taken when a DataSampler is present. + * + * @throws Exception + */ + @Test + public void dataSampling() throws Exception { + final String pTransformIdA = "pTransformIdA"; + + ShortIdMap shortIds = new ShortIdMap(); + BundleProgressReporter.InMemory reporterAndRegistrar = new BundleProgressReporter.InMemory(); + DataSampler dataSampler = new DataSampler(); + PCollectionConsumerRegistry consumers = + new PCollectionConsumerRegistry( + sampler.create(), shortIds, reporterAndRegistrar, TEST_DESCRIPTOR, dataSampler); + FnDataReceiver> consumerA1 = mock(FnDataReceiver.class); + + consumers.register(P_COLLECTION_A, pTransformIdA, pTransformIdA + "Name", consumerA1); + + FnDataReceiver> wrapperConsumer = + (FnDataReceiver>) + (FnDataReceiver) consumers.getMultiplexingConsumer(P_COLLECTION_A); + String elementValue = "elem"; + WindowedValue element = valueInGlobalWindow(elementValue); + int numElements = 10; + for (int i = 0; i < numElements; i++) { + wrapperConsumer.accept(element); + } + + BeamFnApi.InstructionRequest request = + BeamFnApi.InstructionRequest.newBuilder() + .setSampleData(BeamFnApi.SampleDataRequest.newBuilder()) + .build(); + BeamFnApi.InstructionResponse response = dataSampler.handleDataSampleRequest(request).build(); + + Map elementSamplesMap = + response.getSampleData().getElementSamplesMap(); + + assertFalse(elementSamplesMap.isEmpty()); + + BeamFnApi.SampleDataResponse.ElementList elementList = elementSamplesMap.get(P_COLLECTION_A); + assertNotNull(elementList); + + List expectedSamples = new ArrayList<>(); + StringUtf8Coder coder = StringUtf8Coder.of(); + for (int i = 0; i < numElements; i++) { + ByteStringOutputStream stream = new ByteStringOutputStream(); + coder.encode(elementValue, stream); + expectedSamples.add( + BeamFnApi.SampledElement.newBuilder().setElement(stream.toByteStringAndReset()).build()); + } + + assertTrue(elementList.getElementsList().containsAll(expectedSamples)); + } + private static class TestElementByteSizeObservableIterable extends ElementByteSizeObservableIterable> { private List elements; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index 416eb1eb4fb1..6dfc75623ef1 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -221,4 +221,44 @@ public void testFiltersMultiplePCollectionIds() throws Exception { assertHasSamples(samples, "a", ImmutableList.of(encodeString("a1"), encodeString("a2"))); assertHasSamples(samples, "c", ImmutableList.of(encodeString("c1"), encodeString("c2"))); } + + /** + * Test that samples can be taken from the DataSampler while adding new OutputSamplers. This fails + * with a ConcurrentModificationException if there is a bug. + * + * @throws Exception + */ + @Test + public void testConcurrentNewSampler() throws Exception { + DataSampler sampler = new DataSampler(); + VarIntCoder coder = VarIntCoder.of(); + + // Create a thread that constantly creates new samplers. + Thread sampleThread = + new Thread( + () -> { + for (int i = 0; i < 1000000; i++) { + sampler.sampleOutput("pcollection-" + i, coder).sample(0); + + // This sleep is here to allow for the test to stop this thread. + try { + Thread.sleep(0); + } catch (InterruptedException e) { + return; + } + } + }); + + sampleThread.start(); + + for (int i = 0; i < 20; i++) { + sampler.handleDataSampleRequest( + BeamFnApi.InstructionRequest.newBuilder() + .setSampleData(BeamFnApi.SampleDataRequest.newBuilder()) + .build()); + } + + sampleThread.interrupt(); + sampleThread.join(); + } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java index e25be2353421..9456a6cb7dad 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -94,4 +94,40 @@ public void testActsLikeCircularBuffer() throws Exception { List samples = outputSampler.samples(); assertThat(samples, containsInAnyOrder(expected.toArray())); } + + /** + * Test that sampling a PCollection while retrieving samples from multiple threads is ok. + * + * @throws Exception + */ + @Test + public void testConcurrentSamples() throws Exception { + VarIntCoder coder = VarIntCoder.of(); + OutputSampler outputSampler = new OutputSampler<>(coder, 100000, 1); + + // Iteration count was empirically chosen to have a high probability of failure without the + // test going for too long. + Thread sampleThreadA = + new Thread( + () -> { + for (int i = 0; i < 10000000; i++) { + outputSampler.sample(i); + } + }); + + Thread sampleThreadB = + new Thread( + () -> { + for (int i = 0; i < 10000000; i++) { + outputSampler.sample(i); + } + }); + + sampleThreadA.start(); + sampleThreadB.start(); + + for (int i = 0; i < 10000; i++) { + outputSampler.samples(); + } + } } From cbdbbc38632019d21f725b168d1565c1c29b518d Mon Sep 17 00:00:00 2001 From: Sam sam Date: Tue, 21 Feb 2023 15:04:44 -0800 Subject: [PATCH 37/42] Update sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java Co-authored-by: Lukasz Cwik --- .../java/org/apache/beam/fn/harness/debug/DataSampler.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index d5c378a1af04..442085800112 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -48,8 +48,10 @@ public DataSampler() { * @param sampleEveryN Sets how often to sample. */ public DataSampler(int maxSamples, int sampleEveryN) { - this.maxSamples = maxSamples <= 0 ? 10 : maxSamples; - this.sampleEveryN = sampleEveryN <= 0 ? 1000 : sampleEveryN; + checkArgument(maxSamples > 0, "Expected positive number of samples, did you mean to disable data sampling?"); + checkArgument(sampleEveryN > 0, "Expected positive number for sampling period, did you mean to disable data sampling?"); + this.maxSamples = maxSamples; + this.sampleEveryN = sampleEveryN; } // Maximum number of elements in buffer. From 8cc8ee02c607ec9a49faa59abdb42e2871593e1f Mon Sep 17 00:00:00 2001 From: Sam sam Date: Tue, 21 Feb 2023 15:05:01 -0800 Subject: [PATCH 38/42] Update sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java Co-authored-by: Lukasz Cwik --- .../org/apache/beam/fn/harness/debug/DataSamplerTest.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index 6dfc75623ef1..56efb23471fb 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -239,13 +239,6 @@ public void testConcurrentNewSampler() throws Exception { () -> { for (int i = 0; i < 1000000; i++) { sampler.sampleOutput("pcollection-" + i, coder).sample(0); - - // This sleep is here to allow for the test to stop this thread. - try { - Thread.sleep(0); - } catch (InterruptedException e) { - return; - } } }); From 10aa7de1c14f6d50b679dcb03b4b42fd9bd40faf Mon Sep 17 00:00:00 2001 From: Sam sam Date: Tue, 21 Feb 2023 15:10:11 -0800 Subject: [PATCH 39/42] Update sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java Co-authored-by: Lukasz Cwik --- .../java/org/apache/beam/fn/harness/debug/DataSamplerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index 56efb23471fb..1eb56dc24f9e 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -128,7 +128,7 @@ public void testSingleOutput() throws Exception { } /** - * Smoke test that a samples show in the output map. + * Smoke test that a sample shows in the output map. * * @throws Exception */ From 04c6c887d5b20632357593f000923fe8d38a289f Mon Sep 17 00:00:00 2001 From: Sam sam Date: Tue, 21 Feb 2023 15:10:23 -0800 Subject: [PATCH 40/42] Update sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java Co-authored-by: Lukasz Cwik --- .../java/org/apache/beam/fn/harness/debug/DataSamplerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index 1eb56dc24f9e..d6c223129a53 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -137,7 +137,7 @@ public void testNestedContext() throws Exception { DataSampler sampler = new DataSampler(); String rawString = "hello"; - byte[] byteArray = rawString.getBytes(Charset.forName("ASCII")); + byte[] byteArray = rawString.getBytes(StandardCharsets.US_ASCII); ByteArrayCoder coder = ByteArrayCoder.of(); sampler.sampleOutput("pcollection-id", coder).sample(byteArray); From 862439b4c0592c5118507506a06f8d7d10496732 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 21 Feb 2023 16:43:31 -0800 Subject: [PATCH 41/42] improve contention tests --- .../beam/fn/harness/debug/DataSampler.java | 2 + .../beam/fn/harness/debug/OutputSampler.java | 15 ++-- .../fn/harness/debug/DataSamplerTest.java | 43 +++++++---- .../fn/harness/debug/OutputSamplerTest.java | 72 +++++++++++++++++-- 4 files changed, 108 insertions(+), 24 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index 442085800112..c2344f77b491 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -17,6 +17,8 @@ */ package org.apache.beam.fn.harness.debug; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + import java.io.IOException; import java.util.List; import java.util.Map; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index 806ad0f12fa0..326f2dbfe8f3 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -35,7 +35,7 @@ public class OutputSampler { // Temporarily holds elements until the SDK receives a sample data request. - private final List buffer; + private List buffer; // Maximum number of elements in buffer. private final int maxElements; @@ -104,19 +104,22 @@ public List samples() throws IOException { // Serializing can take a lot of CPU time for larger or complex elements. Copy the array here // so as to not slow down the main processing hot path. - List copiedBuffer; + List bufferToSend; + int sampleIndex = 0; synchronized (this) { - copiedBuffer = new ArrayList<>(buffer); - buffer.clear(); + bufferToSend = buffer; + sampleIndex = resampleIndex; + buffer = new ArrayList<>(maxElements); resampleIndex = 0; } ByteStringOutputStream stream = new ByteStringOutputStream(); - for (T el : copiedBuffer) { + for (int i = 0; i < bufferToSend.size(); i++) { + int index = (sampleIndex + i) % bufferToSend.size(); // This is deprecated, but until this is fully removed, this specifically needs the nested // context. This is because the SDK will need to decode the sampled elements with the // ToStringFn. - coder.encode(el, stream, Coder.Context.NESTED); + coder.encode(bufferToSend.get(index), stream, Coder.Context.NESTED); ret.add( BeamFnApi.SampledElement.newBuilder().setElement(stream.toByteStringAndReset()).build()); } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index d6c223129a53..6398f51dadd6 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -26,10 +26,13 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.CountDownLatch; + import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; @@ -233,25 +236,39 @@ public void testConcurrentNewSampler() throws Exception { DataSampler sampler = new DataSampler(); VarIntCoder coder = VarIntCoder.of(); - // Create a thread that constantly creates new samplers. - Thread sampleThread = - new Thread( - () -> { - for (int i = 0; i < 1000000; i++) { - sampler.sampleOutput("pcollection-" + i, coder).sample(0); - } - }); - - sampleThread.start(); + // Make threads that will create 100 individual OutputSamplers each. + Thread[] sampleThreads = new Thread[100]; + CountDownLatch startSignal = new CountDownLatch(1); + CountDownLatch doneSignal = new CountDownLatch(sampleThreads.length); + + for (int i = 0; i < sampleThreads.length; i++) { + sampleThreads[i] = new Thread( + () -> { + try { + startSignal.await(); + } catch (InterruptedException e) { + return; + } + + for (int j = 0; j < 100; j++) { + sampler.sampleOutput("pcollection-" + j, coder).sample(0); + } + + doneSignal.countDown(); + }); + sampleThreads[i].start(); + } - for (int i = 0; i < 20; i++) { + startSignal.countDown(); + while(doneSignal.getCount() > 0) { sampler.handleDataSampleRequest( BeamFnApi.InstructionRequest.newBuilder() .setSampleData(BeamFnApi.SampleDataRequest.newBuilder()) .build()); } - sampleThread.interrupt(); - sampleThread.join(); + for (Thread sampleThread : sampleThreads) { + sampleThread.join(); + } } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java index 9456a6cb7dad..c4c20d42a326 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -17,13 +17,23 @@ */ package org.apache.beam.fn.harness.debug; +import static junit.framework.TestCase.assertEquals; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.InputStream; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.PriorityQueue; +import java.util.concurrent.CountDownLatch; + import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString; @@ -103,31 +113,83 @@ public void testActsLikeCircularBuffer() throws Exception { @Test public void testConcurrentSamples() throws Exception { VarIntCoder coder = VarIntCoder.of(); - OutputSampler outputSampler = new OutputSampler<>(coder, 100000, 1); + OutputSampler outputSampler = new OutputSampler<>(coder, 10, 2); + + CountDownLatch startSignal = new CountDownLatch(1); + CountDownLatch doneSignal = new CountDownLatch(2); // Iteration count was empirically chosen to have a high probability of failure without the // test going for too long. + // Generates a range of numbers from 0 to 1000000. Thread sampleThreadA = new Thread( () -> { - for (int i = 0; i < 10000000; i++) { + try { + startSignal.await(); + } catch (InterruptedException e) { + return; + } + + for (int i = 0; i < 1000000; i++) { outputSampler.sample(i); } + + doneSignal.countDown(); }); + // Generates a range of numbers from -1000000 to 0. Thread sampleThreadB = new Thread( () -> { - for (int i = 0; i < 10000000; i++) { + try { + startSignal.await(); + } catch (InterruptedException e) { + return; + } + + for (int i = -1000000; i < 0; i++) { outputSampler.sample(i); } + + doneSignal.countDown(); }); + // Ready the threads. sampleThreadA.start(); sampleThreadB.start(); - for (int i = 0; i < 10000; i++) { - outputSampler.samples(); + // Start the threads at the same time. + startSignal.countDown(); + + // Generate contention by sampling at the same time as the samples are generated. + List samples = new ArrayList<>(); + while (doneSignal.getCount() > 0) { + samples.addAll(outputSampler.samples()); + } + + // Stop the threads and sort the samples from which thread it came from. + sampleThreadA.join(); + sampleThreadB.join(); + List samplesFromThreadA = new ArrayList<>(); + List samplesFromThreadB = new ArrayList<>(); + for (BeamFnApi.SampledElement sampledElement : samples) { + int el = coder.decode(sampledElement.getElement().newInput()); + if (el >= 0) { + samplesFromThreadA.add(el); + } else { + samplesFromThreadB.add(el); + } } + + // Copy the array and sort it. + List sortedSamplesFromThreadA = new ArrayList<>(samplesFromThreadA); + List sortedSamplesFromThreadB = new ArrayList<>(samplesFromThreadB); + Collections.sort(sortedSamplesFromThreadA); + Collections.sort(sortedSamplesFromThreadB); + + // Order is preserved when getting the samples. If there is a weird race condition, these + // numbers may be out of order. + assertEquals(samplesFromThreadA, sortedSamplesFromThreadA); + assertEquals(samplesFromThreadB, sortedSamplesFromThreadB); } } From 490be4ac81a6f4260c55c99ff6abc71b0238fb06 Mon Sep 17 00:00:00 2001 From: Sam Rohde Date: Tue, 21 Feb 2023 16:46:55 -0800 Subject: [PATCH 42/42] spotless --- .../beam/fn/harness/debug/DataSampler.java | 8 +++-- .../fn/harness/debug/DataSamplerTest.java | 35 +++++++++---------- .../fn/harness/debug/OutputSamplerTest.java | 7 ---- 3 files changed, 23 insertions(+), 27 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java index c2344f77b491..2a13b5dac3d3 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java @@ -50,8 +50,12 @@ public DataSampler() { * @param sampleEveryN Sets how often to sample. */ public DataSampler(int maxSamples, int sampleEveryN) { - checkArgument(maxSamples > 0, "Expected positive number of samples, did you mean to disable data sampling?"); - checkArgument(sampleEveryN > 0, "Expected positive number for sampling period, did you mean to disable data sampling?"); + checkArgument( + maxSamples > 0, + "Expected positive number of samples, did you mean to disable data sampling?"); + checkArgument( + sampleEveryN > 0, + "Expected positive number for sampling period, did you mean to disable data sampling?"); this.maxSamples = maxSamples; this.sampleEveryN = sampleEveryN; } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index 6398f51dadd6..4b874dd7e980 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -25,14 +25,12 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; - import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; @@ -240,27 +238,28 @@ public void testConcurrentNewSampler() throws Exception { Thread[] sampleThreads = new Thread[100]; CountDownLatch startSignal = new CountDownLatch(1); CountDownLatch doneSignal = new CountDownLatch(sampleThreads.length); - + for (int i = 0; i < sampleThreads.length; i++) { - sampleThreads[i] = new Thread( - () -> { - try { - startSignal.await(); - } catch (InterruptedException e) { - return; - } - - for (int j = 0; j < 100; j++) { - sampler.sampleOutput("pcollection-" + j, coder).sample(0); - } - - doneSignal.countDown(); - }); + sampleThreads[i] = + new Thread( + () -> { + try { + startSignal.await(); + } catch (InterruptedException e) { + return; + } + + for (int j = 0; j < 100; j++) { + sampler.sampleOutput("pcollection-" + j, coder).sample(0); + } + + doneSignal.countDown(); + }); sampleThreads[i].start(); } startSignal.countDown(); - while(doneSignal.getCount() > 0) { + while (doneSignal.getCount() > 0) { sampler.handleDataSampleRequest( BeamFnApi.InstructionRequest.newBuilder() .setSampleData(BeamFnApi.SampleDataRequest.newBuilder()) diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java index c4c20d42a326..953ccce9e235 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -19,21 +19,14 @@ import static junit.framework.TestCase.assertEquals; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; -import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.io.InputStream; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.PriorityQueue; import java.util.concurrent.CountDownLatch; - import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.vendor.grpc.v1p48p1.com.google.protobuf.ByteString;