You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/07/06 18:51:07 UTC

[GitHub] [beam] danthev opened a new pull request, #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

danthev opened a new pull request, #22175:
URL: https://github.com/apache/beam/pull/22175

   This fixes a condition where, when a RunQuery request is retried, the cursor inferred from the last received value is set incorrectly, potentially yielding no or skipping some results. This can happen due to nested field paths in the orderBy or when the implicit order (e.g. by __name__) is not filled correctly. See also #22089.
   This PR fixes both root-causes along with unit test cases for each.
   
   ------------------------
   
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   
   


-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] clement commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
clement commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917203520


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -151,7 +169,7 @@ static List<String> resolveOrderByFieldPath(String fieldPath) {
       // Unquoted group is null, use quoted group.
       if (fieldName == null) {
         fieldName = segmentMatcher.group(2);
-        String escaped = escapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+        String escaped = unescapeFieldName(fieldName.substring(1, fieldName.length() - 1));

Review Comment:
   s/escaped/unescaped ?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();

Review Comment:
   > So I guess with partial progress response -> query times out -> no document to restart from, we'd have a problem?
   
   Yes, looks like it. Perhaps add an abstract reducer-like method to `StreamingFirestoreV1ReadFn`? Something with a signature like `OutT resumptionValue(OutT previousValue, OutT newValue)`?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }

Review Comment:
   > that the filters in a composite filter would already be sorted.
   
   The filter order in a composite filter is arbitrary, and picked by the user.
   
   > If not I'll add sorting to the composite filter case
   
   Composite filters can be nested, the sorting needs to be done for all field paths that appear at any point in the query filter "tree". I think it's also reasonable to not sort and only ensure that the results contains only one *canonical* field paths, otherwise fail with a clear message that **this** version of the beam connector does not support query with multiple inequalities (this is punting the issue for later, and hoping we will have a better mechanism for resumption by then).



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r919438297


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.Document;
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  /**
+   * Populates implicit orderBy of a query in accordance with our documentation. * Required
+   * inequality fields are appended in field name order. * __name__ is appended if not specified.
+   * See <a
+   * href=https://github.com/googleapis/googleapis/tree/master/google/firestore/v1/query.proto#L254>here</a>
+   * for more details.
+   *
+   * @param query The StructuredQuery of the original request.
+   * @return A list of additional orderBy fields, excluding the explicit ones.
+   */
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<OrderByFieldPath> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    Collections.sort(expectedImplicitOrders);
+    if (expectedImplicitOrders.stream().noneMatch(OrderByFieldPath::isDocumentName)) {
+      expectedImplicitOrders.add(OrderByFieldPath.fromString("__name__"));
+    }
+    for (Order order : query.getOrderByList()) {
+      OrderByFieldPath orderField = OrderByFieldPath.fromString(order.getField().getFieldPath());
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (OrderByFieldPath field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field.canonicalString()).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<OrderByFieldPath> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          OrderByFieldPath fieldPath =
+              OrderByFieldPath.fromString(filter.getFieldFilter().getField().getFieldPath());
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          OrderByFieldPath fieldPath =
+              OrderByFieldPath.fromString(filter.getUnaryFilter().getField().getFieldPath());
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  static @Nullable Value lookupDocumentValue(Document document, String fieldPath) {
+    OrderByFieldPath resolvedPath = OrderByFieldPath.fromString(fieldPath);
+    // __name__ is a special field and doesn't exist in (top-level) valueMap
+    if (resolvedPath.isDocumentName()) {
+      return Value.newBuilder().setReferenceValue(document.getName()).build();
+    }
+    return findMapValue(new ArrayList<>(resolvedPath.getSegments()), document.getFieldsMap());
+  }
+
+  private static @Nullable Value findMapValue(List<String> segments, Map<String, Value> valueMap) {
+    if (segments.isEmpty()) {
+      return null;
+    }
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return findMapValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static class OrderByFieldPath implements Comparable<OrderByFieldPath> {
+
+    public static OrderByFieldPath fromString(String fieldPath) {
+      if (fieldPath.isEmpty()) {
+        throw new IllegalArgumentException("Could not resolve empty field path");
+      }
+      List<String> segments = new ArrayList<>();
+      while (!fieldPath.isEmpty()) {
+        Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+        boolean foundMatch = segmentMatcher.lookingAt();
+        if (!foundMatch) {
+          throw new IllegalArgumentException("OrderBy field path was malformed");
+        }
+        String fieldName;
+        if ((fieldName = segmentMatcher.group(1)) != null) {
+          segments.add(fieldName);
+        } else if ((fieldName = segmentMatcher.group(2)) != null) {
+          String unescaped = unescapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+          segments.add(unescaped);
+        } else {
+          throw new IllegalArgumentException("OrderBy field path was malformed");
+        }
+        fieldPath = fieldPath.substring(fieldName.length());
+        // Due to the regex, any non-empty fieldPath will have a dot before the next nested field.
+        if (fieldPath.startsWith(".")) {
+          fieldPath = fieldPath.substring(1);
+        }
+      }
+      return new OrderByFieldPath(ImmutableList.copyOf(segments));
+    }
+
+    private final ImmutableList<String> segments;
+
+    private OrderByFieldPath(ImmutableList<String> segments) {
+      this.segments = segments;
+    }
+
+    public String canonicalString() {
+      return String.join(".", segments);
+    }
+
+    public boolean isDocumentName() {
+      return segments.size() == 1 && "__name__".equals(segments.get(0));
+    }
+
+    public ImmutableList<String> getSegments() {
+      return segments;
+    }
+
+    @Override
+    public boolean equals(@Nullable Object other) {
+      if (other instanceof OrderByFieldPath) {
+        return this.segments.equals(((OrderByFieldPath) other).getSegments());
+      }
+      return super.equals(other);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(segments);
+    }
+
+    @Override
+    public int compareTo(OrderByFieldPath other) {
+      // Taken from com.google.cloud.firestore.FieldPath.
+      int length = Math.min(this.getSegments().size(), other.getSegments().size());
+      for (int i = 0; i < length; i++) {
+        int cmp = this.getSegments().get(i).compareTo(other.getSegments().get(i));

Review Comment:
   So I guess the Firestore SDK FieldPath implementation is wrong? I've switched it to comparing UTF8 bytes.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] johnjcasey commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
johnjcasey commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917099010


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery
+          throw new IllegalArgumentException(
+              "Could not retry query due to malformed orderBy field", e.getCause());
+        }
+        // __name__ is a special field and doesn't exist in valueMap

Review Comment:
   Is this general firestore knowledge? if not, can we add some sort of documentation link here for the future?



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r919472639


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,402 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.Document;
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  /**
+   * Populates implicit orderBy of a query in accordance with our documentation. * Required
+   * inequality fields are appended in field name order. * __name__ is appended if not specified.
+   * See <a
+   * href=https://github.com/googleapis/googleapis/tree/master/google/firestore/v1/query.proto#L254>here</a>
+   * for more details.
+   *
+   * @param query The StructuredQuery of the original request.
+   * @return A list of additional orderBy fields, excluding the explicit ones.
+   */
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<OrderByFieldPath> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    Collections.sort(expectedImplicitOrders);
+    if (expectedImplicitOrders.stream().noneMatch(OrderByFieldPath::isDocumentName)) {
+      expectedImplicitOrders.add(OrderByFieldPath.fromString("__name__"));
+    }
+    for (Order order : query.getOrderByList()) {
+      OrderByFieldPath orderField = OrderByFieldPath.fromString(order.getField().getFieldPath());
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (OrderByFieldPath field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(
+                    FieldReference.newBuilder().setFieldPath(field.getOriginalString()).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<OrderByFieldPath> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          OrderByFieldPath fieldPath =
+              OrderByFieldPath.fromString(filter.getFieldFilter().getField().getFieldPath());
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          OrderByFieldPath fieldPath =
+              OrderByFieldPath.fromString(filter.getUnaryFilter().getField().getFieldPath());
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  static @Nullable Value lookupDocumentValue(Document document, String fieldPath) {
+    OrderByFieldPath resolvedPath = OrderByFieldPath.fromString(fieldPath);
+    // __name__ is a special field and doesn't exist in (top-level) valueMap (see
+    // https://firebase.google.com/docs/firestore/reference/rest/v1/projects.databases.documents#Document).
+    if (resolvedPath.isDocumentName()) {
+      return Value.newBuilder().setReferenceValue(document.getName()).build();
+    }
+    return findMapValue(new ArrayList<>(resolvedPath.getSegments()), document.getFieldsMap());
+  }
+
+  private static @Nullable Value findMapValue(List<String> segments, Map<String, Value> valueMap) {
+    if (segments.isEmpty()) {
+      return null;
+    }
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return findMapValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static class OrderByFieldPath implements Comparable<OrderByFieldPath> {
+
+    private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+    private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+    // After each segment follows a dot and more characters, or the end of the string.
+    private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+        Pattern.compile(
+            String.format(
+                "(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+            Pattern.DOTALL);
+
+    public static OrderByFieldPath fromString(String fieldPath) {
+      if (fieldPath.isEmpty()) {
+        throw new IllegalArgumentException("Could not resolve empty field path");
+      }
+      String originalString = fieldPath;
+      List<String> segments = new ArrayList<>();
+      while (!fieldPath.isEmpty()) {
+        Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+        boolean foundMatch = segmentMatcher.lookingAt();
+        if (!foundMatch) {
+          throw new IllegalArgumentException("OrderBy field path was malformed");
+        }
+        String fieldName;
+        if ((fieldName = segmentMatcher.group(1)) != null) {
+          segments.add(fieldName);
+        } else if ((fieldName = segmentMatcher.group(2)) != null) {
+          String unescaped = unescapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+          segments.add(unescaped);
+        } else {
+          throw new IllegalArgumentException("OrderBy field path was malformed");
+        }
+        fieldPath = fieldPath.substring(fieldName.length());
+        // Due to the regex, any non-empty fieldPath will have a dot before the next nested field.
+        if (fieldPath.startsWith(".")) {
+          fieldPath = fieldPath.substring(1);
+        }
+      }
+      return new OrderByFieldPath(originalString, ImmutableList.copyOf(segments));
+    }
+
+    private final String originalString;
+    private final ImmutableList<String> segments;
+
+    private OrderByFieldPath(String originalString, ImmutableList<String> segments) {
+      this.originalString = originalString;
+      this.segments = segments;
+    }
+
+    public String getOriginalString() {
+      return originalString;
+    }
+
+    public boolean isDocumentName() {
+      return segments.size() == 1 && "__name__".equals(segments.get(0));
+    }
+
+    public ImmutableList<String> getSegments() {
+      return segments;
+    }
+
+    @Override
+    public boolean equals(@Nullable Object other) {
+      if (other instanceof OrderByFieldPath) {
+        return this.segments.equals(((OrderByFieldPath) other).getSegments());
+      }
+      return super.equals(other);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(segments);
+    }
+
+    @Override
+    public int compareTo(OrderByFieldPath other) {
+      // Inspired by com.google.cloud.firestore.FieldPath.
+      int length = Math.min(this.getSegments().size(), other.getSegments().size());
+      for (int i = 0; i < length; i++) {
+        byte[] thisField = this.getSegments().get(i).getBytes(StandardCharsets.UTF_8);
+        byte[] otherField = other.getSegments().get(i).getBytes(StandardCharsets.UTF_8);
+        int bytes_length = Math.min(thisField.length, otherField.length);
+        for (int j = 0; j < bytes_length; j++) {
+          int cmp = Byte.compare(thisField[j], otherField[j]);

Review Comment:
   Done. Thanks!



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917026607


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1FnRunQueryTest.java:
##########
@@ -283,62 +299,86 @@ protected RunQueryFn getFn(
 
   private static final class TestData {
 
+    static final FieldReference FILTER_FIELD_PATH =
+        FieldReference.newBuilder().setFieldPath("foo").build();
+    static final Filter FIELD_EQUALS_BAR =
+        Filter.newBuilder()
+            .setFieldFilter(
+                FieldFilter.newBuilder()
+                    .setField(FILTER_FIELD_PATH)
+                    .setOp(Operator.EQUAL)
+                    .setValue(Value.newBuilder().setStringValue("bar"))
+                    .build())
+            .build();
+    static final Filter FIELD_NOT_EQUALS_FOO =
+        Filter.newBuilder()
+            .setFieldFilter(
+                FieldFilter.newBuilder()
+                    .setField(FILTER_FIELD_PATH)
+                    .setOp(Operator.NOT_EQUAL)
+                    .setValue(Value.newBuilder().setStringValue("foo"))
+                    .build())
+            .build();
+
     private final RunQueryRequest request;
     private final RunQueryResponse response1;
     private final RunQueryResponse response2;
     private final RunQueryResponse response3;
 
-    public TestData(String projectId, Function<FieldReference, List<Order>> orderFunction) {
-      String fieldPath = "foo";
-      FieldReference foo = FieldReference.newBuilder().setFieldPath(fieldPath).build();
+    public TestData(
+        String projectId, Function<FieldReference, List<Order>> orderFunction, Filter filter) {
       StructuredQuery.Builder builder =
           StructuredQuery.newBuilder()
               .addFrom(
                   CollectionSelector.newBuilder()
                       .setAllDescendants(false)
                       .setCollectionId("collection"))
-              .setWhere(
-                  Filter.newBuilder()
-                      .setFieldFilter(
-                          FieldFilter.newBuilder()
-                              .setField(foo)
-                              .setOp(Operator.EQUAL)
-                              .setValue(Value.newBuilder().setStringValue("bar"))
-                              .build()));
-
-      orderFunction.apply(foo).forEach(builder::addOrderBy);
+              .setWhere(filter);
+
+      orderFunction.apply(FILTER_FIELD_PATH).forEach(builder::addOrderBy);
       request =
           RunQueryRequest.newBuilder()
               .setParent(String.format("projects/%s/databases/(default)/document", projectId))
               .setStructuredQuery(builder)
               .build();
 
-      response1 = newResponse(fieldPath, 1);
-      response2 = newResponse(fieldPath, 2);
-      response3 = newResponse(fieldPath, 3);
+      response1 = newResponse(1);
+      response2 = newResponse(2);
+      response3 = newResponse(3);
     }
 
-    private static RunQueryResponse newResponse(String field, int docNumber) {
+    /**
+     * Returns single-document response like this: { "__name__": "doc-{docNumber}", "foo": "bar",
+     * "fo`o.m`ap": { "bar.key": "bar.val" } }

Review Comment:
   Done.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r918365836


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();

Review Comment:
   That's a great idea, just added that. 
   I saw BatchGetDocuments may also return no results, `setStartFrom` will request all documents in that case anyway, but I added an appropriate resumptionValue there so we don't even iterate.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917198127


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }

Review Comment:
   Done.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r916359229


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return lookupDocumentValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<String> resolveOrderByFieldPath(String fieldPath) throws IllegalArgumentException {
+    if (fieldPath.isEmpty()) {
+      throw new IllegalArgumentException("Could not resolve empty field path");
+    }
+    List<String> segments = new ArrayList<>();
+    while (!fieldPath.isEmpty()) {
+      Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+      boolean foundMatch = segmentMatcher.lookingAt();
+      if (!foundMatch) {
+        throw new IllegalArgumentException("OrderBy field path was malformed");
+      }
+      String fieldName = segmentMatcher.group(1);
+      // Unquoted group is null, use quoted group.
+      if (fieldName == null) {
+        fieldName = segmentMatcher.group(2);
+        String escaped = escapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+        segments.add(escaped);
+      } else {
+        segments.add(fieldName);
+      }
+      fieldPath = fieldPath.substring(fieldName.length());
+      if (fieldPath.startsWith(".")) {

Review Comment:
   Because the regex matched, fieldPath is either empty here, or it is `.nestedfieldname`, so I'm removing the dot. `if(!fieldPath.isEmpty())` would have the same effect, but I think it's probably less clear? 



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r916360120


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,37 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery
+          throw new IllegalArgumentException(
+              "Could not retry query due to malformed orderBy field", e.getCause());
+        }
+        // __name__ is a special field and doesn't exist in valueMap
+        if (segments.size() == 1 && "__name__".equals(segments.get(0))) {
+          cursor.addValues(
+              Value.newBuilder()
+                  .setReferenceValue(runQueryResponse.getDocument().getName())
+                  .build());
+        } else {
+          Value value;
+          value = QueryUtils.lookupDocumentValue(segments, valueMap);

Review Comment:
   Done.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return lookupDocumentValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<String> resolveOrderByFieldPath(String fieldPath) throws IllegalArgumentException {

Review Comment:
   Removed them.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return lookupDocumentValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";

Review Comment:
   Done.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);

Review Comment:
   Done.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r918365460


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }

Review Comment:
   I went ahead and implemented a class modeled after [FieldPath](https://github.com/googleapis/java-firestore/blob/main/google-cloud-firestore/src/main/java/com/google/cloud/firestore/FieldPath.java) with the extra escaping logic but without support for non-Where field paths. Just adding a compareTo there was easy, so the sorting is fully implemented now.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {

Review Comment:
   Done.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -151,7 +169,7 @@ static List<String> resolveOrderByFieldPath(String fieldPath) {
       // Unquoted group is null, use quoted group.
       if (fieldName == null) {
         fieldName = segmentMatcher.group(2);
-        String escaped = escapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+        String escaped = unescapeFieldName(fieldName.substring(1, fieldName.length() - 1));

Review Comment:
   Yes, done.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] johnjcasey commented on pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
johnjcasey commented on PR #22175:
URL: https://github.com/apache/beam/pull/22175#issuecomment-1179312821

   Nothing looks bad to me, but a lot of this is firestore specific, so its hard to judge from a BeamIO perspective


-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] clement commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
clement commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r919454067


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,402 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.Document;
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  /**
+   * Populates implicit orderBy of a query in accordance with our documentation. * Required
+   * inequality fields are appended in field name order. * __name__ is appended if not specified.
+   * See <a
+   * href=https://github.com/googleapis/googleapis/tree/master/google/firestore/v1/query.proto#L254>here</a>
+   * for more details.
+   *
+   * @param query The StructuredQuery of the original request.
+   * @return A list of additional orderBy fields, excluding the explicit ones.
+   */
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<OrderByFieldPath> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    Collections.sort(expectedImplicitOrders);
+    if (expectedImplicitOrders.stream().noneMatch(OrderByFieldPath::isDocumentName)) {
+      expectedImplicitOrders.add(OrderByFieldPath.fromString("__name__"));
+    }
+    for (Order order : query.getOrderByList()) {
+      OrderByFieldPath orderField = OrderByFieldPath.fromString(order.getField().getFieldPath());
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (OrderByFieldPath field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(
+                    FieldReference.newBuilder().setFieldPath(field.getOriginalString()).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<OrderByFieldPath> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          OrderByFieldPath fieldPath =
+              OrderByFieldPath.fromString(filter.getFieldFilter().getField().getFieldPath());
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          OrderByFieldPath fieldPath =
+              OrderByFieldPath.fromString(filter.getUnaryFilter().getField().getFieldPath());
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  static @Nullable Value lookupDocumentValue(Document document, String fieldPath) {
+    OrderByFieldPath resolvedPath = OrderByFieldPath.fromString(fieldPath);
+    // __name__ is a special field and doesn't exist in (top-level) valueMap (see
+    // https://firebase.google.com/docs/firestore/reference/rest/v1/projects.databases.documents#Document).
+    if (resolvedPath.isDocumentName()) {
+      return Value.newBuilder().setReferenceValue(document.getName()).build();
+    }
+    return findMapValue(new ArrayList<>(resolvedPath.getSegments()), document.getFieldsMap());
+  }
+
+  private static @Nullable Value findMapValue(List<String> segments, Map<String, Value> valueMap) {
+    if (segments.isEmpty()) {
+      return null;
+    }
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return findMapValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static class OrderByFieldPath implements Comparable<OrderByFieldPath> {
+
+    private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+    private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+    // After each segment follows a dot and more characters, or the end of the string.
+    private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+        Pattern.compile(
+            String.format(
+                "(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+            Pattern.DOTALL);
+
+    public static OrderByFieldPath fromString(String fieldPath) {
+      if (fieldPath.isEmpty()) {
+        throw new IllegalArgumentException("Could not resolve empty field path");
+      }
+      String originalString = fieldPath;
+      List<String> segments = new ArrayList<>();
+      while (!fieldPath.isEmpty()) {
+        Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+        boolean foundMatch = segmentMatcher.lookingAt();
+        if (!foundMatch) {
+          throw new IllegalArgumentException("OrderBy field path was malformed");
+        }
+        String fieldName;
+        if ((fieldName = segmentMatcher.group(1)) != null) {
+          segments.add(fieldName);
+        } else if ((fieldName = segmentMatcher.group(2)) != null) {
+          String unescaped = unescapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+          segments.add(unescaped);
+        } else {
+          throw new IllegalArgumentException("OrderBy field path was malformed");
+        }
+        fieldPath = fieldPath.substring(fieldName.length());
+        // Due to the regex, any non-empty fieldPath will have a dot before the next nested field.
+        if (fieldPath.startsWith(".")) {
+          fieldPath = fieldPath.substring(1);
+        }
+      }
+      return new OrderByFieldPath(originalString, ImmutableList.copyOf(segments));
+    }
+
+    private final String originalString;
+    private final ImmutableList<String> segments;
+
+    private OrderByFieldPath(String originalString, ImmutableList<String> segments) {
+      this.originalString = originalString;
+      this.segments = segments;
+    }
+
+    public String getOriginalString() {
+      return originalString;
+    }
+
+    public boolean isDocumentName() {
+      return segments.size() == 1 && "__name__".equals(segments.get(0));
+    }
+
+    public ImmutableList<String> getSegments() {
+      return segments;
+    }
+
+    @Override
+    public boolean equals(@Nullable Object other) {
+      if (other instanceof OrderByFieldPath) {
+        return this.segments.equals(((OrderByFieldPath) other).getSegments());
+      }
+      return super.equals(other);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(segments);
+    }
+
+    @Override
+    public int compareTo(OrderByFieldPath other) {
+      // Inspired by com.google.cloud.firestore.FieldPath.
+      int length = Math.min(this.getSegments().size(), other.getSegments().size());
+      for (int i = 0; i < length; i++) {
+        byte[] thisField = this.getSegments().get(i).getBytes(StandardCharsets.UTF_8);
+        byte[] otherField = other.getSegments().get(i).getBytes(StandardCharsets.UTF_8);
+        int bytes_length = Math.min(thisField.length, otherField.length);
+        for (int j = 0; j < bytes_length; j++) {
+          int cmp = Byte.compare(thisField[j], otherField[j]);

Review Comment:
   This looks like a signed byte comparator, which would be wrong (try a test with an ascii character vs a UTF8 multi-byte code point).
   
   If you have access to Guava's `UnsignedBytes.lexicographicalComparator` this can be used directly on the byte arrays returned by `getBytes`.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r919438611


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.Document;
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  /**
+   * Populates implicit orderBy of a query in accordance with our documentation. * Required
+   * inequality fields are appended in field name order. * __name__ is appended if not specified.
+   * See <a
+   * href=https://github.com/googleapis/googleapis/tree/master/google/firestore/v1/query.proto#L254>here</a>
+   * for more details.
+   *
+   * @param query The StructuredQuery of the original request.
+   * @return A list of additional orderBy fields, excluding the explicit ones.
+   */
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<OrderByFieldPath> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    Collections.sort(expectedImplicitOrders);
+    if (expectedImplicitOrders.stream().noneMatch(OrderByFieldPath::isDocumentName)) {
+      expectedImplicitOrders.add(OrderByFieldPath.fromString("__name__"));
+    }
+    for (Order order : query.getOrderByList()) {
+      OrderByFieldPath orderField = OrderByFieldPath.fromString(order.getField().getFieldPath());
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (OrderByFieldPath field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field.canonicalString()).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<OrderByFieldPath> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          OrderByFieldPath fieldPath =
+              OrderByFieldPath.fromString(filter.getFieldFilter().getField().getFieldPath());
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          OrderByFieldPath fieldPath =
+              OrderByFieldPath.fromString(filter.getUnaryFilter().getField().getFieldPath());
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  static @Nullable Value lookupDocumentValue(Document document, String fieldPath) {
+    OrderByFieldPath resolvedPath = OrderByFieldPath.fromString(fieldPath);
+    // __name__ is a special field and doesn't exist in (top-level) valueMap
+    if (resolvedPath.isDocumentName()) {
+      return Value.newBuilder().setReferenceValue(document.getName()).build();
+    }
+    return findMapValue(new ArrayList<>(resolvedPath.getSegments()), document.getFieldsMap());
+  }
+
+  private static @Nullable Value findMapValue(List<String> segments, Map<String, Value> valueMap) {
+    if (segments.isEmpty()) {
+      return null;
+    }
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return findMapValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static class OrderByFieldPath implements Comparable<OrderByFieldPath> {
+
+    public static OrderByFieldPath fromString(String fieldPath) {
+      if (fieldPath.isEmpty()) {
+        throw new IllegalArgumentException("Could not resolve empty field path");
+      }
+      List<String> segments = new ArrayList<>();
+      while (!fieldPath.isEmpty()) {
+        Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+        boolean foundMatch = segmentMatcher.lookingAt();
+        if (!foundMatch) {
+          throw new IllegalArgumentException("OrderBy field path was malformed");
+        }
+        String fieldName;
+        if ((fieldName = segmentMatcher.group(1)) != null) {
+          segments.add(fieldName);
+        } else if ((fieldName = segmentMatcher.group(2)) != null) {
+          String unescaped = unescapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+          segments.add(unescaped);
+        } else {
+          throw new IllegalArgumentException("OrderBy field path was malformed");
+        }
+        fieldPath = fieldPath.substring(fieldName.length());
+        // Due to the regex, any non-empty fieldPath will have a dot before the next nested field.
+        if (fieldPath.startsWith(".")) {
+          fieldPath = fieldPath.substring(1);
+        }
+      }
+      return new OrderByFieldPath(ImmutableList.copyOf(segments));
+    }
+
+    private final ImmutableList<String> segments;
+
+    private OrderByFieldPath(ImmutableList<String> segments) {
+      this.segments = segments;
+    }
+
+    public String canonicalString() {
+      return String.join(".", segments);

Review Comment:
   Ah yeah, added a test case that shows it. I've switched it to returning the original string.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on PR #22175:
URL: https://github.com/apache/beam/pull/22175#issuecomment-1176563739

   R: @yixiaoshen 
   R: @clement


-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r916233386


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1FnRunQueryTest.java:
##########
@@ -117,65 +119,62 @@ public void resumeFromLastReadValue() throws Exception {
                     .setStartAt(
                         Cursor.newBuilder()
                             .setBefore(false)
-                            .addValues(Value.newBuilder().setStringValue("bar"))))
+                            .addValues(Value.newBuilder().setStringValue("bar"))
+                            .addValues(
+                                Value.newBuilder()
+                                    .setReferenceValue(testData.response2.getDocument().getName()))
+                    )
+                    .addOrderBy(
+                        Order.newBuilder()
+                            .setField(FieldReference.newBuilder().setFieldPath("__name__"))
+                            .setDirection(Direction.ASCENDING))
+            )
             .build();
 
-    List<RunQueryResponse> responses =
-        ImmutableList.of(testData.response1, testData.response2, testData.response3);
-    when(responseStream1.iterator())
-        .thenReturn(
-            new AbstractIterator<RunQueryResponse>() {
-              private int invocationCount = 1;
-
-              @Override
-              protected RunQueryResponse computeNext() {
-                int count = invocationCount++;
-                if (count == 1) {
-                  return responses.get(0);
-                } else if (count == 2) {
-                  return responses.get(1);
-                } else {
-                  throw RETRYABLE_ERROR;
-                }
-              }
-            });
-
-    when(callable.call(testData.request)).thenReturn(responseStream1);
-    doNothing().when(attempt).checkCanRetry(any(), eq(RETRYABLE_ERROR));
-    when(responseStream2.iterator()).thenReturn(ImmutableList.of(responses.get(2)).iterator());
-    when(callable.call(request2)).thenReturn(responseStream2);
-
-    when(stub.runQueryCallable()).thenReturn(callable);
-
-    when(ff.getFirestoreStub(any())).thenReturn(stub);
-    when(ff.getRpcQos(any())).thenReturn(rpcQos);
-    when(rpcQos.newReadAttempt(any())).thenReturn(attempt);
-    when(attempt.awaitSafeToProceed(any())).thenReturn(true);
-
-    ArgumentCaptor<RunQueryResponse> responsesCaptor =
-        ArgumentCaptor.forClass(RunQueryResponse.class);
-
-    doNothing().when(processContext).output(responsesCaptor.capture());
-
-    when(processContext.element()).thenReturn(testData.request);
-
-    RunQueryFn fn = new RunQueryFn(clock, ff, rpcQosOptions);
-
-    runFunction(fn);
+    runQueryRetryTest(testData, expectedRetryRequest);
+  }
 
-    List<RunQueryResponse> allValues = responsesCaptor.getAllValues();
-    assertEquals(responses, allValues);
+  @Test
+  public void resumeFromLastReadValue_nestedOrderBy() throws Exception {

Review Comment:
   Implemented escaping logic to handle these cases.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] johnjcasey commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
johnjcasey commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917097819


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return lookupDocumentValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<String> resolveOrderByFieldPath(String fieldPath) throws IllegalArgumentException {
+    if (fieldPath.isEmpty()) {
+      throw new IllegalArgumentException("Could not resolve empty field path");
+    }
+    List<String> segments = new ArrayList<>();
+    while (!fieldPath.isEmpty()) {
+      Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+      boolean foundMatch = segmentMatcher.lookingAt();
+      if (!foundMatch) {
+        throw new IllegalArgumentException("OrderBy field path was malformed");
+      }
+      String fieldName = segmentMatcher.group(1);
+      // Unquoted group is null, use quoted group.
+      if (fieldName == null) {
+        fieldName = segmentMatcher.group(2);
+        String escaped = escapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+        segments.add(escaped);
+      } else {
+        segments.add(fieldName);
+      }
+      fieldPath = fieldPath.substring(fieldName.length());
+      if (fieldPath.startsWith(".")) {
+        fieldPath = fieldPath.substring(1);
+      }
+    }
+    return segments;
+  }
+
+  private static String escapeFieldName(String fieldName) throws IllegalArgumentException {
+    if (fieldName.isEmpty()) {

Review Comment:
   Agreed. A lot of this looks specific to firestore internals, so it would be nice to push it down, away from user space



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917198155


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery

Review Comment:
   Done.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r919440065


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtilsTest.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.firestore;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+import com.google.firestore.v1.Document;
+import com.google.firestore.v1.MapValue;
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.CollectionSelector;
+import com.google.firestore.v1.StructuredQuery.CompositeFilter;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import java.util.List;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.junit.Before;
+import org.junit.Test;
+
+public class QueryUtilsTest {
+
+  private Document testDocument;
+  private StructuredQuery testQuery;
+
+  @Before
+  public void setUp() {
+    // { "__name__": "doc-123", "fo`o.m`ap": { "bar.key": "bar.val" } }
+    testDocument =
+        Document.newBuilder()
+            .setName("doc-123")
+            .putAllFields(
+                ImmutableMap.of(
+                    "fo`o.m`ap",
+                    Value.newBuilder()
+                        .setMapValue(
+                            MapValue.newBuilder()
+                                .putFields(
+                                    "bar.key", Value.newBuilder().setStringValue("bar.val").build())
+                                .build())
+                        .build()))
+            .build();
+
+    // WHERE (z.a.a != "" AND b > "") AND c == "" AND z IS NOT NAN
+    Filter.Builder filter =
+        Filter.newBuilder()
+            .setCompositeFilter(
+                CompositeFilter.newBuilder()
+                    .addFilters(
+                        Filter.newBuilder()
+                            .setCompositeFilter(
+                                CompositeFilter.newBuilder()
+                                    .addFilters(
+                                        Filter.newBuilder()
+                                            .setFieldFilter(
+                                                FieldFilter.newBuilder()
+                                                    .setField(
+                                                        FieldReference.newBuilder()
+                                                            .setFieldPath("z.a.a"))
+                                                    .setOp(FieldFilter.Operator.NOT_EQUAL)
+                                                    .setValue(
+                                                        Value.newBuilder().setStringValue(""))))
+                                    .addFilters(
+                                        Filter.newBuilder()
+                                            .setFieldFilter(
+                                                FieldFilter.newBuilder()
+                                                    .setField(
+                                                        FieldReference.newBuilder()
+                                                            .setFieldPath("b"))
+                                                    .setOp(FieldFilter.Operator.GREATER_THAN)
+                                                    .setValue(
+                                                        Value.newBuilder().setStringValue(""))))
+                                    .setOp(CompositeFilter.Operator.AND)))
+                    .addFilters(
+                        Filter.newBuilder()
+                            .setFieldFilter(
+                                FieldFilter.newBuilder()
+                                    .setField(FieldReference.newBuilder().setFieldPath("c"))
+                                    .setOp(FieldFilter.Operator.EQUAL)
+                                    .setValue(Value.newBuilder().setStringValue(""))))
+                    .addFilters(
+                        Filter.newBuilder()
+                            .setUnaryFilter(
+                                UnaryFilter.newBuilder()
+                                    .setField(FieldReference.newBuilder().setFieldPath("z"))
+                                    .setOp(UnaryFilter.Operator.IS_NOT_NAN)))
+                    .setOp(CompositeFilter.Operator.AND)
+                    .build());
+    testQuery =
+        StructuredQuery.newBuilder()
+            .addFrom(
+                CollectionSelector.newBuilder()
+                    .setAllDescendants(false)
+                    .setCollectionId("collection"))
+            .setWhere(filter)
+            .addOrderBy(
+                Order.newBuilder()
+                    .setField(FieldReference.newBuilder().setFieldPath("b"))
+                    .setDirection(Direction.DESCENDING))
+            .build();
+  }
+
+  @Test
+  public void getImplicitOrderBy_success() {

Review Comment:
   \#3 was already in there, I've added another test case for \#2 and tweaked the main one to cover \#1 and \#4. 



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r915273632


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1FnRunQueryTest.java:
##########
@@ -117,65 +119,62 @@ public void resumeFromLastReadValue() throws Exception {
                     .setStartAt(
                         Cursor.newBuilder()
                             .setBefore(false)
-                            .addValues(Value.newBuilder().setStringValue("bar"))))
+                            .addValues(Value.newBuilder().setStringValue("bar"))
+                            .addValues(
+                                Value.newBuilder()
+                                    .setReferenceValue(testData.response2.getDocument().getName()))
+                    )
+                    .addOrderBy(
+                        Order.newBuilder()
+                            .setField(FieldReference.newBuilder().setFieldPath("__name__"))
+                            .setDirection(Direction.ASCENDING))
+            )
             .build();
 
-    List<RunQueryResponse> responses =
-        ImmutableList.of(testData.response1, testData.response2, testData.response3);
-    when(responseStream1.iterator())
-        .thenReturn(
-            new AbstractIterator<RunQueryResponse>() {
-              private int invocationCount = 1;
-
-              @Override
-              protected RunQueryResponse computeNext() {
-                int count = invocationCount++;
-                if (count == 1) {
-                  return responses.get(0);
-                } else if (count == 2) {
-                  return responses.get(1);
-                } else {
-                  throw RETRYABLE_ERROR;
-                }
-              }
-            });
-
-    when(callable.call(testData.request)).thenReturn(responseStream1);
-    doNothing().when(attempt).checkCanRetry(any(), eq(RETRYABLE_ERROR));
-    when(responseStream2.iterator()).thenReturn(ImmutableList.of(responses.get(2)).iterator());
-    when(callable.call(request2)).thenReturn(responseStream2);
-
-    when(stub.runQueryCallable()).thenReturn(callable);
-
-    when(ff.getFirestoreStub(any())).thenReturn(stub);
-    when(ff.getRpcQos(any())).thenReturn(rpcQos);
-    when(rpcQos.newReadAttempt(any())).thenReturn(attempt);
-    when(attempt.awaitSafeToProceed(any())).thenReturn(true);
-
-    ArgumentCaptor<RunQueryResponse> responsesCaptor =
-        ArgumentCaptor.forClass(RunQueryResponse.class);
-
-    doNothing().when(processContext).output(responsesCaptor.capture());
-
-    when(processContext.element()).thenReturn(testData.request);
-
-    RunQueryFn fn = new RunQueryFn(clock, ff, rpcQosOptions);
-
-    runFunction(fn);
+    runQueryRetryTest(testData, expectedRetryRequest);
+  }
 
-    List<RunQueryResponse> allValues = responsesCaptor.getAllValues();
-    assertEquals(responses, allValues);
+  @Test
+  public void resumeFromLastReadValue_nestedOrderBy() throws Exception {

Review Comment:
   Done. I've changed the nested test to use `foo_map.\`bar.key\``.



##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1FnRunQueryTest.java:
##########
@@ -117,65 +119,62 @@ public void resumeFromLastReadValue() throws Exception {
                     .setStartAt(
                         Cursor.newBuilder()
                             .setBefore(false)
-                            .addValues(Value.newBuilder().setStringValue("bar"))))
+                            .addValues(Value.newBuilder().setStringValue("bar"))
+                            .addValues(
+                                Value.newBuilder()
+                                    .setReferenceValue(testData.response2.getDocument().getName()))
+                    )
+                    .addOrderBy(
+                        Order.newBuilder()
+                            .setField(FieldReference.newBuilder().setFieldPath("__name__"))
+                            .setDirection(Direction.ASCENDING))
+            )
             .build();
 
-    List<RunQueryResponse> responses =
-        ImmutableList.of(testData.response1, testData.response2, testData.response3);
-    when(responseStream1.iterator())
-        .thenReturn(
-            new AbstractIterator<RunQueryResponse>() {
-              private int invocationCount = 1;
-
-              @Override
-              protected RunQueryResponse computeNext() {
-                int count = invocationCount++;
-                if (count == 1) {
-                  return responses.get(0);
-                } else if (count == 2) {
-                  return responses.get(1);
-                } else {
-                  throw RETRYABLE_ERROR;
-                }
-              }
-            });
-
-    when(callable.call(testData.request)).thenReturn(responseStream1);
-    doNothing().when(attempt).checkCanRetry(any(), eq(RETRYABLE_ERROR));
-    when(responseStream2.iterator()).thenReturn(ImmutableList.of(responses.get(2)).iterator());
-    when(callable.call(request2)).thenReturn(responseStream2);
-
-    when(stub.runQueryCallable()).thenReturn(callable);
-
-    when(ff.getFirestoreStub(any())).thenReturn(stub);
-    when(ff.getRpcQos(any())).thenReturn(rpcQos);
-    when(rpcQos.newReadAttempt(any())).thenReturn(attempt);
-    when(attempt.awaitSafeToProceed(any())).thenReturn(true);
-
-    ArgumentCaptor<RunQueryResponse> responsesCaptor =
-        ArgumentCaptor.forClass(RunQueryResponse.class);
-
-    doNothing().when(processContext).output(responsesCaptor.capture());
-
-    when(processContext.element()).thenReturn(testData.request);
-
-    RunQueryFn fn = new RunQueryFn(clock, ff, rpcQosOptions);
-
-    runFunction(fn);
+    runQueryRetryTest(testData, expectedRetryRequest);
+  }
 
-    List<RunQueryResponse> allValues = responsesCaptor.getAllValues();
-    assertEquals(responses, allValues);
+  @Test
+  public void resumeFromLastReadValue_nestedOrderBy() throws Exception {

Review Comment:
   Done. I've changed the nested test to use ```foo_map.`bar.key````.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] yixiaoshen commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
yixiaoshen commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r915278107


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1FnRunQueryTest.java:
##########
@@ -117,65 +119,62 @@ public void resumeFromLastReadValue() throws Exception {
                     .setStartAt(
                         Cursor.newBuilder()
                             .setBefore(false)
-                            .addValues(Value.newBuilder().setStringValue("bar"))))
+                            .addValues(Value.newBuilder().setStringValue("bar"))
+                            .addValues(
+                                Value.newBuilder()
+                                    .setReferenceValue(testData.response2.getDocument().getName()))
+                    )
+                    .addOrderBy(
+                        Order.newBuilder()
+                            .setField(FieldReference.newBuilder().setFieldPath("__name__"))
+                            .setDirection(Direction.ASCENDING))
+            )
             .build();
 
-    List<RunQueryResponse> responses =
-        ImmutableList.of(testData.response1, testData.response2, testData.response3);
-    when(responseStream1.iterator())
-        .thenReturn(
-            new AbstractIterator<RunQueryResponse>() {
-              private int invocationCount = 1;
-
-              @Override
-              protected RunQueryResponse computeNext() {
-                int count = invocationCount++;
-                if (count == 1) {
-                  return responses.get(0);
-                } else if (count == 2) {
-                  return responses.get(1);
-                } else {
-                  throw RETRYABLE_ERROR;
-                }
-              }
-            });
-
-    when(callable.call(testData.request)).thenReturn(responseStream1);
-    doNothing().when(attempt).checkCanRetry(any(), eq(RETRYABLE_ERROR));
-    when(responseStream2.iterator()).thenReturn(ImmutableList.of(responses.get(2)).iterator());
-    when(callable.call(request2)).thenReturn(responseStream2);
-
-    when(stub.runQueryCallable()).thenReturn(callable);
-
-    when(ff.getFirestoreStub(any())).thenReturn(stub);
-    when(ff.getRpcQos(any())).thenReturn(rpcQos);
-    when(rpcQos.newReadAttempt(any())).thenReturn(attempt);
-    when(attempt.awaitSafeToProceed(any())).thenReturn(true);
-
-    ArgumentCaptor<RunQueryResponse> responsesCaptor =
-        ArgumentCaptor.forClass(RunQueryResponse.class);
-
-    doNothing().when(processContext).output(responsesCaptor.capture());
-
-    when(processContext.element()).thenReturn(testData.request);
-
-    RunQueryFn fn = new RunQueryFn(clock, ff, rpcQosOptions);
-
-    runFunction(fn);
+    runQueryRetryTest(testData, expectedRetryRequest);
+  }
 
-    List<RunQueryResponse> allValues = responsesCaptor.getAllValues();
-    assertEquals(responses, allValues);
+  @Test
+  public void resumeFromLastReadValue_nestedOrderBy() throws Exception {

Review Comment:
   oh I didn't think about this case (that a back ticked field name is inside a nested map), I don't know what the code is going to do but seems that it's handling it just fine?



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r915273298


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1FnRunQueryTest.java:
##########
@@ -283,62 +292,86 @@ protected RunQueryFn getFn(
 
   private static final class TestData {
 
+    static final FieldReference FILTER_FIELD_PATH = FieldReference.newBuilder().setFieldPath("foo")
+        .build();
+
     private final RunQueryRequest request;
     private final RunQueryResponse response1;
     private final RunQueryResponse response2;
     private final RunQueryResponse response3;
 
-    public TestData(String projectId, Function<FieldReference, List<Order>> orderFunction) {
-      String fieldPath = "foo";
-      FieldReference foo = FieldReference.newBuilder().setFieldPath(fieldPath).build();
+    public TestData(String projectId, Function<FieldReference, List<Order>> orderFunction,
+        Filter filter) {
       StructuredQuery.Builder builder =
           StructuredQuery.newBuilder()
               .addFrom(
                   CollectionSelector.newBuilder()
                       .setAllDescendants(false)
                       .setCollectionId("collection"))
-              .setWhere(
-                  Filter.newBuilder()
-                      .setFieldFilter(
-                          FieldFilter.newBuilder()
-                              .setField(foo)
-                              .setOp(Operator.EQUAL)
-                              .setValue(Value.newBuilder().setStringValue("bar"))
-                              .build()));
-
-      orderFunction.apply(foo).forEach(builder::addOrderBy);
+              .setWhere(filter);
+
+      orderFunction.apply(FILTER_FIELD_PATH).forEach(builder::addOrderBy);
       request =
           RunQueryRequest.newBuilder()
               .setParent(String.format("projects/%s/databases/(default)/document", projectId))
               .setStructuredQuery(builder)
               .build();
 
-      response1 = newResponse(fieldPath, 1);
-      response2 = newResponse(fieldPath, 2);
-      response3 = newResponse(fieldPath, 3);
+      response1 = newResponse(1);
+      response2 = newResponse(2);
+      response3 = newResponse(3);
     }
 
-    private static RunQueryResponse newResponse(String field, int docNumber) {
+    /**
+     * Returns single-document response like this:
+     * {
+     *   "__name__": "doc-{docNumber}"
+     *   "foo": "bar"

Review Comment:
   I switched to `foo_map`, `bar.key` etc., that should be clearer I think.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917190859


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }

Review Comment:
   @yixiaoshen was saying that the filters in a composite filter would already be sorted. But since StructuredQuery is a raw proto, I'm guessing this would only be the case on the backend?
   If not I'll add sorting to the composite filter case, and the default case should probably throw a NotImplemented instead of a break.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r919258564


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery
+          throw new IllegalArgumentException(
+              "Could not retry query due to malformed orderBy field", e.getCause());
+        }
+        // __name__ is a special field and doesn't exist in valueMap

Review Comment:
   The [documentation](https://firebase.google.com/docs/firestore/reference/rest/v1/projects.databases.documents#Document) for `Document` says that __.*__ is reserved, so it sort of implies that __name__ can't be in there. I can link that here.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery
+          throw new IllegalArgumentException(
+              "Could not retry query due to malformed orderBy field", e.getCause());
+        }
+        // __name__ is a special field and doesn't exist in valueMap

Review Comment:
   The [documentation](https://firebase.google.com/docs/firestore/reference/rest/v1/projects.databases.documents#Document) for `Document` says that `__.*__` is reserved, so it sort of implies that __name__ can't be in there. I can link that here.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] chamikaramj merged pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
chamikaramj merged PR #22175:
URL: https://github.com/apache/beam/pull/22175


-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] johnjcasey commented on pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
johnjcasey commented on PR #22175:
URL: https://github.com/apache/beam/pull/22175#issuecomment-1184468293

   @pabloem @chamikaramj for final pass & merge


-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917105693


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return lookupDocumentValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<String> resolveOrderByFieldPath(String fieldPath) throws IllegalArgumentException {
+    if (fieldPath.isEmpty()) {
+      throw new IllegalArgumentException("Could not resolve empty field path");
+    }
+    List<String> segments = new ArrayList<>();
+    while (!fieldPath.isEmpty()) {
+      Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+      boolean foundMatch = segmentMatcher.lookingAt();
+      if (!foundMatch) {
+        throw new IllegalArgumentException("OrderBy field path was malformed");
+      }
+      String fieldName = segmentMatcher.group(1);
+      // Unquoted group is null, use quoted group.
+      if (fieldName == null) {
+        fieldName = segmentMatcher.group(2);
+        String escaped = escapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+        segments.add(escaped);
+      } else {
+        segments.add(fieldName);
+      }
+      fieldPath = fieldPath.substring(fieldName.length());
+      if (fieldPath.startsWith(".")) {
+        fieldPath = fieldPath.substring(1);
+      }
+    }
+    return segments;
+  }
+
+  private static String escapeFieldName(String fieldName) throws IllegalArgumentException {
+    if (fieldName.isEmpty()) {

Review Comment:
   Yes, this is a good idea not just for Beam but also to make it harder for users tp shoot themselves in the foot with the Rest API. We'll have an AI to implement this there, but I'll leave it in Beam for now.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917187222


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();

Review Comment:
   Hmm, that reporting of partial progress sounds like a potential failure case. Could you have another look at the implementation there around line 415? I think we might have to completely override `processElement` for RunQueryFn and only set lastReceivedValue if the response contained a document.
   Beam only checks if the response is null before calling `setStartFrom`, though it also only calls it if the ServerStream has ended in a RuntimeException. So I guess with partial progress response -> query times out -> no document to restart from, we'd have a problem? 



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] yixiaoshen commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
yixiaoshen commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r915191331


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1FnRunQueryTest.java:
##########
@@ -283,62 +292,86 @@ protected RunQueryFn getFn(
 
   private static final class TestData {
 
+    static final FieldReference FILTER_FIELD_PATH = FieldReference.newBuilder().setFieldPath("foo")
+        .build();
+
     private final RunQueryRequest request;
     private final RunQueryResponse response1;
     private final RunQueryResponse response2;
     private final RunQueryResponse response3;
 
-    public TestData(String projectId, Function<FieldReference, List<Order>> orderFunction) {
-      String fieldPath = "foo";
-      FieldReference foo = FieldReference.newBuilder().setFieldPath(fieldPath).build();
+    public TestData(String projectId, Function<FieldReference, List<Order>> orderFunction,
+        Filter filter) {
       StructuredQuery.Builder builder =
           StructuredQuery.newBuilder()
               .addFrom(
                   CollectionSelector.newBuilder()
                       .setAllDescendants(false)
                       .setCollectionId("collection"))
-              .setWhere(
-                  Filter.newBuilder()
-                      .setFieldFilter(
-                          FieldFilter.newBuilder()
-                              .setField(foo)
-                              .setOp(Operator.EQUAL)
-                              .setValue(Value.newBuilder().setStringValue("bar"))
-                              .build()));
-
-      orderFunction.apply(foo).forEach(builder::addOrderBy);
+              .setWhere(filter);
+
+      orderFunction.apply(FILTER_FIELD_PATH).forEach(builder::addOrderBy);
       request =
           RunQueryRequest.newBuilder()
               .setParent(String.format("projects/%s/databases/(default)/document", projectId))
               .setStructuredQuery(builder)
               .build();
 
-      response1 = newResponse(fieldPath, 1);
-      response2 = newResponse(fieldPath, 2);
-      response3 = newResponse(fieldPath, 3);
+      response1 = newResponse(1);
+      response2 = newResponse(2);
+      response3 = newResponse(3);
     }
 
-    private static RunQueryResponse newResponse(String field, int docNumber) {
+    /**
+     * Returns single-document response like this:
+     * {
+     *   "__name__": "doc-{docNumber}"
+     *   "foo": "bar"
+     *   "foobar": {"bar": "foo"}
+     * }
+     */
+    private static RunQueryResponse newResponse(int docNumber) {
       String docId = String.format("doc-%d", docNumber);
       return RunQueryResponse.newBuilder()
           .setDocument(
               Document.newBuilder()
                   .setName(docId)
                   .putAllFields(
-                      ImmutableMap.of(field, Value.newBuilder().setStringValue("bar").build()))
-                  .build())
+                      ImmutableMap.of(
+                          "foo", Value.newBuilder().setStringValue("bar").build(),
+                          "foobar", Value.newBuilder().setMapValue(MapValue.newBuilder()
+                              .putFields("bar", Value.newBuilder().setStringValue("foo").build())
+                              .build()).build()
+                      )))
           .build();
     }
 
     private static Builder fieldEqualsBar() {
-      return new Builder();
+      return new Builder().setFilter(Filter.newBuilder()
+          .setFieldFilter(
+              FieldFilter.newBuilder()
+                  .setField(FILTER_FIELD_PATH)
+                  .setOp(Operator.EQUAL)
+                  .setValue(Value.newBuilder().setStringValue("bar"))

Review Comment:
   maybe just declare two Filter constants FOO_EQUALS_BAR and FOO_NOT_EQUALS_FOO around line 297? there's no argument to these two methods so everytime these methods get called they just return the same filter, constants will do just as good as methods.



##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1FnRunQueryTest.java:
##########
@@ -117,65 +119,62 @@ public void resumeFromLastReadValue() throws Exception {
                     .setStartAt(
                         Cursor.newBuilder()
                             .setBefore(false)
-                            .addValues(Value.newBuilder().setStringValue("bar"))))
+                            .addValues(Value.newBuilder().setStringValue("bar"))
+                            .addValues(
+                                Value.newBuilder()
+                                    .setReferenceValue(testData.response2.getDocument().getName()))
+                    )
+                    .addOrderBy(
+                        Order.newBuilder()
+                            .setField(FieldReference.newBuilder().setFieldPath("__name__"))
+                            .setDirection(Direction.ASCENDING))
+            )
             .build();
 
-    List<RunQueryResponse> responses =
-        ImmutableList.of(testData.response1, testData.response2, testData.response3);
-    when(responseStream1.iterator())
-        .thenReturn(
-            new AbstractIterator<RunQueryResponse>() {
-              private int invocationCount = 1;
-
-              @Override
-              protected RunQueryResponse computeNext() {
-                int count = invocationCount++;
-                if (count == 1) {
-                  return responses.get(0);
-                } else if (count == 2) {
-                  return responses.get(1);
-                } else {
-                  throw RETRYABLE_ERROR;
-                }
-              }
-            });
-
-    when(callable.call(testData.request)).thenReturn(responseStream1);
-    doNothing().when(attempt).checkCanRetry(any(), eq(RETRYABLE_ERROR));
-    when(responseStream2.iterator()).thenReturn(ImmutableList.of(responses.get(2)).iterator());
-    when(callable.call(request2)).thenReturn(responseStream2);
-
-    when(stub.runQueryCallable()).thenReturn(callable);
-
-    when(ff.getFirestoreStub(any())).thenReturn(stub);
-    when(ff.getRpcQos(any())).thenReturn(rpcQos);
-    when(rpcQos.newReadAttempt(any())).thenReturn(attempt);
-    when(attempt.awaitSafeToProceed(any())).thenReturn(true);
-
-    ArgumentCaptor<RunQueryResponse> responsesCaptor =
-        ArgumentCaptor.forClass(RunQueryResponse.class);
-
-    doNothing().when(processContext).output(responsesCaptor.capture());
-
-    when(processContext.element()).thenReturn(testData.request);
-
-    RunQueryFn fn = new RunQueryFn(clock, ff, rpcQosOptions);
-
-    runFunction(fn);
+    runQueryRetryTest(testData, expectedRetryRequest);
+  }
 
-    List<RunQueryResponse> allValues = responsesCaptor.getAllValues();
-    assertEquals(responses, allValues);
+  @Test
+  public void resumeFromLastReadValue_nestedOrderBy() throws Exception {

Review Comment:
   maybe also add a test for backticked fields, e.g. the query has an inequality on field "\`x.y\`" or has an order by on field "\`x.y\`", and also add a field "x.y" to the returned document in newResponse() to support this test.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r915273632


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1FnRunQueryTest.java:
##########
@@ -117,65 +119,62 @@ public void resumeFromLastReadValue() throws Exception {
                     .setStartAt(
                         Cursor.newBuilder()
                             .setBefore(false)
-                            .addValues(Value.newBuilder().setStringValue("bar"))))
+                            .addValues(Value.newBuilder().setStringValue("bar"))
+                            .addValues(
+                                Value.newBuilder()
+                                    .setReferenceValue(testData.response2.getDocument().getName()))
+                    )
+                    .addOrderBy(
+                        Order.newBuilder()
+                            .setField(FieldReference.newBuilder().setFieldPath("__name__"))
+                            .setDirection(Direction.ASCENDING))
+            )
             .build();
 
-    List<RunQueryResponse> responses =
-        ImmutableList.of(testData.response1, testData.response2, testData.response3);
-    when(responseStream1.iterator())
-        .thenReturn(
-            new AbstractIterator<RunQueryResponse>() {
-              private int invocationCount = 1;
-
-              @Override
-              protected RunQueryResponse computeNext() {
-                int count = invocationCount++;
-                if (count == 1) {
-                  return responses.get(0);
-                } else if (count == 2) {
-                  return responses.get(1);
-                } else {
-                  throw RETRYABLE_ERROR;
-                }
-              }
-            });
-
-    when(callable.call(testData.request)).thenReturn(responseStream1);
-    doNothing().when(attempt).checkCanRetry(any(), eq(RETRYABLE_ERROR));
-    when(responseStream2.iterator()).thenReturn(ImmutableList.of(responses.get(2)).iterator());
-    when(callable.call(request2)).thenReturn(responseStream2);
-
-    when(stub.runQueryCallable()).thenReturn(callable);
-
-    when(ff.getFirestoreStub(any())).thenReturn(stub);
-    when(ff.getRpcQos(any())).thenReturn(rpcQos);
-    when(rpcQos.newReadAttempt(any())).thenReturn(attempt);
-    when(attempt.awaitSafeToProceed(any())).thenReturn(true);
-
-    ArgumentCaptor<RunQueryResponse> responsesCaptor =
-        ArgumentCaptor.forClass(RunQueryResponse.class);
-
-    doNothing().when(processContext).output(responsesCaptor.capture());
-
-    when(processContext.element()).thenReturn(testData.request);
-
-    RunQueryFn fn = new RunQueryFn(clock, ff, rpcQosOptions);
-
-    runFunction(fn);
+    runQueryRetryTest(testData, expectedRetryRequest);
+  }
 
-    List<RunQueryResponse> allValues = responsesCaptor.getAllValues();
-    assertEquals(responses, allValues);
+  @Test
+  public void resumeFromLastReadValue_nestedOrderBy() throws Exception {

Review Comment:
   Done. I've changed the nested test to use 
   ```
   foo_map.`bar.key`
   ```



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] clement commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
clement commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r919268771


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.Document;
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  /**
+   * Populates implicit orderBy of a query in accordance with our documentation. * Required
+   * inequality fields are appended in field name order. * __name__ is appended if not specified.
+   * See <a
+   * href=https://github.com/googleapis/googleapis/tree/master/google/firestore/v1/query.proto#L254>here</a>
+   * for more details.
+   *
+   * @param query The StructuredQuery of the original request.
+   * @return A list of additional orderBy fields, excluding the explicit ones.
+   */
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<OrderByFieldPath> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    Collections.sort(expectedImplicitOrders);
+    if (expectedImplicitOrders.stream().noneMatch(OrderByFieldPath::isDocumentName)) {
+      expectedImplicitOrders.add(OrderByFieldPath.fromString("__name__"));
+    }
+    for (Order order : query.getOrderByList()) {
+      OrderByFieldPath orderField = OrderByFieldPath.fromString(order.getField().getFieldPath());
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (OrderByFieldPath field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field.canonicalString()).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<OrderByFieldPath> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          OrderByFieldPath fieldPath =
+              OrderByFieldPath.fromString(filter.getFieldFilter().getField().getFieldPath());
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          OrderByFieldPath fieldPath =
+              OrderByFieldPath.fromString(filter.getUnaryFilter().getField().getFieldPath());
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  static @Nullable Value lookupDocumentValue(Document document, String fieldPath) {
+    OrderByFieldPath resolvedPath = OrderByFieldPath.fromString(fieldPath);
+    // __name__ is a special field and doesn't exist in (top-level) valueMap
+    if (resolvedPath.isDocumentName()) {
+      return Value.newBuilder().setReferenceValue(document.getName()).build();
+    }
+    return findMapValue(new ArrayList<>(resolvedPath.getSegments()), document.getFieldsMap());
+  }
+
+  private static @Nullable Value findMapValue(List<String> segments, Map<String, Value> valueMap) {
+    if (segments.isEmpty()) {
+      return null;
+    }
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return findMapValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static class OrderByFieldPath implements Comparable<OrderByFieldPath> {
+
+    public static OrderByFieldPath fromString(String fieldPath) {
+      if (fieldPath.isEmpty()) {
+        throw new IllegalArgumentException("Could not resolve empty field path");
+      }
+      List<String> segments = new ArrayList<>();
+      while (!fieldPath.isEmpty()) {
+        Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+        boolean foundMatch = segmentMatcher.lookingAt();
+        if (!foundMatch) {
+          throw new IllegalArgumentException("OrderBy field path was malformed");
+        }
+        String fieldName;
+        if ((fieldName = segmentMatcher.group(1)) != null) {
+          segments.add(fieldName);
+        } else if ((fieldName = segmentMatcher.group(2)) != null) {
+          String unescaped = unescapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+          segments.add(unescaped);
+        } else {
+          throw new IllegalArgumentException("OrderBy field path was malformed");
+        }
+        fieldPath = fieldPath.substring(fieldName.length());
+        // Due to the regex, any non-empty fieldPath will have a dot before the next nested field.
+        if (fieldPath.startsWith(".")) {
+          fieldPath = fieldPath.substring(1);
+        }
+      }
+      return new OrderByFieldPath(ImmutableList.copyOf(segments));
+    }
+
+    private final ImmutableList<String> segments;
+
+    private OrderByFieldPath(ImmutableList<String> segments) {
+      this.segments = segments;
+    }
+
+    public String canonicalString() {
+      return String.join(".", segments);
+    }
+
+    public boolean isDocumentName() {
+      return segments.size() == 1 && "__name__".equals(segments.get(0));
+    }
+
+    public ImmutableList<String> getSegments() {
+      return segments;
+    }
+
+    @Override
+    public boolean equals(@Nullable Object other) {
+      if (other instanceof OrderByFieldPath) {
+        return this.segments.equals(((OrderByFieldPath) other).getSegments());
+      }
+      return super.equals(other);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(segments);
+    }
+
+    @Override
+    public int compareTo(OrderByFieldPath other) {
+      // Taken from com.google.cloud.firestore.FieldPath.
+      int length = Math.min(this.getSegments().size(), other.getSegments().size());
+      for (int i = 0; i < length; i++) {
+        int cmp = this.getSegments().get(i).compareTo(other.getSegments().get(i));

Review Comment:
   UTF16 does not follow unicode code point ordering. Can we use something like the ICU4J library which has better support for unicode string comparison here? Or compare the UTF8 bytes.



##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtilsTest.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.firestore;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+import com.google.firestore.v1.Document;
+import com.google.firestore.v1.MapValue;
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.CollectionSelector;
+import com.google.firestore.v1.StructuredQuery.CompositeFilter;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import java.util.List;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.junit.Before;
+import org.junit.Test;
+
+public class QueryUtilsTest {
+
+  private Document testDocument;
+  private StructuredQuery testQuery;
+
+  @Before
+  public void setUp() {
+    // { "__name__": "doc-123", "fo`o.m`ap": { "bar.key": "bar.val" } }
+    testDocument =
+        Document.newBuilder()
+            .setName("doc-123")
+            .putAllFields(
+                ImmutableMap.of(
+                    "fo`o.m`ap",
+                    Value.newBuilder()
+                        .setMapValue(
+                            MapValue.newBuilder()
+                                .putFields(
+                                    "bar.key", Value.newBuilder().setStringValue("bar.val").build())
+                                .build())
+                        .build()))
+            .build();
+
+    // WHERE (z.a.a != "" AND b > "") AND c == "" AND z IS NOT NAN
+    Filter.Builder filter =
+        Filter.newBuilder()
+            .setCompositeFilter(
+                CompositeFilter.newBuilder()
+                    .addFilters(
+                        Filter.newBuilder()
+                            .setCompositeFilter(
+                                CompositeFilter.newBuilder()
+                                    .addFilters(
+                                        Filter.newBuilder()
+                                            .setFieldFilter(
+                                                FieldFilter.newBuilder()
+                                                    .setField(
+                                                        FieldReference.newBuilder()
+                                                            .setFieldPath("z.a.a"))
+                                                    .setOp(FieldFilter.Operator.NOT_EQUAL)
+                                                    .setValue(
+                                                        Value.newBuilder().setStringValue(""))))
+                                    .addFilters(
+                                        Filter.newBuilder()
+                                            .setFieldFilter(
+                                                FieldFilter.newBuilder()
+                                                    .setField(
+                                                        FieldReference.newBuilder()
+                                                            .setFieldPath("b"))
+                                                    .setOp(FieldFilter.Operator.GREATER_THAN)
+                                                    .setValue(
+                                                        Value.newBuilder().setStringValue(""))))
+                                    .setOp(CompositeFilter.Operator.AND)))
+                    .addFilters(
+                        Filter.newBuilder()
+                            .setFieldFilter(
+                                FieldFilter.newBuilder()
+                                    .setField(FieldReference.newBuilder().setFieldPath("c"))
+                                    .setOp(FieldFilter.Operator.EQUAL)
+                                    .setValue(Value.newBuilder().setStringValue(""))))
+                    .addFilters(
+                        Filter.newBuilder()
+                            .setUnaryFilter(
+                                UnaryFilter.newBuilder()
+                                    .setField(FieldReference.newBuilder().setFieldPath("z"))
+                                    .setOp(UnaryFilter.Operator.IS_NOT_NAN)))
+                    .setOp(CompositeFilter.Operator.AND)
+                    .build());
+    testQuery =
+        StructuredQuery.newBuilder()
+            .addFrom(
+                CollectionSelector.newBuilder()
+                    .setAllDescendants(false)
+                    .setCollectionId("collection"))
+            .setWhere(filter)
+            .addOrderBy(
+                Order.newBuilder()
+                    .setField(FieldReference.newBuilder().setFieldPath("b"))
+                    .setDirection(Direction.DESCENDING))
+            .build();
+  }
+
+  @Test
+  public void getImplicitOrderBy_success() {

Review Comment:
   This could use some extra test case, for example:
   - multiple equivalent field paths on inequalities (\`foo\` > 1 AND foo < 2)
   - inequalities on `__name__` + other field
   - orderby partially matching some inequalities / `__name__`
   - order by matching but different escaping: WHERE \`foo\` > 1 ORDER BY foo



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.Document;
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  /**
+   * Populates implicit orderBy of a query in accordance with our documentation. * Required
+   * inequality fields are appended in field name order. * __name__ is appended if not specified.
+   * See <a
+   * href=https://github.com/googleapis/googleapis/tree/master/google/firestore/v1/query.proto#L254>here</a>
+   * for more details.
+   *
+   * @param query The StructuredQuery of the original request.
+   * @return A list of additional orderBy fields, excluding the explicit ones.
+   */
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<OrderByFieldPath> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    Collections.sort(expectedImplicitOrders);
+    if (expectedImplicitOrders.stream().noneMatch(OrderByFieldPath::isDocumentName)) {
+      expectedImplicitOrders.add(OrderByFieldPath.fromString("__name__"));
+    }
+    for (Order order : query.getOrderByList()) {
+      OrderByFieldPath orderField = OrderByFieldPath.fromString(order.getField().getFieldPath());
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (OrderByFieldPath field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field.canonicalString()).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<OrderByFieldPath> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          OrderByFieldPath fieldPath =
+              OrderByFieldPath.fromString(filter.getFieldFilter().getField().getFieldPath());
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          OrderByFieldPath fieldPath =
+              OrderByFieldPath.fromString(filter.getUnaryFilter().getField().getFieldPath());
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  static @Nullable Value lookupDocumentValue(Document document, String fieldPath) {
+    OrderByFieldPath resolvedPath = OrderByFieldPath.fromString(fieldPath);
+    // __name__ is a special field and doesn't exist in (top-level) valueMap
+    if (resolvedPath.isDocumentName()) {
+      return Value.newBuilder().setReferenceValue(document.getName()).build();
+    }
+    return findMapValue(new ArrayList<>(resolvedPath.getSegments()), document.getFieldsMap());
+  }
+
+  private static @Nullable Value findMapValue(List<String> segments, Map<String, Value> valueMap) {
+    if (segments.isEmpty()) {
+      return null;
+    }
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return findMapValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static class OrderByFieldPath implements Comparable<OrderByFieldPath> {
+
+    public static OrderByFieldPath fromString(String fieldPath) {
+      if (fieldPath.isEmpty()) {
+        throw new IllegalArgumentException("Could not resolve empty field path");
+      }
+      List<String> segments = new ArrayList<>();
+      while (!fieldPath.isEmpty()) {
+        Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+        boolean foundMatch = segmentMatcher.lookingAt();
+        if (!foundMatch) {
+          throw new IllegalArgumentException("OrderBy field path was malformed");
+        }
+        String fieldName;
+        if ((fieldName = segmentMatcher.group(1)) != null) {
+          segments.add(fieldName);
+        } else if ((fieldName = segmentMatcher.group(2)) != null) {
+          String unescaped = unescapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+          segments.add(unescaped);
+        } else {
+          throw new IllegalArgumentException("OrderBy field path was malformed");
+        }
+        fieldPath = fieldPath.substring(fieldName.length());
+        // Due to the regex, any non-empty fieldPath will have a dot before the next nested field.
+        if (fieldPath.startsWith(".")) {
+          fieldPath = fieldPath.substring(1);
+        }
+      }
+      return new OrderByFieldPath(ImmutableList.copyOf(segments));
+    }
+
+    private final ImmutableList<String> segments;
+
+    private OrderByFieldPath(ImmutableList<String> segments) {
+      this.segments = segments;
+    }
+
+    public String canonicalString() {
+      return String.join(".", segments);

Review Comment:
   The segments must be escaped for this to be valid.
   
   Rather than also writing the code for escaping, one suggestion would be to save the escaped field path this instance was constructed from, and return it. It will not be the canonical escaped representation, but it will be valid.
   
   A unit test should show the issue. For ex a query with no order by and an inequality filter on `foo.bar`.bar would add foo.bar.baz to the order by, which is different.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917105693


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return lookupDocumentValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<String> resolveOrderByFieldPath(String fieldPath) throws IllegalArgumentException {
+    if (fieldPath.isEmpty()) {
+      throw new IllegalArgumentException("Could not resolve empty field path");
+    }
+    List<String> segments = new ArrayList<>();
+    while (!fieldPath.isEmpty()) {
+      Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+      boolean foundMatch = segmentMatcher.lookingAt();
+      if (!foundMatch) {
+        throw new IllegalArgumentException("OrderBy field path was malformed");
+      }
+      String fieldName = segmentMatcher.group(1);
+      // Unquoted group is null, use quoted group.
+      if (fieldName == null) {
+        fieldName = segmentMatcher.group(2);
+        String escaped = escapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+        segments.add(escaped);
+      } else {
+        segments.add(fieldName);
+      }
+      fieldPath = fieldPath.substring(fieldName.length());
+      if (fieldPath.startsWith(".")) {
+        fieldPath = fieldPath.substring(1);
+      }
+    }
+    return segments;
+  }
+
+  private static String escapeFieldName(String fieldName) throws IllegalArgumentException {
+    if (fieldName.isEmpty()) {

Review Comment:
   Yes, this is a good idea not just for Beam but also to make it harder for users to shoot themselves in the foot with the Rest API. We'll have an AI to implement this there, but I'll leave it in Beam for now.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on PR #22175:
URL: https://github.com/apache/beam/pull/22175#issuecomment-1179236713

   R: @johnjcasey 
   (yixiaoshen@ has approved, review by clem@ still pending)


-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917105693


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return lookupDocumentValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<String> resolveOrderByFieldPath(String fieldPath) throws IllegalArgumentException {
+    if (fieldPath.isEmpty()) {
+      throw new IllegalArgumentException("Could not resolve empty field path");
+    }
+    List<String> segments = new ArrayList<>();
+    while (!fieldPath.isEmpty()) {
+      Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+      boolean foundMatch = segmentMatcher.lookingAt();
+      if (!foundMatch) {
+        throw new IllegalArgumentException("OrderBy field path was malformed");
+      }
+      String fieldName = segmentMatcher.group(1);
+      // Unquoted group is null, use quoted group.
+      if (fieldName == null) {
+        fieldName = segmentMatcher.group(2);
+        String escaped = escapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+        segments.add(escaped);
+      } else {
+        segments.add(fieldName);
+      }
+      fieldPath = fieldPath.substring(fieldName.length());
+      if (fieldPath.startsWith(".")) {
+        fieldPath = fieldPath.substring(1);
+      }
+    }
+    return segments;
+  }
+
+  private static String escapeFieldName(String fieldName) throws IllegalArgumentException {
+    if (fieldName.isEmpty()) {

Review Comment:
   Yes, this is a good idea not just for Beam but also to make it harder for users tp shoot themselves in the foot with the Rest API. We'll have an AI to implement this there, but leave it in Beam for now.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917189489


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }

Review Comment:
   Ah, nice catch. Guess I'll have to unescape all these too.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917198087


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    if (segments.isEmpty()) {
+      return null;
+    }
+    String field = segments.remove(0);

Review Comment:
   Done, made it so RunQueryFn only calls `lookupDocumentValue(Document document, String fieldPath)`



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {

Review Comment:
   Will add on Monday.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917197965


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery
+          throw new IllegalArgumentException(
+              "Could not retry query due to malformed orderBy field", e.getCause());
+        }
+        // __name__ is a special field and doesn't exist in valueMap
+        if (segments.size() == 1 && "__name__".equals(segments.get(0))) {
+          cursor.addValues(
+              Value.newBuilder()
+                  .setReferenceValue(runQueryResponse.getDocument().getName())
+                  .build());
+        } else {

Review Comment:
   Done.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] yixiaoshen commented on pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
yixiaoshen commented on PR #22175:
URL: https://github.com/apache/beam/pull/22175#issuecomment-1178400072

   LGTM. please also get a review from someone else.


-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r915272851


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1FnRunQueryTest.java:
##########
@@ -283,62 +292,86 @@ protected RunQueryFn getFn(
 
   private static final class TestData {
 
+    static final FieldReference FILTER_FIELD_PATH = FieldReference.newBuilder().setFieldPath("foo")
+        .build();
+
     private final RunQueryRequest request;
     private final RunQueryResponse response1;
     private final RunQueryResponse response2;
     private final RunQueryResponse response3;
 
-    public TestData(String projectId, Function<FieldReference, List<Order>> orderFunction) {
-      String fieldPath = "foo";
-      FieldReference foo = FieldReference.newBuilder().setFieldPath(fieldPath).build();
+    public TestData(String projectId, Function<FieldReference, List<Order>> orderFunction,
+        Filter filter) {
       StructuredQuery.Builder builder =
           StructuredQuery.newBuilder()
               .addFrom(
                   CollectionSelector.newBuilder()
                       .setAllDescendants(false)
                       .setCollectionId("collection"))
-              .setWhere(
-                  Filter.newBuilder()
-                      .setFieldFilter(
-                          FieldFilter.newBuilder()
-                              .setField(foo)
-                              .setOp(Operator.EQUAL)
-                              .setValue(Value.newBuilder().setStringValue("bar"))
-                              .build()));
-
-      orderFunction.apply(foo).forEach(builder::addOrderBy);
+              .setWhere(filter);
+
+      orderFunction.apply(FILTER_FIELD_PATH).forEach(builder::addOrderBy);
       request =
           RunQueryRequest.newBuilder()
               .setParent(String.format("projects/%s/databases/(default)/document", projectId))
               .setStructuredQuery(builder)
               .build();
 
-      response1 = newResponse(fieldPath, 1);
-      response2 = newResponse(fieldPath, 2);
-      response3 = newResponse(fieldPath, 3);
+      response1 = newResponse(1);
+      response2 = newResponse(2);
+      response3 = newResponse(3);
     }
 
-    private static RunQueryResponse newResponse(String field, int docNumber) {
+    /**
+     * Returns single-document response like this:
+     * {
+     *   "__name__": "doc-{docNumber}"
+     *   "foo": "bar"
+     *   "foobar": {"bar": "foo"}
+     * }
+     */
+    private static RunQueryResponse newResponse(int docNumber) {
       String docId = String.format("doc-%d", docNumber);
       return RunQueryResponse.newBuilder()
           .setDocument(
               Document.newBuilder()
                   .setName(docId)
                   .putAllFields(
-                      ImmutableMap.of(field, Value.newBuilder().setStringValue("bar").build()))
-                  .build())
+                      ImmutableMap.of(
+                          "foo", Value.newBuilder().setStringValue("bar").build(),
+                          "foobar", Value.newBuilder().setMapValue(MapValue.newBuilder()
+                              .putFields("bar", Value.newBuilder().setStringValue("foo").build())
+                              .build()).build()
+                      )))
           .build();
     }
 
     private static Builder fieldEqualsBar() {
-      return new Builder();
+      return new Builder().setFilter(Filter.newBuilder()
+          .setFieldFilter(
+              FieldFilter.newBuilder()
+                  .setField(FILTER_FIELD_PATH)
+                  .setOp(Operator.EQUAL)
+                  .setValue(Value.newBuilder().setStringValue("bar"))

Review Comment:
   Sure why not. Done.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917026753


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return lookupDocumentValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<String> resolveOrderByFieldPath(String fieldPath) throws IllegalArgumentException {
+    if (fieldPath.isEmpty()) {
+      throw new IllegalArgumentException("Could not resolve empty field path");
+    }
+    List<String> segments = new ArrayList<>();
+    while (!fieldPath.isEmpty()) {
+      Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+      boolean foundMatch = segmentMatcher.lookingAt();
+      if (!foundMatch) {
+        throw new IllegalArgumentException("OrderBy field path was malformed");
+      }
+      String fieldName = segmentMatcher.group(1);
+      // Unquoted group is null, use quoted group.
+      if (fieldName == null) {
+        fieldName = segmentMatcher.group(2);
+        String escaped = escapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+        segments.add(escaped);
+      } else {
+        segments.add(fieldName);
+      }
+      fieldPath = fieldPath.substring(fieldName.length());
+      if (fieldPath.startsWith(".")) {

Review Comment:
   Done.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] clement commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
clement commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917167502


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery
+          throw new IllegalArgumentException(
+              "Could not retry query due to malformed orderBy field", e.getCause());
+        }
+        // __name__ is a special field and doesn't exist in valueMap

Review Comment:
   That's not actually quite right. Some legacy types in Datastore will translate to maps in Firestore with the `__name__` field set.
   
   But it is true that `__name__` will never be a top-level field in a document.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    if (segments.isEmpty()) {
+      return null;
+    }
+    String field = segments.remove(0);

Review Comment:
   The fact that this (non-private) method mutates its argument is a really sharp corner. A couple of possible ways out of that:
   
   - make it and `resolveOrderByFieldPath` private, and expose only a single method that returns the `Value` for a `Document` by field-path, with the path given as a string.
   - pass the current index of the segment as an extra parameter



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();

Review Comment:
   The document may not be set on all RunQueryResponse (see https://cloud.google.com/firestore/docs/reference/rpc/google.firestore.v1#runqueryresponse)
   
   Are the callers of `setStartFrom` ensuring that this is set in the messages they are passing?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery
+          throw new IllegalArgumentException(
+              "Could not retry query due to malformed orderBy field", e.getCause());

Review Comment:
   This message may be a bit misleading. I assume that a call to `setStartFrom` is predicated on a successful initial RunQuery RPC with the query given by the user, so the order by *they* specified is valid, and either the segment parsing code or the implicit completion of the order by is wrong.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {

Review Comment:
   This class could use unit-tests.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery

Review Comment:
   This does not add anything useful in the error message that the strack trace does not already provide.
   
   Either do not rethrow, or add relevant context for debugging (for example, the original query + modified order by).



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }

Review Comment:
   The contract of the implicit order by is that inequality fields must be added in field-path-order (which is lexicographical order of the segment list, and segments are unicode lexicographical ordering).
   
   If/when Firestore starts supporting multiple inequalities, this code will become silently incorrect. Either this should properly order addition to the order by, or this should hard-fail with inequalities on more than one field path.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }

Review Comment:
   Equality test should be done on unescaped paths (segment lists) because the escaped form is not canonical.
   
   Consider the following test cases, that would not pass with the current code:
   
   ```
   ... WHERE `a` > 1 ORDER BY a  -->  ORDER BY a, __name__
   ... WHERE `a` > 1 AND a < 3  -->  ORDER BY a, __name__
   ... ORDER BY `__name__` --> ORDER BY `__name__`
   ... WHERE `__name__` > ".../foo/bar" --> ORDER BY __name__
   ```



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery
+          throw new IllegalArgumentException(
+              "Could not retry query due to malformed orderBy field", e.getCause());
+        }
+        // __name__ is a special field and doesn't exist in valueMap
+        if (segments.size() == 1 && "__name__".equals(segments.get(0))) {
+          cursor.addValues(
+              Value.newBuilder()
+                  .setReferenceValue(runQueryResponse.getDocument().getName())
+                  .build());
+        } else {

Review Comment:
   Worth putting this logic in a new method `QueryUtils.lookupDocumentValue(segments, runQueryResponse.getDocument())` ?



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917187916


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery
+          throw new IllegalArgumentException(
+              "Could not retry query due to malformed orderBy field", e.getCause());

Review Comment:
   You're right, this is basically a "this should never happen". I'll remove the rethrow altogether.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r917197901


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery
+          throw new IllegalArgumentException(
+              "Could not retry query due to malformed orderBy field", e.getCause());
+        }
+        // __name__ is a special field and doesn't exist in valueMap

Review Comment:
   Updated comment to be explicit that this is in the top-level valueMap.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,36 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery
+          throw new IllegalArgumentException(
+              "Could not retry query due to malformed orderBy field", e.getCause());

Review Comment:
   Done.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] yixiaoshen commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
yixiaoshen commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r915186619


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1FnRunQueryTest.java:
##########
@@ -283,62 +292,86 @@ protected RunQueryFn getFn(
 
   private static final class TestData {
 
+    static final FieldReference FILTER_FIELD_PATH = FieldReference.newBuilder().setFieldPath("foo")
+        .build();
+
     private final RunQueryRequest request;
     private final RunQueryResponse response1;
     private final RunQueryResponse response2;
     private final RunQueryResponse response3;
 
-    public TestData(String projectId, Function<FieldReference, List<Order>> orderFunction) {
-      String fieldPath = "foo";
-      FieldReference foo = FieldReference.newBuilder().setFieldPath(fieldPath).build();
+    public TestData(String projectId, Function<FieldReference, List<Order>> orderFunction,
+        Filter filter) {
       StructuredQuery.Builder builder =
           StructuredQuery.newBuilder()
               .addFrom(
                   CollectionSelector.newBuilder()
                       .setAllDescendants(false)
                       .setCollectionId("collection"))
-              .setWhere(
-                  Filter.newBuilder()
-                      .setFieldFilter(
-                          FieldFilter.newBuilder()
-                              .setField(foo)
-                              .setOp(Operator.EQUAL)
-                              .setValue(Value.newBuilder().setStringValue("bar"))
-                              .build()));
-
-      orderFunction.apply(foo).forEach(builder::addOrderBy);
+              .setWhere(filter);
+
+      orderFunction.apply(FILTER_FIELD_PATH).forEach(builder::addOrderBy);
       request =
           RunQueryRequest.newBuilder()
               .setParent(String.format("projects/%s/databases/(default)/document", projectId))
               .setStructuredQuery(builder)
               .build();
 
-      response1 = newResponse(fieldPath, 1);
-      response2 = newResponse(fieldPath, 2);
-      response3 = newResponse(fieldPath, 3);
+      response1 = newResponse(1);
+      response2 = newResponse(2);
+      response3 = newResponse(3);
     }
 
-    private static RunQueryResponse newResponse(String field, int docNumber) {
+    /**
+     * Returns single-document response like this:
+     * {
+     *   "__name__": "doc-{docNumber}"
+     *   "foo": "bar"

Review Comment:
   nit: since the field names are using "foo", "foobar", "bar", I'd suggest changing field values to something else instead of "bar" and "foo" to avoid confusion between field names and field values.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] yixiaoshen commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
yixiaoshen commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r916315190


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return lookupDocumentValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<String> resolveOrderByFieldPath(String fieldPath) throws IllegalArgumentException {
+    if (fieldPath.isEmpty()) {
+      throw new IllegalArgumentException("Could not resolve empty field path");
+    }
+    List<String> segments = new ArrayList<>();
+    while (!fieldPath.isEmpty()) {
+      Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+      boolean foundMatch = segmentMatcher.lookingAt();
+      if (!foundMatch) {
+        throw new IllegalArgumentException("OrderBy field path was malformed");
+      }
+      String fieldName = segmentMatcher.group(1);
+      // Unquoted group is null, use quoted group.
+      if (fieldName == null) {
+        fieldName = segmentMatcher.group(2);
+        String escaped = escapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+        segments.add(escaped);
+      } else {
+        segments.add(fieldName);
+      }
+      fieldPath = fieldPath.substring(fieldName.length());
+      if (fieldPath.startsWith(".")) {
+        fieldPath = fieldPath.substring(1);
+      }
+    }
+    return segments;
+  }
+
+  private static String escapeFieldName(String fieldName) throws IllegalArgumentException {
+    if (fieldName.isEmpty()) {

Review Comment:
   I feel this should ideally be added as some class in googleapis/java-firestore and imported by beam as a library, in that way we will have better control over this logic and less likely to have it diverge from our backend (not that this logic is changed that frequently though). I guess it's OK as a temporary quick fix.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);

Review Comment:
   nit: potential IndexOutOfBoundsException if the list is empty, given the method is not public the risk should be small. Maybe still worth adding an empty check before trying to remove from the list?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return lookupDocumentValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";

Review Comment:
   nit: consider moving all constants to before all methods in the class



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1ReadFn.java:
##########
@@ -109,44 +107,37 @@ protected ServerStreamingCallable<RunQueryRequest, RunQueryResponse> getCallable
     protected RunQueryRequest setStartFrom(
         RunQueryRequest element, RunQueryResponse runQueryResponse) {
       StructuredQuery query = element.getStructuredQuery();
-      StructuredQuery.Builder builder;
-      List<Order> orderByList = query.getOrderByList();
-      // if the orderByList is empty that means the default sort of "__name__ ASC" will be used
-      // Before we can set the cursor to the last document name read, we need to explicitly add
-      // the order of "__name__ ASC" because a cursor value must map to an order by
-      if (orderByList.isEmpty()) {
-        builder =
-            query
-                .toBuilder()
-                .addOrderBy(
-                    Order.newBuilder()
-                        .setField(FieldReference.newBuilder().setFieldPath("__name__").build())
-                        .setDirection(Direction.ASCENDING)
-                        .build())
-                .setStartAt(
-                    Cursor.newBuilder()
-                        .setBefore(false)
-                        .addValues(
-                            Value.newBuilder()
-                                .setReferenceValue(runQueryResponse.getDocument().getName())
-                                .build()));
-      } else {
-        Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
-        Map<String, Value> fieldsMap = runQueryResponse.getDocument().getFieldsMap();
-        for (Order order : orderByList) {
-          String fieldPath = order.getField().getFieldPath();
-          Value value = fieldsMap.get(fieldPath);
-          if (value != null) {
-            cursor.addValues(value);
-          } else if ("__name__".equals(fieldPath)) {
-            cursor.addValues(
-                Value.newBuilder()
-                    .setReferenceValue(runQueryResponse.getDocument().getName())
-                    .build());
+      StructuredQuery.Builder builder = query.toBuilder();
+      builder.addAllOrderBy(QueryUtils.getImplicitOrderBy(query));
+      Cursor.Builder cursor = Cursor.newBuilder().setBefore(false);
+
+      Map<String, Value> valueMap = runQueryResponse.getDocument().getFieldsMap();
+      for (Order order : builder.getOrderByList()) {
+        List<String> segments;
+        try {
+          segments = QueryUtils.resolveOrderByFieldPath(order.getField().getFieldPath());
+        } catch (IllegalArgumentException e) {
+          // Rethrow as something specific to RunQuery
+          throw new IllegalArgumentException(
+              "Could not retry query due to malformed orderBy field", e.getCause());
+        }
+        // __name__ is a special field and doesn't exist in valueMap
+        if (segments.size() == 1 && "__name__".equals(segments.get(0))) {
+          cursor.addValues(
+              Value.newBuilder()
+                  .setReferenceValue(runQueryResponse.getDocument().getName())
+                  .build());
+        } else {
+          Value value;
+          value = QueryUtils.lookupDocumentValue(segments, valueMap);

Review Comment:
   nit: can we combine these two lines?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return lookupDocumentValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<String> resolveOrderByFieldPath(String fieldPath) throws IllegalArgumentException {
+    if (fieldPath.isEmpty()) {
+      throw new IllegalArgumentException("Could not resolve empty field path");
+    }
+    List<String> segments = new ArrayList<>();
+    while (!fieldPath.isEmpty()) {
+      Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+      boolean foundMatch = segmentMatcher.lookingAt();
+      if (!foundMatch) {
+        throw new IllegalArgumentException("OrderBy field path was malformed");
+      }
+      String fieldName = segmentMatcher.group(1);
+      // Unquoted group is null, use quoted group.
+      if (fieldName == null) {
+        fieldName = segmentMatcher.group(2);
+        String escaped = escapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+        segments.add(escaped);
+      } else {
+        segments.add(fieldName);
+      }
+      fieldPath = fieldPath.substring(fieldName.length());
+      if (fieldPath.startsWith(".")) {

Review Comment:
   this line looks a bit suspicious, when fieldPath is not empty here, when would it start with "." and when would it not start with "."?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return lookupDocumentValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<String> resolveOrderByFieldPath(String fieldPath) throws IllegalArgumentException {

Review Comment:
   RuntimeExceptions do not need to be declared in the throws clause. 



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] yixiaoshen commented on a diff in pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
yixiaoshen commented on code in PR #22175:
URL: https://github.com/apache/beam/pull/22175#discussion_r916367451


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/firestore/FirestoreV1FnRunQueryTest.java:
##########
@@ -283,62 +299,86 @@ protected RunQueryFn getFn(
 
   private static final class TestData {
 
+    static final FieldReference FILTER_FIELD_PATH =
+        FieldReference.newBuilder().setFieldPath("foo").build();
+    static final Filter FIELD_EQUALS_BAR =
+        Filter.newBuilder()
+            .setFieldFilter(
+                FieldFilter.newBuilder()
+                    .setField(FILTER_FIELD_PATH)
+                    .setOp(Operator.EQUAL)
+                    .setValue(Value.newBuilder().setStringValue("bar"))
+                    .build())
+            .build();
+    static final Filter FIELD_NOT_EQUALS_FOO =
+        Filter.newBuilder()
+            .setFieldFilter(
+                FieldFilter.newBuilder()
+                    .setField(FILTER_FIELD_PATH)
+                    .setOp(Operator.NOT_EQUAL)
+                    .setValue(Value.newBuilder().setStringValue("foo"))
+                    .build())
+            .build();
+
     private final RunQueryRequest request;
     private final RunQueryResponse response1;
     private final RunQueryResponse response2;
     private final RunQueryResponse response3;
 
-    public TestData(String projectId, Function<FieldReference, List<Order>> orderFunction) {
-      String fieldPath = "foo";
-      FieldReference foo = FieldReference.newBuilder().setFieldPath(fieldPath).build();
+    public TestData(
+        String projectId, Function<FieldReference, List<Order>> orderFunction, Filter filter) {
       StructuredQuery.Builder builder =
           StructuredQuery.newBuilder()
               .addFrom(
                   CollectionSelector.newBuilder()
                       .setAllDescendants(false)
                       .setCollectionId("collection"))
-              .setWhere(
-                  Filter.newBuilder()
-                      .setFieldFilter(
-                          FieldFilter.newBuilder()
-                              .setField(foo)
-                              .setOp(Operator.EQUAL)
-                              .setValue(Value.newBuilder().setStringValue("bar"))
-                              .build()));
-
-      orderFunction.apply(foo).forEach(builder::addOrderBy);
+              .setWhere(filter);
+
+      orderFunction.apply(FILTER_FIELD_PATH).forEach(builder::addOrderBy);
       request =
           RunQueryRequest.newBuilder()
               .setParent(String.format("projects/%s/databases/(default)/document", projectId))
               .setStructuredQuery(builder)
               .build();
 
-      response1 = newResponse(fieldPath, 1);
-      response2 = newResponse(fieldPath, 2);
-      response3 = newResponse(fieldPath, 3);
+      response1 = newResponse(1);
+      response2 = newResponse(2);
+      response3 = newResponse(3);
     }
 
-    private static RunQueryResponse newResponse(String field, int docNumber) {
+    /**
+     * Returns single-document response like this: { "__name__": "doc-{docNumber}", "foo": "bar",
+     * "fo`o.m`ap": { "bar.key": "bar.val" } }

Review Comment:
   You added a super complicated case "\`fo\\\\\`o.m\\\\\`ap\`.\`bar.key\`" and the most simple case "foo" for the order by, can we also add two test cases that are in between these two: (1) x.y as y inside the map x, both x and y are simple strings and not escaped; (2) \`x.y\` as a single property with a dot in the property name. So the document might look like this:
   ```
   {
     "foo": "bar"
     "fo`o.m`ap": { "bar.key": "bar.val" }
     "baz" : {"qux" : "val"}
     "quux.quuz" : 123
   }
   ```
   you might parameterize the tests if it makes things easier, or just writing these 4 separate tests also seem fine.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/firestore/QueryUtils.java:
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.firestore;
+
+import com.google.firestore.v1.StructuredQuery;
+import com.google.firestore.v1.StructuredQuery.Direction;
+import com.google.firestore.v1.StructuredQuery.FieldFilter;
+import com.google.firestore.v1.StructuredQuery.FieldFilter.Operator;
+import com.google.firestore.v1.StructuredQuery.FieldReference;
+import com.google.firestore.v1.StructuredQuery.Filter;
+import com.google.firestore.v1.StructuredQuery.Order;
+import com.google.firestore.v1.StructuredQuery.UnaryFilter;
+import com.google.firestore.v1.Value;
+import com.google.firestore.v1.Value.ValueTypeCase;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Ascii;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Contains several internal utility functions for Firestore query handling, such as filling
+ * implicit ordering or escaping field references.
+ */
+class QueryUtils {
+
+  private static final ImmutableSet<Operator> INEQUALITY_FIELD_FILTER_OPS =
+      ImmutableSet.of(
+          FieldFilter.Operator.LESS_THAN,
+          FieldFilter.Operator.LESS_THAN_OR_EQUAL,
+          FieldFilter.Operator.GREATER_THAN,
+          FieldFilter.Operator.GREATER_THAN_OR_EQUAL,
+          FieldFilter.Operator.NOT_EQUAL,
+          FieldFilter.Operator.NOT_IN);
+  private static final ImmutableSet<UnaryFilter.Operator> INEQUALITY_UNARY_FILTER_OPS =
+      ImmutableSet.of(UnaryFilter.Operator.IS_NOT_NAN, UnaryFilter.Operator.IS_NOT_NULL);
+
+  static List<Order> getImplicitOrderBy(StructuredQuery query) {
+    List<String> expectedImplicitOrders = new ArrayList<>();
+    if (query.hasWhere()) {
+      fillInequalityFields(query.getWhere(), expectedImplicitOrders);
+    }
+    if (!expectedImplicitOrders.contains("__name__")) {
+      expectedImplicitOrders.add("__name__");
+    }
+    for (Order order : query.getOrderByList()) {
+      String orderField = order.getField().getFieldPath();
+      expectedImplicitOrders.remove(orderField);
+    }
+
+    List<Order> additionalOrders = new ArrayList<>();
+    if (!expectedImplicitOrders.isEmpty()) {
+      Direction lastDirection =
+          query.getOrderByCount() == 0
+              ? Direction.ASCENDING
+              : query.getOrderByList().get(query.getOrderByCount() - 1).getDirection();
+
+      for (String field : expectedImplicitOrders) {
+        additionalOrders.add(
+            Order.newBuilder()
+                .setDirection(lastDirection)
+                .setField(FieldReference.newBuilder().setFieldPath(field).build())
+                .build());
+      }
+    }
+
+    return additionalOrders;
+  }
+
+  private static void fillInequalityFields(Filter filter, List<String> result) {
+    switch (filter.getFilterTypeCase()) {
+      case FIELD_FILTER:
+        if (INEQUALITY_FIELD_FILTER_OPS.contains(filter.getFieldFilter().getOp())) {
+          String fieldPath = filter.getFieldFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      case COMPOSITE_FILTER:
+        filter.getCompositeFilter().getFiltersList().forEach(f -> fillInequalityFields(f, result));
+        break;
+      case UNARY_FILTER:
+        if (INEQUALITY_UNARY_FILTER_OPS.contains(filter.getUnaryFilter().getOp())) {
+          String fieldPath = filter.getUnaryFilter().getField().getFieldPath();
+          if (!result.contains(fieldPath)) {
+            result.add(fieldPath);
+          }
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  @Nullable
+  static Value lookupDocumentValue(List<String> segments, Map<String, Value> valueMap) {
+    String field = segments.remove(0);
+    Value value = valueMap.get(field);
+    if (segments.isEmpty()) {
+      return value;
+    }
+    // Field path traversal is not done, recurse into map values.
+    if (value == null || !value.getValueTypeCase().equals(ValueTypeCase.MAP_VALUE)) {
+      return null;
+    }
+    return lookupDocumentValue(segments, value.getMapValue().getFieldsMap());
+  }
+
+  private static final String UNQUOTED_NAME_REGEX_STRING = "([a-zA-Z_][a-zA-Z_0-9]*)";
+  private static final String QUOTED_NAME_REGEX_STRING = "(`(?:[^`\\\\]|(?:\\\\.))+`)";
+  // After each segment follows a dot and more characters, or the end of the string.
+  private static final Pattern FIELD_PATH_SEGMENT_REGEX =
+      Pattern.compile(
+          String.format("(?:%s|%s)(\\..+|$)", UNQUOTED_NAME_REGEX_STRING, QUOTED_NAME_REGEX_STRING),
+          Pattern.DOTALL);
+
+  static List<String> resolveOrderByFieldPath(String fieldPath) throws IllegalArgumentException {
+    if (fieldPath.isEmpty()) {
+      throw new IllegalArgumentException("Could not resolve empty field path");
+    }
+    List<String> segments = new ArrayList<>();
+    while (!fieldPath.isEmpty()) {
+      Matcher segmentMatcher = FIELD_PATH_SEGMENT_REGEX.matcher(fieldPath);
+      boolean foundMatch = segmentMatcher.lookingAt();
+      if (!foundMatch) {
+        throw new IllegalArgumentException("OrderBy field path was malformed");
+      }
+      String fieldName = segmentMatcher.group(1);
+      // Unquoted group is null, use quoted group.
+      if (fieldName == null) {
+        fieldName = segmentMatcher.group(2);
+        String escaped = escapeFieldName(fieldName.substring(1, fieldName.length() - 1));
+        segments.add(escaped);
+      } else {
+        segments.add(fieldName);
+      }
+      fieldPath = fieldPath.substring(fieldName.length());
+      if (fieldPath.startsWith(".")) {

Review Comment:
   SG, maybe add a inline comment that when fieldPath is not empty it must start with "." here.



-- 
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-unsubscribe@beam.apache.org

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


[GitHub] [beam] danthev commented on pull request #22175: [BEAM-22089] Fix query retry in Java FirestoreIO

Posted by GitBox <gi...@apache.org>.
danthev commented on PR #22175:
URL: https://github.com/apache/beam/pull/22175#issuecomment-1182010502

   Code is ready again, fixed some spotless errors after adding all the unit tests, filter sorting etc.


-- 
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-unsubscribe@beam.apache.org

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