steveniemitz commented on code in PR #32340:
URL: https://github.com/apache/beam/pull/32340#discussion_r1734610265


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableSharedClientTest.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.io.gcp.bigtable;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import com.google.api.gax.grpc.ChannelPoolSettings;
+import com.google.api.gax.grpc.InstantiatingGrpcChannelProvider;
+import com.google.bigtable.v2.BigtableGrpc;
+import com.google.bigtable.v2.MutateRowsRequest;
+import com.google.bigtable.v2.MutateRowsResponse;
+import com.google.bigtable.v2.MutateRowsResponse.Entry;
+import com.google.bigtable.v2.Mutation;
+import com.google.bigtable.v2.Mutation.SetCell;
+import com.google.bigtable.v2.PingAndWarmRequest;
+import com.google.bigtable.v2.PingAndWarmResponse;
+import com.google.bigtable.v2.ReadRowsRequest;
+import com.google.bigtable.v2.ReadRowsResponse;
+import com.google.cloud.bigtable.data.v2.BigtableDataSettings.Builder;
+import com.google.protobuf.ByteString;
+import com.google.rpc.Code;
+import com.google.rpc.Status;
+import io.grpc.BindableService;
+import io.grpc.ForwardingServerCallListener.SimpleForwardingServerCallListener;
+import io.grpc.Grpc;
+import io.grpc.Metadata;
+import io.grpc.Server;
+import io.grpc.ServerBuilder;
+import io.grpc.ServerCall;
+import io.grpc.ServerCall.Listener;
+import io.grpc.ServerCallHandler;
+import io.grpc.ServerInterceptor;
+import io.grpc.stub.StreamObserver;
+import java.io.IOException;
+import java.net.ServerSocket;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiFunction;
+import org.apache.beam.runners.direct.DirectRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.io.GenerateSequence;
+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.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import org.hamcrest.Matchers;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Ensure that BigtableIO.write() reuses the same instance of the underlying 
bigtable client. This
+ * test will create a toy pipeline using DirectRunner and have it write to a 
local emulator. The
+ * emulator will record all of the client connections. Then the test will 
check that only a single
+ * connection was used.
+ */
+@RunWith(JUnit4.class)
+public class BigtableSharedClientTest {
+
+  private FakeBigtable fakeService;
+  private ServerClientConnectionCounterInterceptor clientConnectionInterceptor;
+  private Server fakeServer;
+
+  @Before
+  public void setUp() throws Exception {
+    clientConnectionInterceptor = new 
ServerClientConnectionCounterInterceptor();
+    this.fakeService = new FakeBigtable();
+
+    IOException lastError = null;
+
+    for (int i = 0; i < 10; i++) {
+      try {
+        this.fakeServer = createServer(fakeService, 
clientConnectionInterceptor);
+        lastError = null;
+        break;
+      } catch (IOException e) {
+        lastError = e;
+      }
+    }
+    if (lastError != null) {
+      throw lastError;
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (fakeServer != null) {
+      fakeServer.shutdownNow();
+    }
+  }
+
+  private static Server createServer(BindableService service, 
ServerInterceptor... interceptors)
+      throws IOException {
+    int port;
+    try (ServerSocket ss = new ServerSocket(0)) {
+      port = ss.getLocalPort();
+    }
+
+    ServerBuilder<?> serverBuilder = 
ServerBuilder.forPort(port).addService(service);
+
+    for (ServerInterceptor interceptor : interceptors) {
+      serverBuilder.intercept(interceptor);
+    }
+    return serverBuilder.build().start();
+  }
+
+  @Test
+  public void testClientReusedAcrossBundles() {
+    PipelineOptions opts = PipelineOptionsFactory.create();
+    opts.setRunner(DirectRunner.class);
+    ExperimentalOptions.addExperiment(
+        opts.as(ExperimentalOptions.class),
+        String.format(
+            "%s=%s",
+            BigtableConfigTranslator.BIGTABLE_SETTINGS_OVERRIDE,
+            ClientSettingsOverride.class.getName()));
+
+    Pipeline pipeline = Pipeline.create(opts);
+
+    MutationsDoFn dofn = new MutationsDoFn();
+
+    pipeline
+        .apply(GenerateSequence.from(0).to(10_000))
+        .apply(ParDo.of(dofn)) // create Mutations & count bundles
+        .apply(
+            BigtableIO.write()
+                .withProjectId("fake-project")
+                .withInstanceId("fake-instance")
+                .withTableId("fake-table")
+                .withEmulator("localhost:" + fakeServer.getPort()));
+
+    assertThat(pipeline.run().waitUntilFinish(), Matchers.equalTo(State.DONE));
+    // Make sure that the test is valid by making sure that multiple bundles 
were processed
+    assertThat(MutationsDoFn.bundleCount.get(), Matchers.greaterThan(1));

Review Comment:
   is there anything to guarantee GenerateSequence will actually produce more 
than one bundle here?  It seems dependent on how the runner decides to execute 
it.  In my experience it can be difficult to make beam actually produce a 
specific number of bundles.
   
   I think the way I solved it was using processing time timers? I don't 
actually remember though.



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