igorbernstein2 commented on code in PR #34245:
URL: https://github.com/apache/beam/pull/34245#discussion_r2054011380


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java:
##########
@@ -631,6 +631,24 @@ public Read withMaxBufferElementCount(@Nullable Integer 
maxBufferElementCount) {
           .build();
     }
 
+    /**
+     * Returns a new {@link BigtableIO.Read} that will skip the large rows 
while reading. This
+     * function will switch the base BigtableIO.Reader class to using the
+     * BigtableReaderWithExperimentalOptions. If
+     *
+     * <p>Does not modify this object.
+     *
+     * <p>This is incompatible with withMaxBufferElementCount()
+     */
+    public Read withExperimentalSkipLargeRows(@Nullable Boolean skipLargeRows) 
{

Review Comment:
   Please dont encode feature status in the public abi. The makes it impossible 
to evolve the feature into ga status. If this feature is not ready for GA, then 
please use
   
[ExperimentalOptions](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/options/ExperimentalOptions.html)
 which we already use for BIGTABLE_ENABLE_CLIENT_SIDE_METRICS
   



##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java:
##########
@@ -119,6 +119,27 @@ public void testE2EBigtableRead() {
     checkLineageSourceMetric(r, tableId);
   }
 
+  @Test
+  public void testE2EBigtableReadWithSkippingLargeRows() {
+    BigtableOptions.Builder bigtableOptionsBuilder =
+        new 
BigtableOptions.Builder().setProjectId(project).setInstanceId(options.getInstanceId());
+
+    final String tableId = "BigtableReadTest";
+    final long numRows = 1000L;

Review Comment:
   how does this test large row skipping?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java:
##########
@@ -174,10 +177,18 @@ public boolean start() throws IOException {
         query.filter(Filters.FILTERS.fromProto(rowFilter));
       }
       try {
-        stream =
-            client
-                .readRowsCallable(new BigtableRowProtoAdapter())
-                .call(query, GrpcCallContext.createDefault());
+        if (bigtableReadOptions != null
+            && 
Boolean.TRUE.equals(bigtableReadOptions.getExperimentalSkipLargeRows())) {
+          stream =
+              client
+                  .skipLargeRowsCallable(new BigtableRowProtoAdapter())
+                  .call(query, GrpcCallContext.createDefault());
+        } else {
+          stream =
+              client
+                  .readRowsCallable(new BigtableRowProtoAdapter())
+                  .call(query, GrpcCallContext.createDefault());
+        }

Review Comment:
   Resolving of the feature should be done during pipeline construction not 
during execution
   
   Also you should factor out the common code:
   
   ```java
   readRowsCallable = client.skipLargeRowsCallable(new 
BigtableRowProtoAdapter())
   if (isLargeRowSkippingEnabled) {
     readRowsCallable = client.readRowsCallable(new BigtableRowProtoAdapter());
   }
   readRowsCallable.call(...
   ```



-- 
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: github-unsubscr...@beam.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to