[ 
https://issues.apache.org/jira/browse/BEAM-3973?focusedWorklogId=96547&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-96547
 ]

ASF GitHub Bot logged work on BEAM-3973:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 30/Apr/18 05:34
            Start Date: 30/Apr/18 05:34
    Worklog Time Spent: 10m 
      Work Description: chamikaramj closed pull request #4946: [BEAM-3973] Adds 
a parameter to the Cloud Spanner read connector that can disable batch API
URL: https://github.com/apache/beam/pull/4946
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java
 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java
new file mode 100644
index 00000000000..3b68d9f91a8
--- /dev/null
+++ 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java
@@ -0,0 +1,111 @@
+/*
+ * 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.spanner;
+
+import com.google.auto.value.AutoValue;
+import com.google.cloud.spanner.BatchReadOnlyTransaction;
+import com.google.cloud.spanner.ResultSet;
+import com.google.cloud.spanner.Struct;
+import com.google.cloud.spanner.TimestampBound;
+import com.google.common.annotations.VisibleForTesting;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+
+/** A naive version of Spanner read that doesn't use the Batch API. */
+@VisibleForTesting
+@AutoValue
+abstract class NaiveSpannerRead
+    extends PTransform<PCollection<ReadOperation>, PCollection<Struct>> {
+
+  public static NaiveSpannerRead create(SpannerConfig spannerConfig,
+      PCollectionView<Transaction> txView, TimestampBound timestampBound) {
+    return new AutoValue_NaiveSpannerRead(spannerConfig, txView, 
timestampBound);
+  }
+
+  abstract SpannerConfig getSpannerConfig();
+
+  @Nullable
+  abstract PCollectionView<Transaction> getTxView();
+
+  abstract TimestampBound getTimestampBound();
+
+  @Override
+  public PCollection<Struct> expand(PCollection<ReadOperation> input) {
+    PCollectionView<Transaction> txView = getTxView();
+    if (txView == null) {
+      Pipeline begin = input.getPipeline();
+      SpannerIO.CreateTransaction createTx = SpannerIO.createTransaction()
+          
.withSpannerConfig(getSpannerConfig()).withTimestampBound(getTimestampBound());
+      txView = begin.apply(createTx);
+    }
+
+    return input.apply("Naive read from Cloud Spanner",
+        ParDo.of(new NaiveSpannerReadFn(getSpannerConfig(), 
txView)).withSideInputs(txView));
+  }
+
+  private static class NaiveSpannerReadFn extends DoFn<ReadOperation, Struct> {
+
+    private final SpannerConfig config;
+    @Nullable private final PCollectionView<Transaction> txView;
+    private transient SpannerAccessor spannerAccessor;
+
+    NaiveSpannerReadFn(SpannerConfig config, @Nullable 
PCollectionView<Transaction> transaction) {
+      this.config = config;
+      this.txView = transaction;
+    }
+
+    @Setup
+    public void setup() throws Exception {
+      spannerAccessor = config.connectToSpanner();
+    }
+
+    @Teardown
+    public void teardown() throws Exception {
+      spannerAccessor.close();
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
+      Transaction tx = c.sideInput(txView);
+      ReadOperation op = c.element();
+      BatchReadOnlyTransaction context = spannerAccessor.getBatchClient()
+          .batchReadOnlyTransaction(tx.transactionId());
+      try (ResultSet resultSet = execute(op, context)) {
+        while (resultSet.next()) {
+          c.output(resultSet.getCurrentRowAsStruct());
+        }
+      }
+    }
+
+    private ResultSet execute(ReadOperation op, BatchReadOnlyTransaction 
readOnlyTransaction) {
+      if (op.getQuery() != null) {
+        return readOnlyTransaction.executeQuery(op.getQuery());
+      }
+      if (op.getIndex() != null) {
+        return readOnlyTransaction
+            .readUsingIndex(op.getTable(), op.getIndex(), op.getKeySet(), 
op.getColumns());
+      }
+      return readOnlyTransaction.read(op.getTable(), op.getKeySet(), 
op.getColumns());
+    }
+  }
+}
diff --git 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
index ce1e51d2e25..b4b542167d9 100644
--- 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
+++ 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
@@ -198,6 +198,7 @@ public static Read read() {
         .setSpannerConfig(SpannerConfig.create())
         .setTimestampBound(TimestampBound.strong())
         .setReadOperation(ReadOperation.create())
+        .setBatching(true)
         .build();
   }
 
@@ -210,6 +211,7 @@ public static ReadAll readAll() {
     return new AutoValue_SpannerIO_ReadAll.Builder()
         .setSpannerConfig(SpannerConfig.create())
         .setTimestampBound(TimestampBound.strong())
+        .setBatching(true)
         .build();
   }
 
@@ -264,6 +266,8 @@ public static Write write() {
 
       abstract Builder setTimestampBound(TimestampBound timestampBound);
 
+      abstract Builder setBatching(Boolean batching);
+
       abstract ReadAll build();
     }
 
@@ -329,12 +333,29 @@ public ReadAll withTimestampBound(TimestampBound 
timestampBound) {
       return toBuilder().setTimestampBound(timestampBound).build();
     }
 
+    /**
+     * By default Batch API is used to read data from Cloud Spanner.
+     * It is useful to disable batching when the underlying query is not 
root-partitionable.
+     */
+    public ReadAll withBatching(boolean batching) {
+      return toBuilder().setBatching(batching).build();
+    }
+
+    abstract Boolean getBatching();
+
     @Override
     public PCollection<Struct> expand(PCollection<ReadOperation> input) {
+      PTransform<PCollection<ReadOperation>, PCollection<Struct>> 
readTransform;
+      if (getBatching()) {
+        readTransform = BatchSpannerRead
+            .create(getSpannerConfig(), getTransaction(), getTimestampBound());
+      } else {
+        readTransform = NaiveSpannerRead
+            .create(getSpannerConfig(), getTransaction(), getTimestampBound());
+      }
       return input
-          .apply("Reshuffle", Reshuffle.<ReadOperation>viaRandomKey())
-          .apply("Read from Cloud Spanner",
-              BatchSpannerRead.create(getSpannerConfig(), getTransaction(), 
getTimestampBound()));
+          .apply("Reshuffle", Reshuffle.viaRandomKey())
+          .apply("Read from Cloud Spanner", readTransform);
     }
   }
 
@@ -356,6 +377,8 @@ public ReadAll withTimestampBound(TimestampBound 
timestampBound) {
     @Nullable
     abstract PartitionOptions getPartitionOptions();
 
+    abstract Boolean getBatching();
+
     abstract Builder toBuilder();
 
     @AutoValue.Builder
@@ -371,6 +394,8 @@ public ReadAll withTimestampBound(TimestampBound 
timestampBound) {
 
       abstract Builder setPartitionOptions(PartitionOptions partitionOptions);
 
+      abstract Builder setBatching(Boolean batching);
+
       abstract Read build();
     }
 
@@ -418,6 +443,11 @@ public Read withHost(String host) {
       return withSpannerConfig(config.withHost(host));
     }
 
+    /** If true the uses Cloud Spanner batch API. */
+    public Read withBatching(boolean batching) {
+      return toBuilder().setBatching(batching).build();
+    }
+
     @VisibleForTesting
     Read withServiceFactory(ServiceFactory<Spanner, SpannerOptions> 
serviceFactory) {
       SpannerConfig config = getSpannerConfig();
@@ -501,6 +531,7 @@ public Read withPartitionOptions(PartitionOptions 
partitionOptions) {
       ReadAll readAll = readAll()
           .withSpannerConfig(getSpannerConfig())
           .withTimestampBound(getTimestampBound())
+          .withBatching(getBatching())
           .withTransaction(getTransaction());
       return input.apply(Create.of(getReadOperation())).apply("Execute query", 
readAll);
     }
diff --git 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerReadIT.java
 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerReadIT.java
index cc3fb88183d..03e5eafd0ec 100644
--- 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerReadIT.java
+++ 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerReadIT.java
@@ -39,8 +39,11 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.testing.TestPipelineOptions;
 import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TypeDescriptor;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -108,32 +111,13 @@ public void setUp() throws Exception {
                     + "  Value         STRING(MAX),"
                     + ") PRIMARY KEY (Key)"));
     op.waitFor();
+    makeTestData();
   }
 
   @Test
   public void testRead() throws Exception {
-    DatabaseClient databaseClient =
-        spanner.getDatabaseClient(
-            DatabaseId.of(
-                project, options.getInstanceId(), databaseName));
 
-    List<Mutation> mutations = new ArrayList<>();
-    for (int i = 0; i < 5L; i++) {
-      mutations.add(
-          Mutation.newInsertOrUpdateBuilder(options.getTable())
-              .set("key")
-              .to((long) i)
-              .set("value")
-              .to(RandomUtils.randomAlphaNumeric(100))
-              .build());
-    }
-
-    databaseClient.writeAtLeastOnce(mutations);
-
-    SpannerConfig spannerConfig = SpannerConfig.create()
-        .withProjectId(project)
-        .withInstanceId(options.getInstanceId())
-        .withDatabaseId(databaseName);
+    SpannerConfig spannerConfig = createSpannerConfig();
 
     PCollectionView<Transaction> tx =
         p.apply(
@@ -154,10 +138,51 @@ public void testRead() throws Exception {
 
   @Test
   public void testQuery() throws Exception {
-    DatabaseClient databaseClient =
-        spanner.getDatabaseClient(
-            DatabaseId.of(
-                project, options.getInstanceId(), databaseName));
+    SpannerConfig spannerConfig = createSpannerConfig();
+
+    PCollectionView<Transaction> tx =
+        p.apply(
+            SpannerIO.createTransaction()
+                .withSpannerConfig(spannerConfig)
+                .withTimestampBound(TimestampBound.strong()));
+
+    PCollection<Struct> output =
+        p.apply(
+            SpannerIO.read()
+                .withSpannerConfig(spannerConfig)
+                .withQuery("SELECT * FROM " + options.getTable())
+                .withTransaction(tx));
+    PAssert.thatSingleton(output.apply("Count rows", 
Count.globally())).isEqualTo(5L);
+    p.run();
+  }
+
+  @Test
+  public void testReadAllRecordsInDb() throws Exception {
+    SpannerConfig spannerConfig = createSpannerConfig();
+
+    PCollectionView<Transaction> tx =
+        p.apply(
+            SpannerIO.createTransaction()
+                .withSpannerConfig(spannerConfig)
+                .withTimestampBound(TimestampBound.strong()));
+
+    PCollection<Struct> allRecords = p.apply(SpannerIO.read()
+        .withSpannerConfig(spannerConfig)
+        .withBatching(false)
+        .withQuery("SELECT t.table_name FROM information_schema.tables AS t 
WHERE t"
+            + ".table_catalog = '' AND t.table_schema = ''")).apply(
+        MapElements.into(TypeDescriptor.of(ReadOperation.class))
+            .via((SerializableFunction<Struct, ReadOperation>) input -> {
+              String tableName = input.getString(0);
+              return ReadOperation.create().withQuery("SELECT * FROM " + 
tableName);
+            
})).apply(SpannerIO.readAll().withTransaction(tx).withSpannerConfig(spannerConfig));
+
+    PAssert.thatSingleton(allRecords.apply("Count rows", 
Count.globally())).isEqualTo(5L);
+    p.run();
+  }
+
+  private void makeTestData() {
+    DatabaseClient databaseClient = getDatabaseClient();
 
     List<Mutation> mutations = new ArrayList<>();
     for (int i = 0; i < 5L; i++) {
@@ -171,26 +196,20 @@ public void testQuery() throws Exception {
     }
 
     databaseClient.writeAtLeastOnce(mutations);
+  }
 
-    SpannerConfig spannerConfig = SpannerConfig.create()
+  private SpannerConfig createSpannerConfig() {
+    return SpannerConfig.create()
         .withProjectId(project)
         .withInstanceId(options.getInstanceId())
         .withDatabaseId(databaseName);
+  }
 
-    PCollectionView<Transaction> tx =
-        p.apply(
-            SpannerIO.createTransaction()
-                .withSpannerConfig(spannerConfig)
-                .withTimestampBound(TimestampBound.strong()));
 
-    PCollection<Struct> output =
-        p.apply(
-            SpannerIO.read()
-                .withSpannerConfig(spannerConfig)
-                .withQuery("SELECT * FROM " + options.getTable())
-                .withTransaction(tx));
-    PAssert.thatSingleton(output.apply("Count rows", 
Count.globally())).isEqualTo(5L);
-    p.run();
+  private DatabaseClient getDatabaseClient() {
+    return spanner.getDatabaseClient(
+        DatabaseId.of(
+            project, options.getInstanceId(), databaseName));
   }
 
   @After


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 96547)
    Time Spent: 3h 40m  (was: 3.5h)

> Allow to disable batch API in SpannerIO
> ---------------------------------------
>
>                 Key: BEAM-3973
>                 URL: https://issues.apache.org/jira/browse/BEAM-3973
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-gcp
>    Affects Versions: 2.4.0
>            Reporter: Mairbek Khadikov
>            Assignee: Mairbek Khadikov
>            Priority: Major
>             Fix For: 2.5.0
>
>          Time Spent: 3h 40m
>  Remaining Estimate: 0h
>
> In 2.4.0, SpannerIO#read has been migrated to use batch API. The batch API 
> provides abstractions to scale out reads from Spanner, but it requires the 
> query to be root-partitionable. The root-partitionable queries cover majority 
> of the use cases, however there are examples when running arbitrary query is 
> useful. For example, reading all the table names from the 
> information_schema.* and reading the content of those tables in the next 
> step. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to