lukecwik commented on code in PR #25354:
URL: https://github.com/apache/beam/pull/25354#discussion_r1102048741


##########
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:
   I see what you mean by using a DataSampler for each bundle but do you really 
want to sample for each bundle independently N elements and then 1 element for 
each 1000 since this would be an issue for streaming since streaming has 100s 
of work items executing in parallel with pretty small bundles so if your 
sampling at the bundle level your likely going to over sample by a lot.
   
   I was thinking that you would have a single object per PCollection 
effectively that stored all the samples so you wouldn't have to worry about 
memory growth beyond the fixed size of the N objects per PCollection.



-- 
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]

Reply via email to