You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by bl...@apache.org on 2022/05/12 19:09:29 UTC

[iceberg] branch master updated: API: Add expression sanitizer, sanitize scan log messages (#4672)

This is an automated email from the ASF dual-hosted git repository.

blue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new 26d8c3ed9e API: Add expression sanitizer, sanitize scan log messages (#4672)
26d8c3ed9e is described below

commit 26d8c3ed9eceaf0fd7d2429302a4c7ccedbb2111
Author: Ryan Blue <bl...@apache.org>
AuthorDate: Thu May 12 12:09:24 2022 -0700

    API: Add expression sanitizer, sanitize scan log messages (#4672)
---
 .../apache/iceberg/expressions/ExpressionUtil.java | 257 ++++++++++++++++++++
 .../apache/iceberg/expressions/Expressions.java    |   2 +-
 .../iceberg/expressions/TestExpressionUtil.java    | 262 +++++++++++++++++++++
 .../apache/iceberg/BaseAllMetadataTableScan.java   |   3 +-
 .../java/org/apache/iceberg/BaseTableScan.java     |   3 +-
 5 files changed, 524 insertions(+), 3 deletions(-)

diff --git a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java
new file mode 100644
index 0000000000..97a3ad0674
--- /dev/null
+++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java
@@ -0,0 +1,257 @@
+/*
+ * 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.expressions;
+
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.iceberg.transforms.Transform;
+import org.apache.iceberg.transforms.Transforms;
+import org.apache.iceberg.types.Types;
+
+/**
+ * Expression utility methods.
+ */
+public class ExpressionUtil {
+  private static final Transform<CharSequence, Integer> HASH_FUNC = Transforms
+      .bucket(Types.StringType.get(), Integer.MAX_VALUE);
+  private static final Pattern DATE = Pattern.compile("\\d\\d\\d\\d-\\d\\d-\\d\\d");
+  private static final Pattern TIME = Pattern.compile(
+      "\\d\\d:\\d\\d(:\\d\\d(.\\d{1,6})?)?");
+  private static final Pattern TIMESTAMP = Pattern.compile(
+      "\\d\\d\\d\\d-\\d\\d-\\d\\dT\\d\\d:\\d\\d(:\\d\\d(.\\d{1,6})?)?([-+]\\d\\d:\\d\\d)?");
+
+  private ExpressionUtil() {
+  }
+
+  /**
+   * Produces an unbound {@link Expression} with the same structure, but with data values replaced by descriptions.
+   * <p>
+   * Numbers are replaced with magnitude and type, string-like values are replaced by hashes, and date/time values are
+   * replaced by the type.
+   *
+   * @param expr an Expression to sanitize
+   * @return a sanitized Expression
+   */
+  public static Expression sanitize(Expression expr) {
+    return ExpressionVisitors.visit(expr, ExpressionSanitizer.INSTANCE);
+  }
+
+  /**
+   * Produces a sanitized expression string with the same structure, but with data values replaced by descriptions.
+   * <p>
+   * Numbers are replaced with magnitude and type, string-like values are replaced by hashes, and date/time values are
+   * replaced by the type.
+   *
+   * @param expr an Expression to sanitize
+   * @return a sanitized expression string
+   */
+  public static String toSanitizedString(Expression expr) {
+    return ExpressionVisitors.visit(expr, StringSanitizer.INSTANCE);
+  }
+
+  private static class ExpressionSanitizer extends ExpressionVisitors.ExpressionVisitor<Expression> {
+    private static final ExpressionSanitizer INSTANCE = new ExpressionSanitizer();
+
+    @Override
+    public Expression alwaysTrue() {
+      return Expressions.alwaysTrue();
+    }
+
+    @Override
+    public Expression alwaysFalse() {
+      return Expressions.alwaysFalse();
+    }
+
+    @Override
+    public Expression not(Expression result) {
+      return Expressions.not(result);
+    }
+
+    @Override
+    public Expression and(Expression leftResult, Expression rightResult) {
+      return Expressions.and(leftResult, rightResult);
+    }
+
+    @Override
+    public Expression or(Expression leftResult, Expression rightResult) {
+      return Expressions.or(leftResult, rightResult);
+    }
+
+    @Override
+    public <T> Expression predicate(BoundPredicate<T> pred) {
+      throw new UnsupportedOperationException("Cannot sanitize bound predicate: " + pred);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> Expression predicate(UnboundPredicate<T> pred) {
+      switch (pred.op()) {
+        case IS_NULL:
+        case NOT_NULL:
+        case IS_NAN:
+        case NOT_NAN:
+          // unary predicates don't need to be sanitized
+          return pred;
+        case LT:
+        case LT_EQ:
+        case GT:
+        case GT_EQ:
+        case EQ:
+        case NOT_EQ:
+        case STARTS_WITH:
+        case NOT_STARTS_WITH:
+          return new UnboundPredicate<>(pred.op(), pred.term(), (T) sanitize(pred.literal()));
+        case IN:
+        case NOT_IN:
+          Iterable<String> iter = () -> pred.literals().stream().map(ExpressionUtil::sanitize).iterator();
+          return new UnboundPredicate<>(pred.op(), pred.term(), (Iterable<T>) iter);
+        default:
+          throw new UnsupportedOperationException("Cannot sanitize unsupported predicate type: " + pred.op());
+      }
+    }
+  }
+
+  private static class StringSanitizer extends ExpressionVisitors.ExpressionVisitor<String> {
+    private static final StringSanitizer INSTANCE = new StringSanitizer();
+
+    @Override
+    public String alwaysTrue() {
+      return "true";
+    }
+
+    @Override
+    public String alwaysFalse() {
+      return "false";
+    }
+
+    @Override
+    public String not(String result) {
+      return "NOT (" + result + ")";
+    }
+
+    @Override
+    public String and(String leftResult, String rightResult) {
+      return "(" + leftResult + " AND " + rightResult + ")";
+    }
+
+    @Override
+    public String or(String leftResult, String rightResult) {
+      return "(" + leftResult + " OR " + rightResult + ")";
+    }
+
+    @Override
+    public <T> String predicate(BoundPredicate<T> pred) {
+      throw new UnsupportedOperationException("Cannot sanitize bound predicate: " + pred);
+    }
+
+    public String termToString(UnboundTerm<?> term) {
+      if (term instanceof UnboundTransform) {
+        return ((UnboundTransform<?, ?>) term).transform() + "(" + termToString(term.ref()) + ")";
+      } else if (term instanceof NamedReference) {
+        return ((NamedReference<?>) term).name();
+      } else {
+        throw new UnsupportedOperationException("Unsupported term: " + term);
+      }
+    }
+
+    @Override
+    public <T> String predicate(UnboundPredicate<T> pred) {
+      String term = termToString(pred.term());
+      switch (pred.op()) {
+        case IS_NULL:
+          return term + " IS NULL";
+        case NOT_NULL:
+          return term + " IS NOT NULL";
+        case IS_NAN:
+          return "is_nan(" + term + ")";
+        case NOT_NAN:
+          return "not_nan(" + term + ")";
+        case LT:
+          return term + " < " + sanitize(pred.literal());
+        case LT_EQ:
+          return term + " <= " + sanitize(pred.literal());
+        case GT:
+          return term + " > " + sanitize(pred.literal());
+        case GT_EQ:
+          return term + " >= " + sanitize(pred.literal());
+        case EQ:
+          return term + " = " + sanitize(pred.literal());
+        case NOT_EQ:
+          return term + " != " + sanitize(pred.literal());
+        case IN:
+          return term + " IN " + pred.literals().stream()
+              .map(ExpressionUtil::sanitize)
+              .collect(Collectors.joining(", ", "(", ")"));
+        case NOT_IN:
+          return term + " NOT IN " + pred.literals().stream()
+              .map(ExpressionUtil::sanitize)
+              .collect(Collectors.joining(", ", "(", ")"));
+        case STARTS_WITH:
+          return term + " STARTS WITH " + sanitize(pred.literal());
+        case NOT_STARTS_WITH:
+          return term + " NOT STARTS WITH " + sanitize(pred.literal());
+        default:
+          throw new UnsupportedOperationException("Cannot sanitize unsupported predicate type: " + pred.op());
+      }
+    }
+  }
+
+  private static String sanitize(Literal<?> literal) {
+    if (literal instanceof Literals.StringLiteral) {
+      CharSequence value = ((Literals.StringLiteral) literal).value();
+      if (DATE.matcher(value).matches()) {
+        return "(date)";
+      } else if (TIME.matcher(value).matches()) {
+        return "(time)";
+      } else if (TIMESTAMP.matcher(value).matches()) {
+        return "(timestamp)";
+      } else {
+        return sanitizeString(value);
+      }
+    } else if (literal instanceof Literals.DateLiteral) {
+      return "(date)";
+    } else if (literal instanceof Literals.TimeLiteral) {
+      return "(time)";
+    } else if (literal instanceof Literals.TimestampLiteral) {
+      return "(timestamp)";
+    } else if (literal instanceof Literals.IntegerLiteral) {
+      return sanitizeNumber(((Literals.IntegerLiteral) literal).value(), "int");
+    } else if (literal instanceof Literals.LongLiteral) {
+      return sanitizeNumber(((Literals.LongLiteral) literal).value(), "int");
+    } else if (literal instanceof Literals.FloatLiteral) {
+      return sanitizeNumber(((Literals.FloatLiteral) literal).value(), "float");
+    } else if (literal instanceof Literals.DoubleLiteral) {
+      return sanitizeNumber(((Literals.DoubleLiteral) literal).value(), "float");
+    } else {
+      // for uuid, decimal, fixed, and binary, match the string result
+      return sanitizeString(literal.value().toString());
+    }
+  }
+
+  private static String sanitizeNumber(Number value, String type) {
+    int numDigits = (int) Math.log10(value.doubleValue()) + 1;
+    return "(" + numDigits + "-digit-" + type + ")";
+  }
+
+  private static String sanitizeString(CharSequence value) {
+    // hash the value and return the hash as hex
+    return String.format("(hash-%08x)", HASH_FUNC.apply(value));
+  }
+}
diff --git a/api/src/main/java/org/apache/iceberg/expressions/Expressions.java b/api/src/main/java/org/apache/iceberg/expressions/Expressions.java
index d06d7c5169..a0690f1e94 100644
--- a/api/src/main/java/org/apache/iceberg/expressions/Expressions.java
+++ b/api/src/main/java/org/apache/iceberg/expressions/Expressions.java
@@ -261,7 +261,7 @@ public class Expressions {
     return new UnboundPredicate<>(op, ref(name));
   }
 
-  private static <T> UnboundPredicate<T> predicate(Operation op, UnboundTerm<T> expr, Iterable<T> values) {
+  public static <T> UnboundPredicate<T> predicate(Operation op, UnboundTerm<T> expr, Iterable<T> values) {
     return new UnboundPredicate<>(op, expr, values);
   }
 
diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java
new file mode 100644
index 0000000000..b4e72e977b
--- /dev/null
+++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java
@@ -0,0 +1,262 @@
+/*
+ * 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.expressions;
+
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestExpressionUtil {
+  @Test
+  public void testUnchangedUnaryPredicates() {
+    for (Expression unary : Lists.newArrayList(
+        Expressions.isNull("test"), Expressions.notNull("test"), Expressions.isNaN("test"), Expressions.notNaN("test"))
+    ) {
+      assertEquals(unary, ExpressionUtil.sanitize(unary));
+    }
+  }
+
+  @Test
+  public void testSanitizeIn() {
+    assertEquals(
+        Expressions.in("test", "(2-digit-int)", "(3-digit-int)"),
+        ExpressionUtil.sanitize(Expressions.in("test", 34, 345)));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test IN ((2-digit-int), (3-digit-int))",
+        ExpressionUtil.toSanitizedString(Expressions.in("test", 34, 345)));
+  }
+
+  @Test
+  public void testSanitizeNotIn() {
+    assertEquals(
+        Expressions.notIn("test", "(2-digit-int)", "(3-digit-int)"),
+        ExpressionUtil.sanitize(Expressions.notIn("test", 34, 345)));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test NOT IN ((2-digit-int), (3-digit-int))",
+        ExpressionUtil.toSanitizedString(Expressions.notIn("test", 34, 345)));
+  }
+
+  @Test
+  public void testSanitizeLessThan() {
+    assertEquals(
+        Expressions.lessThan("test", "(2-digit-int)"),
+        ExpressionUtil.sanitize(Expressions.lessThan("test", 34)));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test < (2-digit-int)",
+        ExpressionUtil.toSanitizedString(Expressions.lessThan("test", 34)));
+  }
+
+  @Test
+  public void testSanitizeLessThanOrEqual() {
+    assertEquals(
+        Expressions.lessThanOrEqual("test", "(2-digit-int)"),
+        ExpressionUtil.sanitize(Expressions.lessThanOrEqual("test", 34)));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test <= (2-digit-int)",
+        ExpressionUtil.toSanitizedString(Expressions.lessThanOrEqual("test", 34)));
+  }
+
+  @Test
+  public void testSanitizeGreaterThan() {
+    assertEquals(
+        Expressions.greaterThan("test", "(2-digit-int)"),
+        ExpressionUtil.sanitize(Expressions.greaterThan("test", 34)));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test > (2-digit-int)",
+        ExpressionUtil.toSanitizedString(Expressions.greaterThan("test", 34)));
+  }
+
+  @Test
+  public void testSanitizeGreaterThanOrEqual() {
+    assertEquals(
+        Expressions.greaterThanOrEqual("test", "(2-digit-int)"),
+        ExpressionUtil.sanitize(Expressions.greaterThanOrEqual("test", 34)));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test >= (2-digit-int)",
+        ExpressionUtil.toSanitizedString(Expressions.greaterThanOrEqual("test", 34)));
+  }
+
+  @Test
+  public void testSanitizeEqual() {
+    assertEquals(
+        Expressions.equal("test", "(2-digit-int)"),
+        ExpressionUtil.sanitize(Expressions.equal("test", 34)));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test = (2-digit-int)",
+        ExpressionUtil.toSanitizedString(Expressions.equal("test", 34)));
+  }
+
+  @Test
+  public void testSanitizeNotEqual() {
+    assertEquals(
+        Expressions.notEqual("test", "(2-digit-int)"),
+        ExpressionUtil.sanitize(Expressions.notEqual("test", 34)));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test != (2-digit-int)",
+        ExpressionUtil.toSanitizedString(Expressions.notEqual("test", 34)));
+  }
+
+  @Test
+  public void testSanitizeStartsWith() {
+    assertEquals(
+        Expressions.startsWith("test", "(hash-34d05fb7)"),
+        ExpressionUtil.sanitize(Expressions.startsWith("test", "aaa")));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test STARTS WITH (hash-34d05fb7)",
+        ExpressionUtil.toSanitizedString(Expressions.startsWith("test", "aaa")));
+  }
+
+  @Test
+  public void testSanitizeNotStartsWith() {
+    assertEquals(
+        Expressions.notStartsWith("test", "(hash-34d05fb7)"),
+        ExpressionUtil.sanitize(Expressions.notStartsWith("test", "aaa")));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test NOT STARTS WITH (hash-34d05fb7)",
+        ExpressionUtil.toSanitizedString(Expressions.notStartsWith("test", "aaa")));
+  }
+
+  @Test
+  public void testSanitizeTransformedTerm() {
+    assertEquals(
+        Expressions.equal(Expressions.truncate("test", 2), "(2-digit-int)"),
+        ExpressionUtil.sanitize(Expressions.equal(Expressions.truncate("test", 2), 34)));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "truncate[2](test) = (2-digit-int)",
+        ExpressionUtil.toSanitizedString(Expressions.equal(Expressions.truncate("test", 2), 34)));
+  }
+
+  @Test
+  public void testSanitizeLong() {
+    assertEquals(
+        Expressions.equal("test", "(2-digit-int)"),
+        ExpressionUtil.sanitize(Expressions.equal("test", 34L)));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test = (2-digit-int)",
+        ExpressionUtil.toSanitizedString(Expressions.equal("test", 34L)));
+  }
+
+  @Test
+  public void testSanitizeFloat() {
+    assertEquals(
+        Expressions.equal("test", "(2-digit-float)"),
+        ExpressionUtil.sanitize(Expressions.equal("test", 34.12F)));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test = (2-digit-float)",
+        ExpressionUtil.toSanitizedString(Expressions.equal("test", 34.12F)));
+  }
+
+  @Test
+  public void testSanitizeDouble() {
+    assertEquals(
+        Expressions.equal("test", "(2-digit-float)"),
+        ExpressionUtil.sanitize(Expressions.equal("test", 34.12D)));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test = (2-digit-float)",
+        ExpressionUtil.toSanitizedString(Expressions.equal("test", 34.12D)));
+  }
+
+  @Test
+  public void testSanitizeDate() {
+    assertEquals(
+        Expressions.equal("test", "(date)"),
+        ExpressionUtil.sanitize(Expressions.equal("test", "2022-04-29")));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test = (date)",
+        ExpressionUtil.toSanitizedString(Expressions.equal("test", "2022-04-29")));
+  }
+
+  @Test
+  public void testSanitizeTime() {
+    assertEquals(
+        Expressions.equal("test", "(time)"),
+        ExpressionUtil.sanitize(Expressions.equal("test", "23:49:51")));
+
+    Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+        "test = (time)",
+        ExpressionUtil.toSanitizedString(Expressions.equal("test", "23:49:51")));
+  }
+
+  @Test
+  public void testSanitizeTimestamp() {
+    for (String timestamp : Lists.newArrayList("2022-04-29T23:49:51", "2022-04-29T23:49:51.123456",
+        "2022-04-29T23:49:51-07:00", "2022-04-29T23:49:51.123456+01:00")) {
+      assertEquals(
+          Expressions.equal("test", "(timestamp)"),
+          ExpressionUtil.sanitize(Expressions.equal("test", timestamp)));
+
+      Assert.assertEquals("Sanitized string should be identical except for descriptive literal",
+          "test = (timestamp)",
+          ExpressionUtil.toSanitizedString(Expressions.equal("test", timestamp)));
+    }
+  }
+
+  private void assertEquals(Expression expected, Expression actual) {
+    if (expected instanceof UnboundPredicate) {
+      Assert.assertTrue("Should be an UnboundPredicate", actual instanceof UnboundPredicate);
+      assertEquals((UnboundPredicate<?>) expected, (UnboundPredicate<?>) actual);
+    } else {
+      Assert.fail("Unknown expected expression: " + expected);
+    }
+  }
+
+  private void assertEquals(UnboundPredicate<?> expected, UnboundPredicate<?> actual) {
+    Assert.assertEquals("Operation should match", expected.op(), actual.op());
+    assertEquals(expected.term(), actual.term());
+    Assert.assertEquals("Literals should match", expected.literals(), actual.literals());
+  }
+
+  private void assertEquals(UnboundTerm<?> expected, UnboundTerm<?> actual) {
+    if (expected instanceof NamedReference) {
+      Assert.assertTrue("Should be a NamedReference", actual instanceof NamedReference);
+      assertEquals((NamedReference<?>) expected, (NamedReference<?>) actual);
+    } else if (expected instanceof UnboundTransform) {
+      Assert.assertTrue("Should be an UnboundTransform", actual instanceof UnboundTransform);
+      assertEquals((UnboundTransform<?, ?>) expected, (UnboundTransform<?, ?>) actual);
+    } else {
+      Assert.fail("Unknown expected term: " + expected);
+    }
+  }
+
+  private void assertEquals(NamedReference<?> expected, NamedReference<?> actual) {
+    Assert.assertEquals("Should reference the same field name", expected.name(), actual.name());
+  }
+
+  private void assertEquals(UnboundTransform<?, ?> expected, UnboundTransform<?, ?> actual) {
+    Assert.assertEquals("Should apply the same transform",
+        expected.transform().toString(), actual.transform().toString());
+    assertEquals(expected.ref(), actual.ref());
+  }
+}
diff --git a/core/src/main/java/org/apache/iceberg/BaseAllMetadataTableScan.java b/core/src/main/java/org/apache/iceberg/BaseAllMetadataTableScan.java
index 75db328cf8..7d97e743ab 100644
--- a/core/src/main/java/org/apache/iceberg/BaseAllMetadataTableScan.java
+++ b/core/src/main/java/org/apache/iceberg/BaseAllMetadataTableScan.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.io.UncheckedIOException;
 import org.apache.iceberg.events.Listeners;
 import org.apache.iceberg.events.ScanEvent;
+import org.apache.iceberg.expressions.ExpressionUtil;
 import org.apache.iceberg.io.CloseableIterable;
 import org.apache.iceberg.relocated.com.google.common.base.Function;
 import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
@@ -55,7 +56,7 @@ abstract class BaseAllMetadataTableScan extends BaseMetadataTableScan {
 
   @Override
   public CloseableIterable<FileScanTask> planFiles() {
-    LOG.info("Scanning metadata table {} with filter {}.", table(), filter());
+    LOG.info("Scanning metadata table {} with filter {}.", table(), ExpressionUtil.toSanitizedString(filter()));
     Listeners.notifyAll(new ScanEvent(table().name(), 0L, filter(), schema()));
 
     return doPlanFiles();
diff --git a/core/src/main/java/org/apache/iceberg/BaseTableScan.java b/core/src/main/java/org/apache/iceberg/BaseTableScan.java
index 29350c1640..83b3ba8159 100644
--- a/core/src/main/java/org/apache/iceberg/BaseTableScan.java
+++ b/core/src/main/java/org/apache/iceberg/BaseTableScan.java
@@ -28,6 +28,7 @@ import org.apache.iceberg.events.Listeners;
 import org.apache.iceberg.events.ScanEvent;
 import org.apache.iceberg.expressions.Binder;
 import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.ExpressionUtil;
 import org.apache.iceberg.expressions.Expressions;
 import org.apache.iceberg.io.CloseableIterable;
 import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
@@ -188,7 +189,7 @@ abstract class BaseTableScan implements TableScan {
     if (snapshot != null) {
       LOG.info("Scanning table {} snapshot {} created at {} with filter {}", table,
           snapshot.snapshotId(), DateTimeUtil.formatTimestampMillis(snapshot.timestampMillis()),
-          filter());
+          ExpressionUtil.toSanitizedString(filter()));
 
       Listeners.notifyAll(new ScanEvent(table.name(), snapshot.snapshotId(), filter(), schema()));