Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

PCollection data sampling for Java SDK harness #25064 #25354

Merged
merged 43 commits into from
Feb 22, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
43 commits
Select commit Hold shift + click to select a range
cd630e4
Data Sampling Java Impl
Jan 26, 2023
e530d14
comments
Jan 26, 2023
a09f5d6
add PBD id to context
rohdesamuel Jan 31, 2023
1cb08f9
merge
Jan 31, 2023
566c7b8
Add more tests and spotless
Jan 31, 2023
7395a2e
Finish Java data sampling impl with tests, adding comments
Feb 1, 2023
4c1253f
more comments, remove Payload
Feb 1, 2023
3d88254
more comments
Feb 1, 2023
5bbef91
spotless
Feb 1, 2023
6993b83
Encode in the nested context
rohdesamuel Feb 3, 2023
769902f
Update sdks/java/harness/src/main/java/org/apache/beam/fn/harness/con…
rohdesamuel Feb 7, 2023
5c06d4e
Apply suggestions from code review
rohdesamuel Feb 7, 2023
99087e8
address pr comments
Feb 9, 2023
5609e4a
give default pbd id to test context
rohdesamuel Feb 9, 2023
694c929
address spotlesscheck
rohdesamuel Feb 9, 2023
519aece
spotless apply
rohdesamuel Feb 9, 2023
8efbb15
style guide spotless apply
rohdesamuel Feb 9, 2023
cd3732d
add serviceloader
rohdesamuel Feb 9, 2023
415e3f0
change datasampling to modify the consumers and not graph for sampling
rohdesamuel Feb 10, 2023
553fc5e
remove redundant SamplerState obj
rohdesamuel Feb 10, 2023
e260eb4
spotless
rohdesamuel Feb 10, 2023
4f29308
replace mutex with atomics in output sampler to reduce contention
rohdesamuel Feb 10, 2023
8cbc6c8
spotless and fix OutputSamplerTest
rohdesamuel Feb 10, 2023
f67234f
Update sdks/java/harness/src/main/java/org/apache/beam/fn/harness/dat…
rohdesamuel Feb 13, 2023
6815330
Update sdks/java/harness/src/main/java/org/apache/beam/fn/harness/dat…
rohdesamuel Feb 13, 2023
4848725
Update sdks/java/harness/src/main/java/org/apache/beam/fn/harness/deb…
rohdesamuel Feb 13, 2023
6c16576
always init outputsampler
rohdesamuel Feb 13, 2023
822587d
add final to DataSampler in FnHarness
rohdesamuel Feb 13, 2023
fe9ab2a
spotless apply
rohdesamuel Feb 13, 2023
07d37ea
update from proto names
rohdesamuel Feb 13, 2023
5e9c4b0
spotless bugs
Feb 14, 2023
f5f97fb
Apply suggestions from code review
rohdesamuel Feb 14, 2023
c3db7c0
address pr comments
Feb 14, 2023
fce6d69
spotlessapply and add byte[] test
Feb 15, 2023
69d8bb4
validate datasampler args
Feb 15, 2023
be2ebe4
add concurrency tests
Feb 15, 2023
93af06e
Merge branch 'master' into data-sampling-java
lukecwik Feb 21, 2023
cbdbbc3
Update sdks/java/harness/src/main/java/org/apache/beam/fn/harness/deb…
rohdesamuel Feb 21, 2023
8cc8ee0
Update sdks/java/harness/src/test/java/org/apache/beam/fn/harness/deb…
rohdesamuel Feb 21, 2023
10aa7de
Update sdks/java/harness/src/test/java/org/apache/beam/fn/harness/deb…
rohdesamuel Feb 21, 2023
04c6c88
Update sdks/java/harness/src/test/java/org/apache/beam/fn/harness/deb…
rohdesamuel Feb 21, 2023
862439b
improve contention tests
Feb 22, 2023
490be4a
spotless
Feb 22, 2023
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
add concurrency tests
  • Loading branch information
Sam Rohde committed Feb 15, 2023
commit be2ebe44661b6d15fb9a8438d9428214610a79ef
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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.
Expand Down Expand Up @@ -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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -34,7 +33,6 @@
* @param <T> the element type of the PCollection.
*/
public class OutputSampler<T> {
private static final Logger LOG = LoggerFactory.getLogger(OutputSampler.class);

// Temporarily holds elements until the SDK receives a sample data request.
private final List<T> buffer;
Expand Down Expand Up @@ -101,7 +99,7 @@ public void sample(T element) {
*
* @return samples taken since last call.
*/
public List<BeamFnApi.SampledElement> samples() {
public List<BeamFnApi.SampledElement> samples() throws IOException {
List<BeamFnApi.SampledElement> ret = new ArrayList<>();

// Serializing can take a lot of CPU time for larger or complex elements. Copy the array here
Expand All @@ -115,18 +113,12 @@ public List<BeamFnApi.SampledElement> 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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -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<WindowedValue<String>> consumerA1 = mock(FnDataReceiver.class);

consumers.register(P_COLLECTION_A, pTransformIdA, pTransformIdA + "Name", consumerA1);

FnDataReceiver<WindowedValue<String>> wrapperConsumer =
(FnDataReceiver<WindowedValue<String>>)
(FnDataReceiver) consumers.getMultiplexingConsumer(P_COLLECTION_A);
String elementValue = "elem";
WindowedValue<String> 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<String, BeamFnApi.SampleDataResponse.ElementList> elementSamplesMap =
response.getSampleData().getElementSamplesMap();

assertFalse(elementSamplesMap.isEmpty());

BeamFnApi.SampleDataResponse.ElementList elementList = elementSamplesMap.get(P_COLLECTION_A);
assertNotNull(elementList);

List<BeamFnApi.SampledElement> 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<T>
extends ElementByteSizeObservableIterable<T, ElementByteSizeObservableIterator<T>> {
private List<T> elements;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
rohdesamuel marked this conversation as resolved.
Show resolved Hide resolved
}
});

sampleThread.start();

for (int i = 0; i < 20; i++) {
sampler.handleDataSampleRequest(
rohdesamuel marked this conversation as resolved.
Show resolved Hide resolved
BeamFnApi.InstructionRequest.newBuilder()
.setSampleData(BeamFnApi.SampleDataRequest.newBuilder())
.build());
}

sampleThread.interrupt();
rohdesamuel marked this conversation as resolved.
Show resolved Hide resolved
sampleThread.join();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -94,4 +94,40 @@ public void testActsLikeCircularBuffer() throws Exception {
List<BeamFnApi.SampledElement> 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<Integer> outputSampler = new OutputSampler<>(coder, 100000, 1);
rohdesamuel marked this conversation as resolved.
Show resolved Hide resolved

// Iteration count was empirically chosen to have a high probability of failure without the
// test going for too long.
Thread sampleThreadA =
rohdesamuel marked this conversation as resolved.
Show resolved Hide resolved
new Thread(
() -> {
for (int i = 0; i < 10000000; i++) {
rohdesamuel marked this conversation as resolved.
Show resolved Hide resolved
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();
}
rohdesamuel marked this conversation as resolved.
Show resolved Hide resolved
}
}