rohdesamuel commented on code in PR #25354: URL: https://github.com/apache/beam/pull/25354#discussion_r1102423922
########## sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSamplingDescriptorModifier.java: ########## @@ -0,0 +1,49 @@ +/* + * 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 { Review Comment: Gotcha, that makes sense. I changed to the suggested implementation and ran the benchmark: **Benchmark with this PR (data sampling not enabled)** Benchmark Mode Cnt Score Error Units ProcessBundleBenchmark.testLargeBundle thrpt 15 1640.020 ± 236.240 ops/s :sdks:java:harness:jmh:jmh (Thread[Execution worker Thread 3,5,main]) completed. Took 5 mins 9.573 secs. **Benchmark with this PR (data sampling enabled)** Benchmark Mode Cnt Score Error Units ProcessBundleBenchmark.testLargeBundle thrpt 15 1094.943 ± 99.748 ops/s :sdks:java:harness:jmh:jmh (Thread[Execution worker Thread 4,5,main]) completed. Took 5 mins 9.603 secs. **Benchmark at upstream/master** Benchmark Mode Cnt Score Error Units ProcessBundleBenchmark.testLargeBundle thrpt 15 1808.108 ± 30.062 ops/s :sdks:java:harness:jmh:jmh (Thread[Execution worker Thread 5,5,main]) completed. Took 5 mins 9.064 secs. Clearly, there's some environmental factors affecting the benchmarking. But, I think it can at least be said this PR with data sampling disabled won't completely hinder performance. ########## sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/DataSampler.java: ########## @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.fn.harness.debug; + +import java.util.ArrayList; +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 + * 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; + } + + // Maximum number of elements in buffer. + private int maxSamples = 10; + + // Sampling rate. + private 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 + // can and will + // access this simultaneously. However, ProcessBundleDescriptors are unique per thread, so only Review Comment: Yeah, those are really good points and I'm surprised I didn't implement it that way given that is the implementation in the runner. I changed it to be a shared class only between PCollections. I don't like the sampledatarequest to get samples for a specific PBD id, so I'll remove that in a PR. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
