You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/08/19 23:50:38 UTC

[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #5302: Add SparkV2Filters

aokolnychyi commented on code in PR #5302:
URL: https://github.com/apache/iceberg/pull/5302#discussion_r950580132


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThan;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notIn;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+import static org.apache.iceberg.expressions.Expressions.startsWith;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.NaNUtil;
+import org.apache.spark.sql.connector.expressions.LiteralValue;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.expressions.filter.And;
+import org.apache.spark.sql.connector.expressions.filter.Not;
+import org.apache.spark.sql.connector.expressions.filter.Or;
+import org.apache.spark.sql.connector.expressions.filter.Predicate;
+import org.apache.spark.unsafe.types.UTF8String;
+
+public class SparkV2Filters {
+
+  private static final Pattern BACKTICKS_PATTERN = Pattern.compile("([`])(.|$)");
+
+  private SparkV2Filters() {}
+
+  private static final String TRUE = "ALWAYS_TRUE";
+  private static final String FALSE = "ALWAYS_FALSE";
+  private static final String EQ = "=";
+  private static final String EQ_NULL_SAFE = "<=>";
+  private static final String GT = ">";
+  private static final String GT_EQ = ">=";
+  private static final String LT = "<";
+  private static final String LT_EQ = "<=";
+  private static final String IN = "IN";
+  private static final String IS_NULL = "IS_NULL";
+  private static final String NOT_NULL = "IS_NOT_NULL";
+  private static final String AND = "AND";
+  private static final String OR = "OR";
+  private static final String NOT = "NOT";
+  private static final String STARTS_WITH = "STARTS_WITH";
+
+  private static final Map<String, Expression.Operation> FILTERS =
+      ImmutableMap.<String, Expression.Operation>builder()
+          .put(TRUE, Expression.Operation.TRUE)
+          .put(FALSE, Expression.Operation.FALSE)
+          .put(EQ, Expression.Operation.EQ)
+          .put(EQ_NULL_SAFE, Expression.Operation.EQ)
+          .put(GT, Expression.Operation.GT)
+          .put(GT_EQ, Expression.Operation.GT_EQ)
+          .put(LT, Expression.Operation.LT)
+          .put(LT_EQ, Expression.Operation.LT_EQ)
+          .put(IN, Expression.Operation.IN)
+          .put(IS_NULL, Expression.Operation.IS_NULL)
+          .put(NOT_NULL, Expression.Operation.NOT_NULL)
+          .put(AND, Expression.Operation.AND)
+          .put(OR, Expression.Operation.OR)
+          .put(NOT, Expression.Operation.NOT)
+          .put(STARTS_WITH, Expression.Operation.STARTS_WITH)
+          .build();
+
+  private static final int FIRST_ORDINAL = 0;
+  private static final int SECOND_ORDINAL = 1;
+
+  public static Expression convert(Predicate[] predicates) {

Review Comment:
   What about removing this method for now? Our existing logic in `SparkFilters` is inconsistent right now as the similar method there does not do binding. We have to fix that first.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThan;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notIn;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+import static org.apache.iceberg.expressions.Expressions.startsWith;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.NaNUtil;
+import org.apache.spark.sql.connector.expressions.LiteralValue;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.expressions.filter.And;
+import org.apache.spark.sql.connector.expressions.filter.Not;
+import org.apache.spark.sql.connector.expressions.filter.Or;
+import org.apache.spark.sql.connector.expressions.filter.Predicate;
+import org.apache.spark.unsafe.types.UTF8String;
+
+public class SparkV2Filters {
+
+  private static final Pattern BACKTICKS_PATTERN = Pattern.compile("([`])(.|$)");
+
+  private SparkV2Filters() {}
+
+  private static final String TRUE = "ALWAYS_TRUE";
+  private static final String FALSE = "ALWAYS_FALSE";
+  private static final String EQ = "=";
+  private static final String EQ_NULL_SAFE = "<=>";
+  private static final String GT = ">";
+  private static final String GT_EQ = ">=";
+  private static final String LT = "<";
+  private static final String LT_EQ = "<=";
+  private static final String IN = "IN";
+  private static final String IS_NULL = "IS_NULL";
+  private static final String NOT_NULL = "IS_NOT_NULL";
+  private static final String AND = "AND";
+  private static final String OR = "OR";
+  private static final String NOT = "NOT";
+  private static final String STARTS_WITH = "STARTS_WITH";
+
+  private static final Map<String, Expression.Operation> FILTERS =
+      ImmutableMap.<String, Expression.Operation>builder()

Review Comment:
   nit: Can we do a direct import for `Operation` like in `SparkFilters`?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThan;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notIn;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+import static org.apache.iceberg.expressions.Expressions.startsWith;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.NaNUtil;
+import org.apache.spark.sql.connector.expressions.LiteralValue;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.expressions.filter.And;
+import org.apache.spark.sql.connector.expressions.filter.Not;
+import org.apache.spark.sql.connector.expressions.filter.Or;
+import org.apache.spark.sql.connector.expressions.filter.Predicate;
+import org.apache.spark.unsafe.types.UTF8String;
+
+public class SparkV2Filters {
+
+  private static final Pattern BACKTICKS_PATTERN = Pattern.compile("([`])(.|$)");
+
+  private SparkV2Filters() {}
+
+  private static final String TRUE = "ALWAYS_TRUE";
+  private static final String FALSE = "ALWAYS_FALSE";
+  private static final String EQ = "=";
+  private static final String EQ_NULL_SAFE = "<=>";
+  private static final String GT = ">";
+  private static final String GT_EQ = ">=";
+  private static final String LT = "<";
+  private static final String LT_EQ = "<=";
+  private static final String IN = "IN";
+  private static final String IS_NULL = "IS_NULL";
+  private static final String NOT_NULL = "IS_NOT_NULL";
+  private static final String AND = "AND";
+  private static final String OR = "OR";
+  private static final String NOT = "NOT";
+  private static final String STARTS_WITH = "STARTS_WITH";
+
+  private static final Map<String, Expression.Operation> FILTERS =
+      ImmutableMap.<String, Expression.Operation>builder()
+          .put(TRUE, Expression.Operation.TRUE)
+          .put(FALSE, Expression.Operation.FALSE)
+          .put(EQ, Expression.Operation.EQ)
+          .put(EQ_NULL_SAFE, Expression.Operation.EQ)
+          .put(GT, Expression.Operation.GT)
+          .put(GT_EQ, Expression.Operation.GT_EQ)
+          .put(LT, Expression.Operation.LT)
+          .put(LT_EQ, Expression.Operation.LT_EQ)
+          .put(IN, Expression.Operation.IN)
+          .put(IS_NULL, Expression.Operation.IS_NULL)
+          .put(NOT_NULL, Expression.Operation.NOT_NULL)
+          .put(AND, Expression.Operation.AND)
+          .put(OR, Expression.Operation.OR)
+          .put(NOT, Expression.Operation.NOT)
+          .put(STARTS_WITH, Expression.Operation.STARTS_WITH)
+          .build();
+
+  private static final int FIRST_ORDINAL = 0;
+  private static final int SECOND_ORDINAL = 1;
+
+  public static Expression convert(Predicate[] predicates) {
+    Expression expression = Expressions.alwaysTrue();
+    for (Predicate predicate : predicates) {
+      Expression converted = convert(predicate);
+      Preconditions.checkArgument(
+          converted != null, "Cannot convert predicate to Iceberg: %s", predicate);
+      expression = Expressions.and(expression, converted);
+    }
+    return expression;
+  }
+
+  @SuppressWarnings({"checkstyle:CyclomaticComplexity", "checkstyle:MethodLength"})
+  public static Expression convert(Predicate predicate) {
+    if (!valid(predicate)) {
+      return null;
+    }
+
+    Expression.Operation op = FILTERS.get(predicate.name());
+    if (op != null) {
+      switch (op) {
+        case TRUE:
+          return Expressions.alwaysTrue();
+
+        case FALSE:
+          return Expressions.alwaysFalse();
+
+        case IS_NULL:
+          return isNull(unquote(predicate.children()[FIRST_ORDINAL].toString()));
+
+        case NOT_NULL:
+          return notNull(unquote(predicate.children()[FIRST_ORDINAL].toString()));
+
+        case LT:
+          if (predicate.children()[SECOND_ORDINAL] instanceof LiteralValue) {
+            return lessThan(
+                unquote(predicate.children()[FIRST_ORDINAL].toString()),
+                convertUTF8StringIfNecessary(((LiteralValue) predicate.children()[1]).value()));
+          } else {
+            return greaterThan(
+                unquote(predicate.children()[SECOND_ORDINAL].toString()),
+                convertUTF8StringIfNecessary(
+                    ((LiteralValue) predicate.children()[FIRST_ORDINAL]).value()));
+          }
+
+        case LT_EQ:
+          if (predicate.children()[SECOND_ORDINAL] instanceof LiteralValue) {
+            return lessThanOrEqual(
+                unquote(predicate.children()[FIRST_ORDINAL].toString()),
+                convertUTF8StringIfNecessary(((LiteralValue) predicate.children()[1]).value()));
+          } else {
+            return greaterThanOrEqual(
+                unquote(predicate.children()[SECOND_ORDINAL].toString()),
+                convertUTF8StringIfNecessary(
+                    ((LiteralValue) predicate.children()[FIRST_ORDINAL]).value()));
+          }
+
+        case GT:
+          if (predicate.children()[SECOND_ORDINAL] instanceof LiteralValue) {
+            return greaterThan(
+                unquote(predicate.children()[FIRST_ORDINAL].toString()),
+                convertUTF8StringIfNecessary(
+                    ((LiteralValue) predicate.children()[SECOND_ORDINAL]).value()));
+          } else {
+            return lessThan(
+                unquote(predicate.children()[SECOND_ORDINAL].toString()),
+                convertUTF8StringIfNecessary(
+                    ((LiteralValue) predicate.children()[FIRST_ORDINAL]).value()));
+          }
+
+        case GT_EQ:
+          if (predicate.children()[SECOND_ORDINAL] instanceof LiteralValue) {
+            return greaterThanOrEqual(
+                unquote(predicate.children()[FIRST_ORDINAL].toString()),
+                convertUTF8StringIfNecessary(((LiteralValue) predicate.children()[1]).value()));
+          } else {
+            return lessThanOrEqual(
+                unquote(predicate.children()[SECOND_ORDINAL].toString()),
+                convertUTF8StringIfNecessary(
+                    ((LiteralValue) predicate.children()[FIRST_ORDINAL]).value()));
+          }
+
+        case EQ: // used for both eq and null-safe-eq
+          Object value;
+          String attributeName;
+          if (predicate.children()[SECOND_ORDINAL] instanceof LiteralValue) {
+            attributeName = predicate.children()[FIRST_ORDINAL].toString();
+            value = convertUTF8StringIfNecessary(((LiteralValue) predicate.children()[1]).value());
+          } else {
+            attributeName = predicate.children()[SECOND_ORDINAL].toString();
+            value =
+                convertUTF8StringIfNecessary(
+                    ((LiteralValue) predicate.children()[FIRST_ORDINAL]).value());
+          }
+
+          if (predicate.name().equals("=")) {
+            // comparison with null in normal equality is always null. this is probably a mistake.
+            Preconditions.checkNotNull(
+                value, "Expression is always false (eq is not null-safe): %s", predicate);
+            return handleEqual(unquote(attributeName), value);
+          } else { // "<=>"
+            if (value == null) {
+              return isNull(unquote(attributeName));
+            } else {
+              return handleEqual(unquote(attributeName), value);
+            }
+          }
+
+        case IN:
+          return in(
+              unquote(predicate.children()[FIRST_ORDINAL].toString()),
+              Arrays.stream(predicate.children())
+                  .skip(1)
+                  .map(val -> convertUTF8StringIfNecessary(((LiteralValue) val).value()))
+                  .filter(Objects::nonNull)
+                  .collect(Collectors.toList()));
+
+        case NOT:
+          Not notFilter = (Not) predicate;
+          Predicate childFilter = notFilter.child();
+          if (childFilter.name().equals("IN")) {
+            // infer an extra notNull predicate for Spark NOT IN filters
+            // as Iceberg expressions don't follow the 3-value SQL boolean logic
+            // col NOT IN (1, 2) in Spark is equivalent to notNull(col) && notIn(col, 1, 2) in
+            // Iceberg
+            Expression notIn =
+                notIn(
+                    unquote(childFilter.children()[FIRST_ORDINAL].toString()),
+                    Arrays.stream(childFilter.children())
+                        .skip(1)
+                        .map(val -> convertUTF8StringIfNecessary(((LiteralValue) val).value()))
+                        .filter(Objects::nonNull)
+                        .collect(Collectors.toList()));
+            return and(notNull(unquote(childFilter.children()[FIRST_ORDINAL].toString())), notIn);
+          } else if (hasNoInFilter(childFilter)) {
+            Expression child = convert(childFilter);
+            if (child != null) {
+              return not(child);
+            }
+          }
+          return null;
+
+        case AND:
+          {

Review Comment:
   nit: Do we need these extra `{}` here and 2 branches below?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThan;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notIn;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+import static org.apache.iceberg.expressions.Expressions.startsWith;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.NaNUtil;
+import org.apache.spark.sql.connector.expressions.LiteralValue;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.expressions.filter.And;
+import org.apache.spark.sql.connector.expressions.filter.Not;
+import org.apache.spark.sql.connector.expressions.filter.Or;
+import org.apache.spark.sql.connector.expressions.filter.Predicate;
+import org.apache.spark.unsafe.types.UTF8String;
+
+public class SparkV2Filters {
+
+  private static final Pattern BACKTICKS_PATTERN = Pattern.compile("([`])(.|$)");
+
+  private SparkV2Filters() {}
+
+  private static final String TRUE = "ALWAYS_TRUE";
+  private static final String FALSE = "ALWAYS_FALSE";
+  private static final String EQ = "=";
+  private static final String EQ_NULL_SAFE = "<=>";
+  private static final String GT = ">";
+  private static final String GT_EQ = ">=";
+  private static final String LT = "<";
+  private static final String LT_EQ = "<=";
+  private static final String IN = "IN";
+  private static final String IS_NULL = "IS_NULL";
+  private static final String NOT_NULL = "IS_NOT_NULL";
+  private static final String AND = "AND";
+  private static final String OR = "OR";
+  private static final String NOT = "NOT";
+  private static final String STARTS_WITH = "STARTS_WITH";
+
+  private static final Map<String, Expression.Operation> FILTERS =
+      ImmutableMap.<String, Expression.Operation>builder()
+          .put(TRUE, Expression.Operation.TRUE)
+          .put(FALSE, Expression.Operation.FALSE)
+          .put(EQ, Expression.Operation.EQ)
+          .put(EQ_NULL_SAFE, Expression.Operation.EQ)
+          .put(GT, Expression.Operation.GT)
+          .put(GT_EQ, Expression.Operation.GT_EQ)
+          .put(LT, Expression.Operation.LT)
+          .put(LT_EQ, Expression.Operation.LT_EQ)
+          .put(IN, Expression.Operation.IN)
+          .put(IS_NULL, Expression.Operation.IS_NULL)
+          .put(NOT_NULL, Expression.Operation.NOT_NULL)
+          .put(AND, Expression.Operation.AND)
+          .put(OR, Expression.Operation.OR)
+          .put(NOT, Expression.Operation.NOT)
+          .put(STARTS_WITH, Expression.Operation.STARTS_WITH)
+          .build();
+
+  private static final int FIRST_ORDINAL = 0;
+  private static final int SECOND_ORDINAL = 1;
+
+  public static Expression convert(Predicate[] predicates) {
+    Expression expression = Expressions.alwaysTrue();
+    for (Predicate predicate : predicates) {
+      Expression converted = convert(predicate);
+      Preconditions.checkArgument(
+          converted != null, "Cannot convert predicate to Iceberg: %s", predicate);
+      expression = Expressions.and(expression, converted);
+    }
+    return expression;
+  }
+
+  @SuppressWarnings({"checkstyle:CyclomaticComplexity", "checkstyle:MethodLength"})
+  public static Expression convert(Predicate predicate) {
+    if (!valid(predicate)) {

Review Comment:
   @huaxingao, can you elaborate a bit on what is considered invalid in this case? Are we trying to validate the Spark logic or are we trying to check if the filter is supported by Iceberg?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java:
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThan;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notIn;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+import static org.apache.iceberg.expressions.Expressions.startsWith;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.NaNUtil;
+import org.apache.spark.sql.connector.expressions.LiteralValue;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.expressions.filter.And;
+import org.apache.spark.sql.connector.expressions.filter.Not;
+import org.apache.spark.sql.connector.expressions.filter.Or;
+import org.apache.spark.sql.connector.expressions.filter.Predicate;
+import org.apache.spark.unsafe.types.UTF8String;
+
+public class SparkV2Filters {
+
+  private static final Pattern BACKTICKS_PATTERN = Pattern.compile("([`])(.|$)");
+
+  private SparkV2Filters() {}

Review Comment:
   I think this follows what we did in `SparkFilters` but I agree with @kbendick that we should move this below all fields.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org