[
https://issues.apache.org/jira/browse/BEAM-7948?focusedWorklogId=344421&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-344421
]
ASF GitHub Bot logged work on BEAM-7948:
----------------------------------------
Author: ASF GitHub Bot
Created on: 15/Nov/19 17:12
Start Date: 15/Nov/19 17:12
Worklog Time Spent: 10m
Work Description: lukecwik commented on pull request #9949: [BEAM-7948]
Add time-based cache threshold support in the Java data s…
URL: https://github.com/apache/beam/pull/9949#discussion_r346923884
##########
File path:
sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserverTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.sdk.fn.data;
+
+import static
org.apache.beam.sdk.fn.data.BeamFnDataSizeBasedBufferingOutboundObserverTest.messageWithData;
+import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.CountDownLatch;
+import java.util.function.Consumer;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.LengthPrefixCoder;
+import org.apache.beam.sdk.fn.test.TestStreams;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.util.WindowedValue;
+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;
+
+/** Tests for {@link BeamFnDataTimeBasedBufferingOutboundObserver}. */
+@RunWith(JUnit4.class)
+public class BeamFnDataTimeBasedBufferingOutboundObserverTest {
+ private static final LogicalEndpoint OUTPUT_LOCATION =
LogicalEndpoint.of("777L", "555L");
+ private static final Coder<WindowedValue<byte[]>> CODER =
+
LengthPrefixCoder.of(WindowedValue.getValueOnlyCoder(ByteArrayCoder.of()));
+
+ @Test
+ public void testConfiguredTimeLimit() throws Exception {
+ Collection<Elements> values = new ArrayList<>();
+ PipelineOptions options = PipelineOptionsFactory.create();
+ options
+ .as(ExperimentalOptions.class)
+ .setExperiments(Arrays.asList("beam_fn_api_data_buffer_time_limit=1"));
+ final CountDownLatch waitForFlush = new CountDownLatch(1);
+ CloseableFnDataReceiver<WindowedValue<byte[]>> consumer =
+ BeamFnDataBufferingOutboundObserver.forLocation(
+ options,
+ OUTPUT_LOCATION,
+ CODER,
+ TestStreams.withOnNext(
+ (Consumer<Elements>)
+ e -> {
+ values.add(e);
+ waitForFlush.countDown();
+ })
+ .build());
+
+ // Test that it emits when time passed the time limit
+ consumer.accept(valueInGlobalWindow(new byte[1]));
+ waitForFlush.await(); // wait the flush thread to flush the buffer
+ assertEquals(messageWithData(new byte[1]), Iterables.get(values, 0));
+ }
+
+ @Test
+ public void testConfiguredTimeLimitExceptionPropagation() throws Exception {
+ PipelineOptions options = PipelineOptionsFactory.create();
+ options
+ .as(ExperimentalOptions.class)
+ .setExperiments(Arrays.asList("beam_fn_api_data_buffer_time_limit=1"));
+ BeamFnDataTimeBasedBufferingOutboundObserver<WindowedValue<byte[]>>
consumer =
+ (BeamFnDataTimeBasedBufferingOutboundObserver<WindowedValue<byte[]>>)
+ BeamFnDataBufferingOutboundObserver.forLocation(
+ options,
+ OUTPUT_LOCATION,
+ CODER,
+ TestStreams.withOnNext(
+ (Consumer<Elements>)
+ e -> {
+ throw new RuntimeException("");
+ })
+ .build());
+
+ // Test that it emits when time passed the time limit
+ consumer.accept(valueInGlobalWindow(new byte[1]));
+ // wait the flush thread to flush the buffer
+ while (consumer.flushException.get() == null) {
+ Thread.sleep(1);
+ }
+ try {
+ // Test that the exception caught in the flush thread is propagate to
+ // the main thread when processing the next element
+ consumer.accept(valueInGlobalWindow(new byte[1]));
+ fail();
+ } catch (IOException e) {
+ // expected
+ }
+
+ consumer =
Review comment:
It seems like your duplicating the above portion of the test here unless I'm
not seeing what the small difference is.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 344421)
Time Spent: 1h 10m (was: 1h)
> Add time-based cache threshold support in the Java data service
> ---------------------------------------------------------------
>
> Key: BEAM-7948
> URL: https://issues.apache.org/jira/browse/BEAM-7948
> Project: Beam
> Issue Type: Sub-task
> Components: java-fn-execution
> Reporter: sunjincheng
> Assignee: sunjincheng
> Priority: Major
> Time Spent: 1h 10m
> Remaining Estimate: 0h
>
> Currently only size-based cache threshold is supported in data service. It
> should also support the time-based cache threshold. This is very important,
> especially for streaming jobs which are sensitive to the delay.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)