akashorabek commented on code in PR #30286:
URL: https://github.com/apache/beam/pull/30286#discussion_r1500616246


##########
it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/pubsub/PubSubIOLT.java:
##########
@@ -0,0 +1,475 @@
+/*
+ * 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.it.gcp.pubsub;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.cloud.Timestamp;
+import com.google.protobuf.ByteString;
+import com.google.pubsub.v1.SubscriptionName;
+import com.google.pubsub.v1.TopicName;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.text.ParseException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.beam.it.common.PipelineLauncher;
+import org.apache.beam.it.common.PipelineOperator;
+import org.apache.beam.it.common.TestProperties;
+import org.apache.beam.it.common.utils.ResourceManagerUtils;
+import org.apache.beam.it.gcp.IOLoadTestBase;
+import org.apache.beam.runners.direct.DirectOptions;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.Primitive;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions;
+import org.apache.beam.sdk.io.synthetic.SyntheticSourceOptions;
+import org.apache.beam.sdk.io.synthetic.SyntheticUnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.testutils.NamedTestResult;
+import org.apache.beam.sdk.testutils.metrics.IOITMetrics;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+/** PubSubIO performance tests. */
+public class PubSubIOLT extends IOLoadTestBase {
+
+  private static final int NUMBER_OF_BUNDLES_FOR_LOCAL = 10;
+  private static final int NUMBER_OF_BUNDLES_FOR_MEDIUM_AND_LARGE = 20;
+  private static final String READ_ELEMENT_METRIC_NAME = "read_count";
+  private static final String MAP_RECORDS_STEP_NAME = "Map records";
+  private static final String WRITE_TO_PUBSUB_STEP_NAME = "Write to PubSub";
+  private static final Map<String, Configuration> TEST_CONFIGS_PRESET;
+  private static TopicName topicName;
+  private static String testConfigName;
+  private static Configuration configuration;
+  private static SubscriptionName subscription;
+  private static InfluxDBSettings influxDBSettings;
+  private static PubsubResourceManager resourceManager;
+
+  @Rule public transient TestPipeline writePipeline = TestPipeline.create();
+  @Rule public transient TestPipeline readPipeline = TestPipeline.create();
+
+  static {
+    try {
+      TEST_CONFIGS_PRESET =
+          ImmutableMap.of(
+              "local",
+              PubSubIOLT.Configuration.fromJsonString(
+                  
"{\"numRecords\":200,\"valueSizeBytes\":1000,\"pipelineTimeout\":7,\"runner\":\"DirectRunner\",\"numWorkers\":1}",
+                  PubSubIOLT.Configuration.class), // 0.2 MB
+              "medium",
+              PubSubIOLT.Configuration.fromJsonString(
+                  
"{\"numRecords\":10000000,\"valueSizeBytes\":1000,\"pipelineTimeout\":20,\"runner\":\"DataflowRunner\",\"numWorkers\":10}",
+                  PubSubIOLT.Configuration.class), // 10 GB
+              "large",
+              PubSubIOLT.Configuration.fromJsonString(
+                  
"{\"numRecords\":100000000,\"valueSizeBytes\":1000,\"pipelineTimeout\":50,\"runner\":\"DataflowRunner\",\"numWorkers\":20}",
+                  PubSubIOLT.Configuration.class) // 100 GB
+              );
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Before
+  public void setup() throws IOException {
+    resourceManager =
+        PubsubResourceManager.builder("io-pubsub-lt", project, 
CREDENTIALS_PROVIDER).build();
+    topicName = resourceManager.createTopic("topic");
+    subscription = resourceManager.createSubscription(topicName, 
"subscription");
+    PipelineOptionsFactory.register(TestPipelineOptions.class);
+
+    // parse configuration
+    testConfigName =
+        TestProperties.getProperty("configuration", "local", 
TestProperties.Type.PROPERTY);
+    configuration = TEST_CONFIGS_PRESET.get(testConfigName);
+    if (configuration == null) {
+      try {
+        configuration =
+            PubSubIOLT.Configuration.fromJsonString(testConfigName, 
PubSubIOLT.Configuration.class);
+      } catch (IOException e) {
+        throw new IllegalArgumentException(
+            String.format(
+                "Unknown test configuration: [%s]. Pass to a valid 
configuration json, or use"
+                    + " config presets: %s",
+                testConfigName, TEST_CONFIGS_PRESET.keySet()));
+      }
+    }
+
+    // Explicitly set up number of bundles in SyntheticUnboundedSource since 
it has a bug in
+    // implementation where
+    // number of lost data in streaming pipeline equals to number of initial 
bundles.

Review Comment:
   It's a bug in the implemenation of `SyntheticUnboundedSource `class. By 
default, it splits the data to some number of bundles and lost records number 
in write streaming pipeline equals to that number of bundles. I'm not sure, 
maybe it doesn't take into consideration last or first record inside of each 
bundle. Anyways, this number should always be positive. I set it excplictly 
here so it will'be easier to test and expect number of lost records when 
verifying the result. 
   [KafkaIO load 
test](https://github.com/apache/beam/blob/master/it/kafka/src/test/java/org/apache/beam/it/kafka/KafkaIOLT.java#L174)
 also uses `SyntheticUnboundedSource` and this bug accurs there as well. Even 
though comment says that it is related to KafkaWriter cache, I think it's more 
likely `SyntheticUnboundedSource` which casuing the issue:
   
![image](https://github.com/apache/beam/assets/70029317/5ef57314-9856-481c-9bad-07bdc0e10927)
   



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