diff --git a/api/src/main/java/org/apache/iceberg/expressions/Evaluator.java b/api/src/main/java/org/apache/iceberg/expressions/Evaluator.java index 96e148a2d438..6270f65b1212 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Evaluator.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Evaluator.java @@ -19,6 +19,7 @@ package org.apache.iceberg.expressions; import java.io.Serializable; +import java.nio.ByteBuffer; import java.util.Comparator; import java.util.Set; import org.apache.iceberg.StructLike; @@ -156,5 +157,17 @@ public Boolean startsWith(Bound valueExpr, Literal lit) { public Boolean notStartsWith(Bound valueExpr, Literal lit) { return !startsWith(valueExpr, lit); } + + @Override + public Boolean stIntersects(Bound valueExpr, Literal literal) { + throw new UnsupportedOperationException( + "Evaluation of stIntersects against geometry/geography value is not implemented."); + } + + @Override + public Boolean stDisjoint(Bound valueExpr, Literal literal) { + throw new UnsupportedOperationException( + "Evaluation of stDisjoint against geometry/geography value is not implemented."); + } } } diff --git a/api/src/main/java/org/apache/iceberg/expressions/Expression.java b/api/src/main/java/org/apache/iceberg/expressions/Expression.java index 4a047e08099f..f2729686ab16 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Expression.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Expression.java @@ -44,6 +44,8 @@ enum Operation { OR, STARTS_WITH, NOT_STARTS_WITH, + ST_INTERSECTS, + ST_DISJOINT, COUNT, COUNT_NULL, COUNT_STAR, @@ -91,6 +93,10 @@ public Operation negate() { return Operation.NOT_STARTS_WITH; case NOT_STARTS_WITH: return Operation.STARTS_WITH; + case ST_INTERSECTS: + return Operation.ST_DISJOINT; + case ST_DISJOINT: + return Operation.ST_INTERSECTS; default: throw new IllegalArgumentException("No negation for operation: " + this); } diff --git a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java index d3dc00d914c7..1995669a5dbf 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java @@ -34,6 +34,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Literals.BoundingBoxLiteral; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.transforms.Transforms; @@ -341,6 +342,8 @@ public Expression predicate(UnboundPredicate pred) { case NOT_EQ: case STARTS_WITH: case NOT_STARTS_WITH: + case ST_INTERSECTS: + case ST_DISJOINT: return new UnboundPredicate<>( pred.op(), pred.term(), (T) sanitize(pred.literal(), now, today)); case IN: @@ -441,6 +444,10 @@ public String predicate(BoundPredicate pred) { return term + " STARTS WITH " + value((BoundLiteralPredicate) pred); case NOT_STARTS_WITH: return term + " NOT STARTS WITH " + value((BoundLiteralPredicate) pred); + case ST_INTERSECTS: + return "st_intersects(" + term + ", " + value((BoundLiteralPredicate) pred) + ")"; + case ST_DISJOINT: + return "st_disjoint(" + term + ", " + value((BoundLiteralPredicate) pred) + ")"; default: throw new UnsupportedOperationException( "Cannot sanitize unsupported predicate type: " + pred.op()); @@ -493,6 +500,10 @@ public String predicate(UnboundPredicate pred) { return term + " STARTS WITH " + sanitize(pred.literal(), nowMicros, today); case NOT_STARTS_WITH: return term + " NOT STARTS WITH " + sanitize(pred.literal(), nowMicros, today); + case ST_INTERSECTS: + return "st_intersects(" + term + ", " + sanitize(pred.literal(), nowMicros, today) + ")"; + case ST_DISJOINT: + return "st_disjoint(" + term + ", " + sanitize(pred.literal(), nowMicros, today) + ")"; default: throw new UnsupportedOperationException( "Cannot sanitize unsupported predicate type: " + pred.op()); @@ -552,6 +563,9 @@ private static String sanitize(Type type, Object value, long now, int today) { case BINARY: // for boolean, uuid, decimal, fixed, unknown, and binary, match the string result return sanitizeSimpleString(value.toString()); + case GEOMETRY: + case GEOGRAPHY: + return "(bounding-box)"; } throw new UnsupportedOperationException( String.format("Cannot sanitize value for unsupported type %s: %s", type, value)); @@ -579,6 +593,8 @@ private static String sanitize(Literal literal, long now, int today) { return sanitizeNumber(((Literals.DoubleLiteral) literal).value(), "float"); } else if (literal instanceof Literals.VariantLiteral) { return sanitizeVariant(((Literals.VariantLiteral) literal).value(), now, today); + } else if (literal instanceof BoundingBoxLiteral) { + return "(bounding-box)"; } else { // for uuid, decimal, fixed and binary, match the string result return sanitizeSimpleString(literal.value().toString()); diff --git a/api/src/main/java/org/apache/iceberg/expressions/ExpressionVisitors.java b/api/src/main/java/org/apache/iceberg/expressions/ExpressionVisitors.java index 79ca6a712887..6321333b69b6 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/ExpressionVisitors.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionVisitors.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.expressions; +import java.nio.ByteBuffer; import java.util.Set; import java.util.function.Supplier; import org.apache.iceberg.exceptions.ValidationException; @@ -126,6 +127,16 @@ public R notStartsWith(BoundReference ref, Literal lit) { "notStartsWith expression is not supported by the visitor"); } + public R stIntersects(BoundReference ref, Literal lit) { + throw new UnsupportedOperationException( + "stIntersects expression is not supported by the visitor"); + } + + public R stDisjoint(BoundReference ref, Literal lit) { + throw new UnsupportedOperationException( + "stDisjoint expression is not supported by the visitor"); + } + /** * Handle a non-reference value in this visitor. * @@ -141,6 +152,7 @@ public R handleNonReference(Bound term) { throw new ValidationException("Visitor %s does not support non-reference: %s", this, term); } + @SuppressWarnings("unchecked") @Override public R predicate(BoundPredicate pred) { if (!(pred.term() instanceof BoundReference)) { @@ -166,6 +178,12 @@ public R predicate(BoundPredicate pred) { return startsWith((BoundReference) pred.term(), literalPred.literal()); case NOT_STARTS_WITH: return notStartsWith((BoundReference) pred.term(), literalPred.literal()); + case ST_INTERSECTS: + return stIntersects( + (BoundReference) pred.term(), (Literal) literalPred.literal()); + case ST_DISJOINT: + return stDisjoint( + (BoundReference) pred.term(), (Literal) literalPred.literal()); default: throw new IllegalStateException( "Invalid operation for BoundLiteralPredicate: " + pred.op()); @@ -266,6 +284,14 @@ public R notStartsWith(Bound expr, Literal lit) { throw new UnsupportedOperationException("Unsupported operation."); } + public R stIntersects(Bound term, Literal literal) { + throw new UnsupportedOperationException("ST_INTERSECTS is not supported by the visitor"); + } + + public R stDisjoint(Bound term, Literal literal) { + throw new UnsupportedOperationException("ST_DISJOINT is not supported by the visitor"); + } + @Override public R predicate(BoundPredicate pred) { if (pred.isLiteralPredicate()) { @@ -287,6 +313,10 @@ public R predicate(BoundPredicate pred) { return startsWith(pred.term(), literalPred.literal()); case NOT_STARTS_WITH: return notStartsWith(pred.term(), literalPred.literal()); + case ST_INTERSECTS: + return stIntersects(pred.term(), (Literal) literalPred.literal()); + case ST_DISJOINT: + return stDisjoint(pred.term(), (Literal) literalPred.literal()); default: throw new IllegalStateException( "Invalid operation for BoundLiteralPredicate: " + pred.op()); @@ -318,7 +348,6 @@ public R predicate(BoundPredicate pred) { "Invalid operation for BoundSetPredicate: " + pred.op()); } } - throw new IllegalStateException("Unsupported bound predicate: " + pred.getClass().getName()); } @@ -465,6 +494,10 @@ public R predicate(BoundPredicate pred) { return startsWith(pred.term(), literalPred.literal()); case NOT_STARTS_WITH: return notStartsWith(pred.term(), literalPred.literal()); + case ST_INTERSECTS: + return stIntersects(pred.term(), (Literal) literalPred.literal()); + case ST_DISJOINT: + return stDisjoint(pred.term(), (Literal) literalPred.literal()); default: throw new IllegalStateException( "Invalid operation for BoundLiteralPredicate: " + pred.op()); @@ -555,6 +588,14 @@ public R startsWith(BoundTerm term, Literal lit) { public R notStartsWith(BoundTerm term, Literal lit) { return null; } + + public R stIntersects(BoundTerm term, Literal lit) { + return null; + } + + public R stDisjoint(BoundTerm term, Literal lit) { + return null; + } } /** 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 78012def5a58..9f524af58836 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Expressions.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Expressions.java @@ -18,8 +18,10 @@ */ package org.apache.iceberg.expressions; +import java.nio.ByteBuffer; import java.util.stream.Stream; import org.apache.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.geospatial.BoundingBox; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.transforms.Transform; @@ -202,6 +204,24 @@ public static UnboundPredicate notStartsWith(UnboundTerm expr, S return new UnboundPredicate<>(Expression.Operation.NOT_STARTS_WITH, expr, value); } + public static UnboundPredicate stIntersects(String name, BoundingBox value) { + return geospatialPredicate(Operation.ST_INTERSECTS, name, value); + } + + public static UnboundPredicate stIntersects( + UnboundTerm expr, BoundingBox value) { + return geospatialPredicate(Operation.ST_INTERSECTS, expr, value); + } + + public static UnboundPredicate stDisjoint(String name, BoundingBox value) { + return geospatialPredicate(Operation.ST_DISJOINT, name, value); + } + + public static UnboundPredicate stDisjoint( + UnboundTerm expr, BoundingBox value) { + return geospatialPredicate(Operation.ST_DISJOINT, expr, value); + } + public static UnboundPredicate in(String name, T... values) { return predicate(Operation.IN, name, Lists.newArrayList(values)); } @@ -280,6 +300,16 @@ public static UnboundPredicate predicate(Operation op, UnboundTerm exp return new UnboundPredicate<>(op, expr); } + public static UnboundPredicate geospatialPredicate( + Operation op, String name, BoundingBox value) { + return geospatialPredicate(op, ref(name), value); + } + + public static UnboundPredicate geospatialPredicate( + Operation op, UnboundTerm expr, BoundingBox value) { + return new UnboundPredicate<>(op, expr, Literal.of(value)); + } + public static True alwaysTrue() { return True.INSTANCE; } diff --git a/api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java b/api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java index aa0441f49011..8044de4e99ea 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java +++ b/api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java @@ -29,6 +29,8 @@ import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.Schema; +import org.apache.iceberg.geospatial.BoundingBox; +import org.apache.iceberg.geospatial.GeospatialPredicateEvaluators; import org.apache.iceberg.transforms.Transform; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Conversions; @@ -471,6 +473,35 @@ public Boolean notStartsWith(Bound term, Literal lit) { return ROWS_MIGHT_MATCH; } + @Override + public Boolean stIntersects(Bound term, Literal lit) { + T lower = lowerBound(term); + T upper = upperBound(term); + + if (lower == null || upper == null) { + return ROWS_MIGHT_MATCH; + } + + if (lit.value() != null && lower instanceof ByteBuffer && upper instanceof ByteBuffer) { + BoundingBox dataBox = BoundingBox.fromByteBuffers((ByteBuffer) lower, (ByteBuffer) upper); + BoundingBox queryBox = BoundingBox.fromByteBuffer(lit.value()); + + // If the data box and query box doesn't intersect, no records can match + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(term.ref().type()); + if (!evaluator.intersects(dataBox, queryBox)) { + return ROWS_CANNOT_MATCH; + } + } + + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean stDisjoint(Bound term, Literal lit) { + return ROWS_MIGHT_MATCH; + } + private boolean mayContainNull(Integer id) { return nullCounts == null || !nullCounts.containsKey(id) || nullCounts.get(id) != 0; } diff --git a/api/src/main/java/org/apache/iceberg/expressions/Literal.java b/api/src/main/java/org/apache/iceberg/expressions/Literal.java index b5d6f72f74d0..77e3cba34b04 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Literal.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Literal.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import java.util.Comparator; import java.util.UUID; +import org.apache.iceberg.geospatial.BoundingBox; import org.apache.iceberg.types.Type; /** @@ -71,6 +72,10 @@ static Literal of(BigDecimal value) { return new Literals.DecimalLiteral(value); } + static Literal of(BoundingBox value) { + return new Literals.BoundingBoxLiteral(value); + } + /** Returns the value wrapped by this literal. */ T value(); diff --git a/api/src/main/java/org/apache/iceberg/expressions/Literals.java b/api/src/main/java/org/apache/iceberg/expressions/Literals.java index 3a45eb804f35..08b3f0f3439b 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Literals.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Literals.java @@ -32,6 +32,7 @@ import java.util.Comparator; import java.util.Objects; import java.util.UUID; +import org.apache.iceberg.geospatial.BoundingBox; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; import org.apache.iceberg.types.Comparators; @@ -85,6 +86,8 @@ static Literal from(T value) { return (Literal) new Literals.DecimalLiteral((BigDecimal) value); } else if (value instanceof Variant) { return (Literal) new Literals.VariantLiteral((Variant) value); + } else if (value instanceof BoundingBox) { + return (Literal) new Literals.BoundingBoxLiteral((BoundingBox) value); } throw new IllegalArgumentException( @@ -719,4 +722,45 @@ public String toString() { return "X'" + BaseEncoding.base16().encode(bytes) + "'"; } } + + static class BoundingBoxLiteral extends BaseLiteral { + private static final Comparator CMP = + Comparators.nullsFirst().thenComparing(Comparators.unsignedBytes()); + + BoundingBoxLiteral(BoundingBox value) { + super(value.toByteBuffer()); + } + + BoundingBoxLiteral(ByteBuffer value) { + super(value); + } + + @Override + protected Type.TypeID typeId() { + return null; + } + + @Override + public Literal to(Type type) { + if (type.typeId() != Type.TypeID.GEOMETRY && type.typeId() != Type.TypeID.GEOGRAPHY) { + return null; + } + + return (Literal) this; + } + + @Override + public Comparator comparator() { + return CMP; + } + + Object writeReplace() throws ObjectStreamException { + return new SerializationProxies.BoundingBoxLiteralProxy(value()); + } + + @Override + public String toString() { + return String.valueOf(value()); + } + } } diff --git a/api/src/main/java/org/apache/iceberg/expressions/SerializationProxies.java b/api/src/main/java/org/apache/iceberg/expressions/SerializationProxies.java index 59fd231cd368..19228136ccd2 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/SerializationProxies.java +++ b/api/src/main/java/org/apache/iceberg/expressions/SerializationProxies.java @@ -81,4 +81,18 @@ protected byte[] bytes() { return bytes; } } + + static class BoundingBoxLiteralProxy extends FixedLiteralProxy { + /** Constructor for Java serialization. */ + BoundingBoxLiteralProxy() {} + + BoundingBoxLiteralProxy(ByteBuffer buffer) { + super(buffer); + } + + @Override + Object readResolve() throws ObjectStreamException { + return new Literals.BoundingBoxLiteral(ByteBuffer.wrap(bytes())); + } + } } diff --git a/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java b/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java index c225f21da8a8..975597b4ab7b 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java +++ b/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java @@ -472,6 +472,16 @@ public Boolean notStartsWith(BoundReference ref, Literal lit) { return ROWS_MIGHT_NOT_MATCH; } + @Override + public Boolean stIntersects(BoundReference ref, Literal lit) { + return ROWS_MIGHT_NOT_MATCH; + } + + @Override + public Boolean stDisjoint(BoundReference ref, Literal lit) { + return ROWS_MIGHT_NOT_MATCH; + } + private boolean isNestedColumn(int id) { return struct.field(id) == null; } diff --git a/api/src/main/java/org/apache/iceberg/expressions/UnboundPredicate.java b/api/src/main/java/org/apache/iceberg/expressions/UnboundPredicate.java index 4736ca4a8668..ec2c8a2a39a4 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/UnboundPredicate.java +++ b/api/src/main/java/org/apache/iceberg/expressions/UnboundPredicate.java @@ -18,9 +18,11 @@ */ package org.apache.iceberg.expressions; +import java.nio.ByteBuffer; import java.util.List; import java.util.Set; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.geospatial.BoundingBox; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -276,6 +278,18 @@ public String toString() { return term() + " startsWith \"" + literal() + "\""; case NOT_STARTS_WITH: return term() + " notStartsWith \"" + literal() + "\""; + case ST_INTERSECTS: + return "st_intersects(" + + term() + + ", " + + BoundingBox.fromByteBuffer((ByteBuffer) literal().value()) + + ")"; + case ST_DISJOINT: + return "st_disjoint(" + + term() + + ", " + + BoundingBox.fromByteBuffer((ByteBuffer) literal().value()) + + ")"; case IN: return term() + " in (" + COMMA.join(literals()) + ")"; case NOT_IN: diff --git a/api/src/main/java/org/apache/iceberg/geospatial/BoundingBox.java b/api/src/main/java/org/apache/iceberg/geospatial/BoundingBox.java new file mode 100644 index 000000000000..0bdb36e4f465 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/geospatial/BoundingBox.java @@ -0,0 +1,155 @@ +/* + * 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.geospatial; + +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Objects; + +/** + * Represents a geospatial bounding box composed of minimum and maximum bounds. + * + *

A bounding box (also called a Minimum Bounding Rectangle or MBR) is defined by two points: the + * minimum and maximum coordinates that define the box's corners. This provides a simple + * approximation of a more complex geometry for efficient filtering and data skipping. + */ +public class BoundingBox implements Serializable { + /** + * Create a {@link BoundingBox} object from buffers containing min and max bounds + * + * @param min the serialized minimum bound + * @param max the serialized maximum bound + * @return a BoundingBox instance + */ + public static BoundingBox fromByteBuffers(ByteBuffer min, ByteBuffer max) { + return new BoundingBox( + GeospatialBound.fromByteBuffer(min), GeospatialBound.fromByteBuffer(max)); + } + + /** + * Deserialize a byte buffer as a {@link BoundingBox} object + * + * @param buffer the serialized bounding box + * @return a BoundingBox instance + */ + public static BoundingBox fromByteBuffer(ByteBuffer buffer) { + int originalPosition = buffer.position(); + ByteOrder originalOrder = buffer.order(); + + try { + buffer.order(ByteOrder.LITTLE_ENDIAN); + + int minLen = buffer.getInt(); + ByteBuffer min = buffer.slice(); + min.limit(minLen); + buffer.position(buffer.position() + minLen); + + int maxLen = buffer.getInt(); + ByteBuffer max = buffer.slice(); + max.limit(maxLen); + + return fromByteBuffers(min, max); + } finally { + // Restore original position and byte order + buffer.position(originalPosition); + buffer.order(originalOrder); + } + } + + /** + * Create an empty bounding box + * + * @return an empty bounding box + */ + public static BoundingBox empty() { + return new BoundingBox( + GeospatialBound.createXY(Double.NaN, Double.NaN), + GeospatialBound.createXY(Double.NaN, Double.NaN)); + } + + public BoundingBox(GeospatialBound min, GeospatialBound max) { + this.min = min; + this.max = max; + } + + private final GeospatialBound min; + private final GeospatialBound max; + + /** + * Get the minimum corner of the bounding box. + * + * @return the minimum bound + */ + public GeospatialBound min() { + return min; + } + + /** + * Get the maximum corner of the bounding box. + * + * @return the maximum bound + */ + public GeospatialBound max() { + return max; + } + + /** + * Serializes this bounding box to a byte buffer. The serialized byte buffer could be deserialized + * using {@link #fromByteBuffer(ByteBuffer)}. + * + * @return a byte buffer containing the serialized bounding box + */ + public ByteBuffer toByteBuffer() { + ByteBuffer minBuffer = min.toByteBuffer(); + ByteBuffer maxBuffer = max.toByteBuffer(); + + int totalSize = Integer.BYTES + minBuffer.remaining() + Integer.BYTES + maxBuffer.remaining(); + ByteBuffer buffer = ByteBuffer.allocate(totalSize).order(ByteOrder.LITTLE_ENDIAN); + + buffer.putInt(minBuffer.remaining()); + buffer.put(minBuffer); + buffer.putInt(maxBuffer.remaining()); + buffer.put(maxBuffer); + buffer.flip(); + return buffer; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (!(other instanceof BoundingBox)) { + return false; + } + + BoundingBox that = (BoundingBox) other; + return Objects.equals(min, that.min) && Objects.equals(max, that.max); + } + + @Override + public int hashCode() { + return Objects.hash(min, max); + } + + @Override + public String toString() { + return "BoundingBox{min={" + min.simpleString() + "}, max={" + max.simpleString() + "}}"; + } +} diff --git a/api/src/main/java/org/apache/iceberg/geospatial/GeospatialBound.java b/api/src/main/java/org/apache/iceberg/geospatial/GeospatialBound.java new file mode 100644 index 000000000000..41d8e7290726 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/geospatial/GeospatialBound.java @@ -0,0 +1,328 @@ +/* + * 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.geospatial; + +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Objects; + +/** + * Represents a geospatial bound (minimum or maximum) for Iceberg tables. + * + *

According to the Bound + * serialization section of Iceberg Table spec, geospatial bounds are serialized differently + * from the regular WKB representation. Geometry and geography bounds are single point encoded as a + * concatenation of 8-byte little-endian IEEE 754 coordinate values in the order X, Y, Z (optional), + * M (optional). + * + *

The encoding varies based on which coordinates are present: + * + *

    + *
  • x:y (2 doubles) when both z and m are unset + *
  • x:y:z (3 doubles) when only m is unset + *
  • x:y:NaN:m (4 doubles) when only z is unset + *
  • x:y:z:m (4 doubles) when all coordinates are set + *
+ * + *

This class represents a lower or upper geospatial bound and handles serialization and + * deserialization of these bounds to/from byte arrays, conforming to the Iceberg specification. + */ +public class GeospatialBound implements Serializable { + /** + * Parses a geospatial bound from a byte buffer according to Iceberg spec. + * + *

Based on the buffer size, this method determines which coordinates are present: - 16 bytes + * (2 doubles): x and y only - 24 bytes (3 doubles): x, y, and z - 32 bytes (4 doubles): x, y, z + * (might be NaN), and m + * + *

The ordinates are encoded as 8-byte little-endian IEEE 754 values. + * + * @param buffer the ByteBuffer containing the serialized geospatial bound + * @return a GeospatialBound object representing the parsed bound + * @throws IllegalArgumentException if the buffer has an invalid size + */ + public static GeospatialBound fromByteBuffer(ByteBuffer buffer) { + // Save original position and byte order to restore them later + int originalPosition = buffer.position(); + ByteOrder originalOrder = buffer.order(); + + try { + buffer.order(ByteOrder.LITTLE_ENDIAN); + int size = buffer.remaining(); + + if (size == 2 * Double.BYTES) { + // x:y format (2 doubles) + double coordX = buffer.getDouble(); + double coordY = buffer.getDouble(); + return createXY(coordX, coordY); + } else if (size == 3 * Double.BYTES) { + // x:y:z format (3 doubles) + double coordX = buffer.getDouble(); + double coordY = buffer.getDouble(); + double coordZ = buffer.getDouble(); + return createXYZ(coordX, coordY, coordZ); + } else if (size == 4 * Double.BYTES) { + // x:y:z:m format (4 doubles) - z might be NaN + double coordX = buffer.getDouble(); + double coordY = buffer.getDouble(); + double coordZ = buffer.getDouble(); + double coordM = buffer.getDouble(); + return new GeospatialBound(coordX, coordY, coordZ, coordM); + } else { + throw new IllegalArgumentException( + "Invalid buffer size for GeospatialBound: expected 16, 24, or 32 bytes, got " + size); + } + } finally { + // Restore original position and byte order + buffer.position(originalPosition); + buffer.order(originalOrder); + } + } + + /** + * Parses a geospatial bound from a byte array according to Iceberg spec. + * + * @param bytes the byte array containing the serialized geospatial bound + * @return a GeospatialBound object representing the parsed bound + * @throws IllegalArgumentException if the byte array has an invalid length + */ + public static GeospatialBound fromByteArray(byte[] bytes) { + int length = bytes.length; + if (length != 2 * Double.BYTES && length != 3 * Double.BYTES && length != 4 * Double.BYTES) { + throw new IllegalArgumentException( + "Invalid byte array length for GeospatialBound: expected 16, 24, or 32 bytes, got " + + length); + } + + return fromByteBuffer(ByteBuffer.wrap(bytes)); + } + + /** + * Creates a GeospatialBound with X and Y coordinates only. + * + * @param x the X coordinate (longitude/easting) + * @param y the Y coordinate (latitude/northing) + * @return a GeospatialBound with XY coordinates + */ + @SuppressWarnings("ParameterName") + public static GeospatialBound createXY(double x, double y) { + return new GeospatialBound(x, y, Double.NaN, Double.NaN); + } + + /** + * Creates a GeospatialBound with X, Y, and Z coordinates, with no M value. + * + * @param x the X coordinate (longitude/easting) + * @param y the Y coordinate (latitude/northing) + * @param z the Z coordinate (elevation) + * @return a GeospatialBound with XYZ coordinates + */ + @SuppressWarnings("ParameterName") + public static GeospatialBound createXYZ(double x, double y, double z) { + return new GeospatialBound(x, y, z, Double.NaN); + } + + /** + * Creates a GeospatialBound with X, Y, Z, and M coordinates. + * + * @param x the X coordinate (longitude/easting) + * @param y the Y coordinate (latitude/northing) + * @param z the Z coordinate (elevation) + * @param m the M value (measure) + * @return a GeospatialBound with XYZM coordinates + */ + @SuppressWarnings("ParameterName") + public static GeospatialBound createXYZM(double x, double y, double z, double m) { + return new GeospatialBound(x, y, z, m); + } + + /** + * Creates a GeospatialBound with X, Y, and M values, with no Z coordinate. + * + * @param x the X coordinate (longitude/easting) + * @param y the Y coordinate (latitude/northing) + * @param m the M value (measure) + * @return a GeospatialBound with XYM coordinates + */ + @SuppressWarnings("ParameterName") + public static GeospatialBound createXYM(double x, double y, double m) { + return new GeospatialBound(x, y, Double.NaN, m); + } + + @SuppressWarnings("MemberName") + private final double x; + + @SuppressWarnings("MemberName") + private final double y; + + @SuppressWarnings("MemberName") + private final double z; + + @SuppressWarnings("MemberName") + private final double m; + + /** Private constructor - use factory methods instead. */ + @SuppressWarnings("ParameterName") + private GeospatialBound(double x, double y, double z, double m) { + this.x = x; + this.y = y; + this.z = z; + this.m = m; + } + + /** + * Get the X coordinate (longitude/easting). + * + * @return X coordinate value + */ + @SuppressWarnings("MethodName") + public double x() { + return x; + } + + /** + * Get the Y coordinate (latitude/northing). + * + * @return Y coordinate value + */ + @SuppressWarnings("MethodName") + public double y() { + return y; + } + + /** + * Get the Z coordinate (typically elevation). + * + * @return Z coordinate value or NaN if not set + */ + @SuppressWarnings("MethodName") + public double z() { + return z; + } + + /** + * Get the M value (measure). + * + * @return M value or NaN if not set + */ + @SuppressWarnings("MethodName") + public double m() { + return m; + } + + /** + * Check if this bound has a defined Z coordinate. + * + * @return true if Z is not NaN + */ + public boolean hasZ() { + return !Double.isNaN(z); + } + + /** + * Check if this bound has a defined M value. + * + * @return true if M is not NaN + */ + public boolean hasM() { + return !Double.isNaN(m); + } + + /** + * Serializes this geospatial bound to a byte buffer according to Iceberg spec. + * + *

Following the Iceberg spec, the bound is serialized based on which coordinates are set: - + * x:y (2 doubles) when both z and m are unset - x:y:z (3 doubles) when only m is unset - + * x:y:NaN:m (4 doubles) when only z is unset - x:y:z:m (4 doubles) when all coordinates are set + * + * @return A ByteBuffer containing the serialized geospatial bound + */ + public ByteBuffer toByteBuffer() { + // Calculate size based on which coordinates are present + int size; + if (!hasZ() && !hasM()) { + // Just x and y + size = 2 * Double.BYTES; + } else if (hasZ() && !hasM()) { + // x, y, and z (no m) + size = 3 * Double.BYTES; + } else { + // x, y, z (or NaN), and m + size = 4 * Double.BYTES; + } + + ByteBuffer buffer = ByteBuffer.allocate(size).order(ByteOrder.LITTLE_ENDIAN); + buffer.putDouble(x); + buffer.putDouble(y); + + if (hasZ() || hasM()) { + // If we have z or m or both, we need to include z (could be NaN) + buffer.putDouble(z); + } + + if (hasM()) { + // If we have m, include it + buffer.putDouble(m); + } + + buffer.flip(); + return buffer; + } + + @Override + public String toString() { + return "GeospatialBound(" + simpleString() + ")"; + } + + public String simpleString() { + StringBuilder sb = new StringBuilder(); + sb.append("x=").append(x).append(", y=").append(y); + + if (hasZ()) { + sb.append(", z=").append(z); + } + + if (hasM()) { + sb.append(", m=").append(m); + } + + return sb.toString(); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (!(other instanceof GeospatialBound)) { + return false; + } + + GeospatialBound that = (GeospatialBound) other; + return Double.compare(that.x, x) == 0 + && Double.compare(that.y, y) == 0 + && Double.compare(that.z, z) == 0 + && Double.compare(that.m, m) == 0; + } + + @Override + public int hashCode() { + return Objects.hash(GeospatialBound.class, x, y, z, m); + } +} diff --git a/api/src/main/java/org/apache/iceberg/geospatial/GeospatialPredicateEvaluators.java b/api/src/main/java/org/apache/iceberg/geospatial/GeospatialPredicateEvaluators.java new file mode 100644 index 000000000000..64943902445c --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/geospatial/GeospatialPredicateEvaluators.java @@ -0,0 +1,128 @@ +/* + * 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.geospatial; + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Type; + +public class GeospatialPredicateEvaluators { + private GeospatialPredicateEvaluators() {} + + public interface GeospatialPredicateEvaluator { + /** + * Test whether this bounding box intersects with another. + * + * @param bbox1 the first bounding box + * @param bbox2 the second bounding box + * @return true if this box intersects the other box + */ + boolean intersects(BoundingBox bbox1, BoundingBox bbox2); + } + + public static GeospatialPredicateEvaluator create(Type type) { + switch (type.typeId()) { + case GEOMETRY: + return new GeometryEvaluator(); + case GEOGRAPHY: + return new GeographyEvaluator(); + default: + throw new UnsupportedOperationException("Unsupported type for BoundingBox: " + type); + } + } + + static class GeometryEvaluator implements GeospatialPredicateEvaluator { + @Override + public boolean intersects(BoundingBox bbox1, BoundingBox bbox2) { + return intersectsWithWrapAround(bbox1, bbox2); + } + + /** + * Check if two bounding boxes intersect, taking wrap-around into account. + * + *

Wraparound (or antimeridian crossing) occurs when a geography crosses the 180°/-180° + * longitude line on a map. In these cases, the minimum X value is greater than the maximum X + * value (xmin > xmax). This represents a bounding box that wraps around the globe. + * + *

For example, a bounding box with xmin=170° and xmax=-170° represents an area that spans + * from 170° east to 190° east (or equivalently, -170° west). This is important for geometries + * that cross the antimeridian, like a path from Japan to Alaska. + * + *

When xmin > xmax, a point matches if its X coordinate is either X ≥ xmin OR X ≤ xmax, + * rather than the usual X ≥ xmin AND X ≤ xmax. In geographic terms, if the westernmost + * longitude is greater than the easternmost longitude, this indicates an antimeridian crossing. + * + *

The Iceberg specification does not explicitly rule out the use of wrap-around in bounding + * boxes for geometry types, so we handle wrap-around for both geography and geometry bounding + * boxes. + * + * @param bbox1 the first bounding box + * @param bbox2 the second bounding box + * @return true if the bounding boxes intersect + */ + static boolean intersectsWithWrapAround(BoundingBox bbox1, BoundingBox bbox2) { + // Let's check y first, and if y does not intersect, we can return false + if (bbox1.min().y() > bbox2.max().y() || bbox1.max().y() < bbox2.min().y()) { + return false; + } + + // Now check x, need to take wrap-around into account + if (bbox1.min().x() <= bbox1.max().x() && bbox2.min().x() <= bbox2.max().x()) { + // No wrap-around + return bbox1.min().x() <= bbox2.max().x() && bbox1.max().x() >= bbox2.min().x(); + } else if (bbox1.min().x() > bbox1.max().x() && bbox2.min().x() <= bbox2.max().x()) { + // bbox1 wraps around the antimeridian, bbox2 does not + return bbox1.min().x() <= bbox2.max().x() || bbox1.max().x() >= bbox2.min().x(); + } else if (bbox1.min().x() <= bbox1.max().x() && bbox2.min().x() > bbox2.max().x()) { + // bbox2 wraps around the antimeridian, bbox1 does not + return intersectsWithWrapAround(bbox2, bbox1); + } else { + // Both wrap around the antimeridian, they must intersect + return true; + } + } + } + + static class GeographyEvaluator implements GeospatialPredicateEvaluator { + @Override + public boolean intersects(BoundingBox bbox1, BoundingBox bbox2) { + validateBoundingBox(bbox1); + validateBoundingBox(bbox2); + return GeometryEvaluator.intersectsWithWrapAround(bbox1, bbox2); + } + + /** + * For geography types, coordinates are restricted to the canonical ranges of [-180°, 180°] for + * longitude (X) and [-90°, 90°] for latitude (Y). + * + * @param bbox the bounding box to validate + * @throws IllegalArgumentException if the bounding box is invalid + */ + private void validateBoundingBox(BoundingBox bbox) { + Preconditions.checkArgument( + bbox.min().y() >= -90 && bbox.max().y() <= 90, "Latitude out of range: %s", bbox); + Preconditions.checkArgument( + bbox.min().x() >= -180 + && bbox.min().x() <= 180 + && bbox.max().x() >= -180 + && bbox.max().x() <= 180, + "Longitude out of range: %s", + bbox); + } + } +} diff --git a/api/src/main/java/org/apache/iceberg/types/Conversions.java b/api/src/main/java/org/apache/iceberg/types/Conversions.java index e18c7b4362e6..54074756a1b1 100644 --- a/api/src/main/java/org/apache/iceberg/types/Conversions.java +++ b/api/src/main/java/org/apache/iceberg/types/Conversions.java @@ -117,6 +117,19 @@ public static ByteBuffer toByteBuffer(Type.TypeID typeId, Object value) { return (ByteBuffer) value; case DECIMAL: return ByteBuffer.wrap(((BigDecimal) value).unscaledValue().toByteArray()); + case GEOMETRY: + case GEOGRAPHY: + // There are 2 representations of geometry and geography in iceberg: + // + // 1. Well-known binary (WKB) format for general storage and processing + // 2. For bound values (partition and sort keys), points are encoded as little-endian + // doubles: + // X (longitude/easting), Y (latitude/northing), Z (optional elevation), and M (optional + // measure) + // + // No matter what representation is used, geospatial values are always represented as byte + // buffers, so we can just return the value as is. + return (ByteBuffer) value; default: throw new UnsupportedOperationException("Cannot serialize type: " + typeId); } @@ -177,6 +190,11 @@ private static Object internalFromByteBuffer(Type type, ByteBuffer buffer) { byte[] unscaledBytes = new byte[buffer.remaining()]; tmp.get(unscaledBytes); return new BigDecimal(new BigInteger(unscaledBytes), decimal.scale()); + case GEOMETRY: + case GEOGRAPHY: + // GEOMETRY and GEOGRAPHY values are represented as byte buffers. Please refer to the + // comment in toByteBuffer for more details. + return tmp; default: throw new UnsupportedOperationException("Cannot deserialize type: " + type); } diff --git a/api/src/main/java/org/apache/iceberg/types/Types.java b/api/src/main/java/org/apache/iceberg/types/Types.java index 1c16c444d4e6..ec6076b04fa0 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -81,7 +81,6 @@ public static Type fromTypeName(String typeString) { Matcher geometry = GEOMETRY_PARAMETERS.matcher(typeString); if (geometry.matches()) { String crs = geometry.group(1); - Preconditions.checkArgument(!crs.contains(","), "Invalid CRS: %s", crs); return GeometryType.of(crs); } @@ -599,7 +598,7 @@ public TypeID typeId() { } public String crs() { - return crs; + return crs != null ? crs : DEFAULT_CRS; } @Override @@ -631,6 +630,7 @@ public String toString() { public static class GeographyType extends PrimitiveType { public static final String DEFAULT_CRS = "OGC:CRS84"; + public static final EdgeAlgorithm DEFAULT_ALGORITHM = EdgeAlgorithm.SPHERICAL; public static GeographyType crs84() { return new GeographyType(); @@ -664,11 +664,11 @@ public TypeID typeId() { } public String crs() { - return crs; + return crs != null ? crs : DEFAULT_CRS; } public EdgeAlgorithm algorithm() { - return algorithm; + return algorithm != null ? algorithm : DEFAULT_ALGORITHM; } @Override diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestEvaluator.java index 995dde539f8c..757ab5690193 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestEvaluator.java @@ -43,15 +43,22 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.stream.Stream; import org.apache.avro.util.Utf8; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.geospatial.BoundingBox; +import org.apache.iceberg.geospatial.GeospatialBound; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.StructType; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; public class TestEvaluator { private static final StructType STRUCT = @@ -809,4 +816,36 @@ public void testNotInExceptions() { .isInstanceOf(ValidationException.class) .hasMessageContaining("Invalid value for conversion to type int"); } + + private static Stream geospatialPredicateParameters() { + return Stream.of( + Arguments.of(Expression.Operation.ST_INTERSECTS, "geom"), + Arguments.of(Expression.Operation.ST_INTERSECTS, "geog"), + Arguments.of(Expression.Operation.ST_DISJOINT, "geom"), + Arguments.of(Expression.Operation.ST_DISJOINT, "geog")); + } + + @ParameterizedTest + @MethodSource("geospatialPredicateParameters") + public void testGeospatialPredicates(Expression.Operation operation, String columnName) { + StructType geoStruct = + StructType.of( + required(1, "geom", Types.GeometryType.crs84()), + required(2, "geog", Types.GeographyType.crs84())); + + BoundingBox bbox = + new BoundingBox(GeospatialBound.createXY(1.0, 2.0), GeospatialBound.createXY(3.0, 4.0)); + + // Create a WKB point at (2, 3) + ByteBuffer wkb = + ByteBuffer.wrap( + new byte[] {1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 64, 0, 0, 0, 0, 0, 0, 8, 64}); + + Evaluator evaluator = + new Evaluator(geoStruct, Expressions.geospatialPredicate(operation, columnName, bbox)); + assertThatThrownBy(() -> evaluator.eval(TestHelpers.Row.of(wkb, wkb))) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageMatching( + "Evaluation of \\w+ against geometry/geography value is not implemented."); + } } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionBinding.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionBinding.java index 5293681f6f5d..4bbf921dae82 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionBinding.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionBinding.java @@ -40,6 +40,8 @@ import org.apache.iceberg.TestHelpers; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.geospatial.BoundingBox; +import org.apache.iceberg.geospatial.GeospatialBound; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.StructType; import org.junit.jupiter.api.Test; @@ -55,7 +57,9 @@ public class TestExpressionBinding { required(3, "data", Types.StringType.get()), required(4, "var", Types.VariantType.get()), optional(5, "nullable", Types.IntegerType.get()), - optional(6, "always_null", Types.UnknownType.get())); + optional(6, "always_null", Types.UnknownType.get()), + required(7, "point", Types.GeometryType.crs84()), + required(8, "geography", Types.GeographyType.crs84())); @Test public void testMissingReference() { @@ -421,4 +425,40 @@ public void testExtractBindingWithTypes(String typeName) { assertThat(pred.term()).as("Should use a BoundExtract").isInstanceOf(BoundExtract.class); assertThat(pred.term().type()).isEqualTo(Types.fromPrimitiveString(typeName)); } + + @Test + public void testStIntersects() { + // Create a bounding box for testing + GeospatialBound min = GeospatialBound.createXY(1.0, 2.0); + GeospatialBound max = GeospatialBound.createXY(3.0, 4.0); + BoundingBox bbox = new BoundingBox(min, max); + + Expression expr = + Expressions.geospatialPredicate(Expression.Operation.ST_INTERSECTS, "point", bbox); + Expression bound = Binder.bind(STRUCT, expr); + + TestHelpers.assertAllReferencesBound("ST_Intersects", bound); + BoundPredicate pred = TestHelpers.assertAndUnwrap(bound); + assertThat(pred.op()).isEqualTo(Expression.Operation.ST_INTERSECTS); + assertThat(pred.term().ref().fieldId()).as("Should bind point correctly").isEqualTo(7); + assertThat(pred.asLiteralPredicate().literal().value()).isEqualTo(bbox.toByteBuffer()); + } + + @Test + public void testStDisjoint() { + // Create a bounding box for testing + GeospatialBound min = GeospatialBound.createXY(1.0, 2.0); + GeospatialBound max = GeospatialBound.createXY(3.0, 4.0); + BoundingBox bbox = new BoundingBox(min, max); + + Expression expr = + Expressions.geospatialPredicate(Expression.Operation.ST_DISJOINT, "geography", bbox); + Expression bound = Binder.bind(STRUCT, expr); + + TestHelpers.assertAllReferencesBound("ST_Disjoint", bound); + BoundPredicate pred = TestHelpers.assertAndUnwrap(bound); + assertThat(pred.op()).isEqualTo(Expression.Operation.ST_DISJOINT); + assertThat(pred.term().ref().fieldId()).as("Should bind geography correctly").isEqualTo(8); + assertThat(pred.asLiteralPredicate().literal().value()).isEqualTo(bbox.toByteBuffer()); + } } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionHelpers.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionHelpers.java index 2a1fab10a445..09c643afd164 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionHelpers.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionHelpers.java @@ -50,6 +50,8 @@ import java.util.List; import java.util.concurrent.Callable; import org.apache.iceberg.Schema; +import org.apache.iceberg.geospatial.BoundingBox; +import org.apache.iceberg.geospatial.GeospatialBound; import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; @@ -273,6 +275,73 @@ public void testInvalidateNaNInput() { assertInvalidateNaNThrows(() -> predicate(Expression.Operation.EQ, "a", Double.NaN)); } + @Test + public void testRewriteNotForGeospatialPredicates() { + // Create a schema with geometry and geography fields + StructType struct = + StructType.of( + NestedField.optional(1, "geom", Types.GeometryType.crs84()), + NestedField.optional(2, "geog", Types.GeographyType.crs84())); + + // Create a bounding box for testing + GeospatialBound min = GeospatialBound.createXY(1.0, 2.0); + GeospatialBound max = GeospatialBound.createXY(3.0, 4.0); + BoundingBox bbox = new BoundingBox(min, max); + + // Test pairs of expressions: (rewritten pred, original pred) + Expression[][] expressions = + new Expression[][] { + // ST_INTERSECTS and its negation (ST_DISJOINT) + { + Expressions.geospatialPredicate(Expression.Operation.ST_INTERSECTS, "geom", bbox), + Expressions.geospatialPredicate(Expression.Operation.ST_INTERSECTS, "geom", bbox) + }, + { + Expressions.geospatialPredicate(Expression.Operation.ST_DISJOINT, "geom", bbox), + not(Expressions.geospatialPredicate(Expression.Operation.ST_INTERSECTS, "geom", bbox)) + }, + { + Expressions.geospatialPredicate(Expression.Operation.ST_DISJOINT, "geom", bbox), + Expressions.geospatialPredicate(Expression.Operation.ST_DISJOINT, "geom", bbox) + }, + { + Expressions.geospatialPredicate(Expression.Operation.ST_INTERSECTS, "geom", bbox), + not(Expressions.geospatialPredicate(Expression.Operation.ST_DISJOINT, "geom", bbox)) + }, + // Same tests with geography type + { + Expressions.geospatialPredicate(Expression.Operation.ST_INTERSECTS, "geog", bbox), + Expressions.geospatialPredicate(Expression.Operation.ST_INTERSECTS, "geog", bbox) + }, + { + Expressions.geospatialPredicate(Expression.Operation.ST_DISJOINT, "geog", bbox), + not(Expressions.geospatialPredicate(Expression.Operation.ST_INTERSECTS, "geog", bbox)) + }, + { + Expressions.geospatialPredicate(Expression.Operation.ST_DISJOINT, "geog", bbox), + Expressions.geospatialPredicate(Expression.Operation.ST_DISJOINT, "geog", bbox) + }, + { + Expressions.geospatialPredicate(Expression.Operation.ST_INTERSECTS, "geog", bbox), + not(Expressions.geospatialPredicate(Expression.Operation.ST_DISJOINT, "geog", bbox)) + } + }; + + for (Expression[] pair : expressions) { + // unbound rewrite + assertThat(rewriteNot(pair[1])) + .as(String.format("rewriteNot(%s) should be %s", pair[1], pair[0])) + .hasToString(pair[0].toString()); + + // bound rewrite + Expression expectedBound = Binder.bind(struct, pair[0]); + Expression toRewriteBound = Binder.bind(struct, pair[1]); + assertThat(rewriteNot(toRewriteBound)) + .as(String.format("rewriteNot(%s) should be %s", toRewriteBound, expectedBound)) + .hasToString(expectedBound.toString()); + } + } + private void assertInvalidateNaNThrows(Callable> callable) { assertThatThrownBy(callable::call) .isInstanceOf(IllegalArgumentException.class) diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionSerialization.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionSerialization.java index fc7ddd035bf2..7aece5e6046d 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionSerialization.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionSerialization.java @@ -24,6 +24,8 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.geospatial.BoundingBox; +import org.apache.iceberg.geospatial.GeospatialBound; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; @@ -33,7 +35,9 @@ public void testExpressions() throws Exception { Schema schema = new Schema( Types.NestedField.optional(34, "a", Types.IntegerType.get()), - Types.NestedField.required(35, "s", Types.StringType.get())); + Types.NestedField.required(35, "s", Types.StringType.get()), + Types.NestedField.required(36, "point", Types.GeometryType.crs84()), + Types.NestedField.required(37, "geography", Types.GeographyType.crs84())); Expression[] expressions = new Expression[] { @@ -61,7 +65,26 @@ public void testExpressions() throws Exception { Expressions.notIn("s", "abc", "xyz").bind(schema.asStruct()), Expressions.isNull("a").bind(schema.asStruct()), Expressions.startsWith("s", "abc").bind(schema.asStruct()), - Expressions.notStartsWith("s", "xyz").bind(schema.asStruct()) + Expressions.notStartsWith("s", "xyz").bind(schema.asStruct()), + Expressions.notStartsWith("s", "xyz").bind(schema.asStruct()), + Expressions.stIntersects( + "point", + new BoundingBox( + GeospatialBound.createXY(1.0, 2.0), GeospatialBound.createXY(3.0, 4.0))), + Expressions.stDisjoint( + "geography", + new BoundingBox( + GeospatialBound.createXY(5.0, 6.0), GeospatialBound.createXY(7.0, 8.0))), + Expressions.stIntersects( + "point", + new BoundingBox( + GeospatialBound.createXY(1.0, 2.0), GeospatialBound.createXY(3.0, 4.0))) + .bind(schema.asStruct()), + Expressions.stDisjoint( + "geography", + new BoundingBox( + GeospatialBound.createXY(5.0, 6.0), GeospatialBound.createXY(7.0, 8.0))) + .bind(schema.asStruct()) }; for (Expression expression : expressions) { @@ -149,7 +172,10 @@ private static boolean equals(Predicate left, Predicate right) { if (left instanceof UnboundPredicate) { UnboundPredicate lpred = (UnboundPredicate) left; UnboundPredicate rpred = (UnboundPredicate) right; - if (left.op() == Operation.IN || left.op() == Operation.NOT_IN) { + if (left.op() == Operation.IN + || left.op() == Operation.NOT_IN + || left.op() == Operation.ST_INTERSECTS + || left.op() == Operation.ST_DISJOINT) { return equals(lpred.literals(), rpred.literals()); } return lpred.literal().comparator().compare(lpred.literal().value(), rpred.literal().value()) diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java index ca08951b1f53..be60ccadf98b 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java @@ -28,11 +28,15 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.regex.Pattern; import java.util.stream.IntStream; +import java.util.stream.Stream; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.geospatial.BoundingBox; +import org.apache.iceberg.geospatial.GeospatialBound; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; @@ -45,6 +49,9 @@ import org.apache.iceberg.variants.VariantTestUtil; import org.apache.iceberg.variants.VariantValue; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; public class TestExpressionUtil { private static final Schema SCHEMA = @@ -1305,6 +1312,48 @@ private static VariantArray createArrayWithNestedTypes() { return (VariantArray) VariantValue.from(metadata, variantBB); } + private static Stream geospatialPredicateParameters() { + return Stream.of( + Arguments.of(Expression.Operation.ST_INTERSECTS, "geom"), + Arguments.of(Expression.Operation.ST_INTERSECTS, "geog"), + Arguments.of(Expression.Operation.ST_DISJOINT, "geom"), + Arguments.of(Expression.Operation.ST_DISJOINT, "geog")); + } + + @ParameterizedTest + @MethodSource("geospatialPredicateParameters") + public void testSanitizeGeospatialPredicates(Expression.Operation operation, String columnName) { + // Create a schema with geometry and geography fields + Schema geoSchema = + new Schema( + Types.NestedField.required(1, "geom", Types.GeometryType.crs84()), + Types.NestedField.required(2, "geog", Types.GeographyType.crs84())); + Types.StructType geoStruct = geoSchema.asStruct(); + + // Create a bounding box for testing + GeospatialBound min = GeospatialBound.createXY(1.0, 2.0); + GeospatialBound max = GeospatialBound.createXY(3.0, 4.0); + BoundingBox bbox = new BoundingBox(min, max); + + UnboundPredicate geoPredicate = + Expressions.geospatialPredicate(operation, columnName, bbox); + Expression predicateSanitized = Expressions.predicate(operation, columnName, "(bounding-box)"); + assertEquals(predicateSanitized, ExpressionUtil.sanitize(geoPredicate)); + assertEquals(predicateSanitized, ExpressionUtil.sanitize(geoStruct, geoPredicate, true)); + + String opString = operation.name(); + String expectedSanitizedString = + opString.toLowerCase(Locale.ROOT) + "(" + columnName + ", (bounding-box))"; + + assertThat(ExpressionUtil.toSanitizedString(geoPredicate)) + .as("Sanitized string should be identical for geospatial predicates") + .isEqualTo(expectedSanitizedString); + + assertThat(ExpressionUtil.toSanitizedString(geoStruct, geoPredicate, true)) + .as("Sanitized string should be identical for geospatial predicates") + .isEqualTo(expectedSanitizedString); + } + private void assertEquals(Expression expected, Expression actual) { assertThat(expected).isInstanceOf(UnboundPredicate.class); assertEquals((UnboundPredicate) expected, (UnboundPredicate) actual); diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java index 7069d891c38d..dec7b2cb71cb 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java @@ -34,6 +34,8 @@ import static org.apache.iceberg.expressions.Expressions.notNull; import static org.apache.iceberg.expressions.Expressions.notStartsWith; import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.expressions.Expressions.stDisjoint; +import static org.apache.iceberg.expressions.Expressions.stIntersects; import static org.apache.iceberg.expressions.Expressions.startsWith; import static org.apache.iceberg.types.Conversions.toByteBuffer; import static org.apache.iceberg.types.Types.NestedField.optional; @@ -47,6 +49,8 @@ import org.apache.iceberg.TestHelpers.Row; import org.apache.iceberg.TestHelpers.TestDataFile; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.geospatial.BoundingBox; +import org.apache.iceberg.geospatial.GeospatialBound; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; @@ -71,7 +75,11 @@ public class TestInclusiveMetricsEvaluator { optional(11, "all_nans_v1_stats", Types.FloatType.get()), optional(12, "nan_and_null_only", Types.DoubleType.get()), optional(13, "no_nan_stats", Types.DoubleType.get()), - optional(14, "some_empty", Types.StringType.get())); + optional(14, "some_empty", Types.StringType.get()), + optional(15, "geom", Types.GeometryType.crs84()), + optional(16, "all_nulls_geom", Types.GeometryType.crs84()), + optional(17, "geog", Types.GeographyType.crs84()), + optional(18, "all_nulls_geog", Types.GeographyType.crs84())); private static final int INT_MIN_VALUE = 30; private static final int INT_MAX_VALUE = 79; @@ -187,6 +195,40 @@ public class TestInclusiveMetricsEvaluator { // upper bounds ImmutableMap.of(3, toByteBuffer(StringType.get(), "abcdefghi"))); + private static final DataFile FILE_6 = + new TestDataFile( + "file_6.avro", + Row.of(), + 50, + // any value counts, including nulls + ImmutableMap.builder() + .put(15, 20L) + .put(16, 20L) + .put(17, 20L) + .put(18, 20L) + .buildOrThrow(), + // null value counts + ImmutableMap.builder() + .put(15, 2L) + .put(16, 20L) + .put(17, 2L) + .put(18, 20L) + .buildOrThrow(), + // nan value counts + null, + // lower bounds + ImmutableMap.of( + 15, + GeospatialBound.createXY(1, 2).toByteBuffer(), + 17, + GeospatialBound.createXY(1, 2).toByteBuffer()), + // upper bounds + ImmutableMap.of( + 15, + GeospatialBound.createXY(10, 20).toByteBuffer(), + 17, + GeospatialBound.createXY(10, 20).toByteBuffer())); + @Test public void testAllNulls() { boolean shouldRead = new InclusiveMetricsEvaluator(SCHEMA, notNull("all_nulls")).eval(FILE); @@ -863,4 +905,62 @@ public void testIntegerNotIn() { shouldRead = new InclusiveMetricsEvaluator(SCHEMA, notIn("no_nulls", "abc", "def")).eval(FILE); assertThat(shouldRead).as("Should read: notIn on no nulls column").isTrue(); } + + @Test + public void testStIntersects() { + boolean shouldRead = + new InclusiveMetricsEvaluator( + SCHEMA, + stIntersects( + "geom", + new BoundingBox( + GeospatialBound.createXY(0, 0), GeospatialBound.createXY(3, 4)))) + .eval(FILE_6); + assertThat(shouldRead).as("Should read: query window intersects the boundary").isTrue(); + + shouldRead = + new InclusiveMetricsEvaluator( + SCHEMA, + stIntersects( + "geom", + new BoundingBox( + GeospatialBound.createXY(0, 0), GeospatialBound.createXY(0.5, 2)))) + .eval(FILE_6); + assertThat(shouldRead) + .as("Should skip: query window does not intersect with the boundary") + .isFalse(); + + shouldRead = + new InclusiveMetricsEvaluator( + SCHEMA, + stIntersects( + "geom", + new BoundingBox( + GeospatialBound.createXY(0, 0), GeospatialBound.createXY(0.5, 2)))) + .eval(FILE); + assertThat(shouldRead).as("Should read: stats is missing").isTrue(); + } + + @Test + public void testStDisjoint() { + boolean shouldRead = + new InclusiveMetricsEvaluator( + SCHEMA, + stDisjoint( + "geom", + new BoundingBox( + GeospatialBound.createXY(0, 0), GeospatialBound.createXY(3, 4)))) + .eval(FILE_6); + assertThat(shouldRead).as("Should read: always read no matter if it's disjoint").isTrue(); + + shouldRead = + new InclusiveMetricsEvaluator( + SCHEMA, + stDisjoint( + "geom", + new BoundingBox( + GeospatialBound.createXY(0, 0), GeospatialBound.createXY(0.5, 2)))) + .eval(FILE); + assertThat(shouldRead).as("Should read: stats is missing").isTrue(); + } } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java b/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java index 24fc458b37b4..d82032b258b6 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java @@ -23,6 +23,8 @@ import java.math.BigDecimal; import java.util.UUID; import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.geospatial.BoundingBox; +import org.apache.iceberg.geospatial.GeospatialBound; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; @@ -47,6 +49,9 @@ public void testLiterals() throws Exception { Literal.of(new byte[] {1, 2, 3}).to(Types.FixedType.ofLength(3)), Literal.of(new byte[] {3, 4, 5, 6}).to(Types.BinaryType.get()), Literal.of(new BigDecimal("122.50")), + Literal.of( + new BoundingBox( + GeospatialBound.createXY(1.0, 2.0), GeospatialBound.createXY(3.0, 4.0))) }; for (Literal lit : literals) { diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java b/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java index e2611ddb281f..8bba39bb314f 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java @@ -25,6 +25,8 @@ import java.util.Arrays; import java.util.List; import java.util.UUID; +import org.apache.iceberg.geospatial.BoundingBox; +import org.apache.iceberg.geospatial.GeospatialBound; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; @@ -48,7 +50,17 @@ public void testIdentityConversions() { Pair.of(Literal.of("abc"), Types.StringType.get()), Pair.of(Literal.of(UUID.randomUUID()), Types.UUIDType.get()), Pair.of(Literal.of(new byte[] {0, 1, 2}), Types.FixedType.ofLength(3)), - Pair.of(Literal.of(ByteBuffer.wrap(new byte[] {0, 1, 2})), Types.BinaryType.get())); + Pair.of(Literal.of(ByteBuffer.wrap(new byte[] {0, 1, 2})), Types.BinaryType.get()), + Pair.of( + Literal.of( + new BoundingBox( + GeospatialBound.createXY(0, 1), GeospatialBound.createXY(2, 3))), + Types.GeometryType.crs84()), + Pair.of( + Literal.of( + new BoundingBox( + GeospatialBound.createXY(0, 1), GeospatialBound.createXY(2, 3))), + Types.GeographyType.crs84())); for (Pair, Type> pair : pairs) { Literal lit = pair.first(); @@ -302,7 +314,9 @@ public void testInvalidStringConversions() { Types.FloatType.get(), Types.DoubleType.get(), Types.FixedType.ofLength(1), - Types.BinaryType.get()); + Types.BinaryType.get(), + Types.GeometryType.crs84(), + Types.GeographyType.crs84()); } @Test @@ -344,7 +358,9 @@ public void testInvalidFixedConversions() { Types.DecimalType.of(9, 2), Types.StringType.get(), Types.UUIDType.get(), - Types.FixedType.ofLength(1)); + Types.FixedType.ofLength(1), + Types.GeometryType.crs84(), + Types.GeographyType.crs84()); } @Test @@ -365,6 +381,29 @@ public void testInvalidBinaryConversions() { Types.DecimalType.of(9, 2), Types.StringType.get(), Types.UUIDType.get(), + Types.FixedType.ofLength(1), + Types.GeometryType.crs84(), + Types.GeographyType.crs84()); + } + + @Test + public void testInvalidGeospatialBoundingBoxConversions() { + GeospatialBound min = GeospatialBound.createXY(1.0, 2.0); + GeospatialBound max = GeospatialBound.createXY(3.0, 4.0); + Literal geoBoundingBoxLiteral = Literal.of(new BoundingBox(min, max)); + testInvalidConversions( + geoBoundingBoxLiteral, + Types.BooleanType.get(), + Types.IntegerType.get(), + Types.LongType.get(), + Types.FloatType.get(), + Types.DoubleType.get(), + Types.DateType.get(), + Types.TimeType.get(), + Types.DecimalType.of(9, 2), + Types.StringType.get(), + Types.UUIDType.get(), + Types.BinaryType.get(), Types.FixedType.ofLength(1)); } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestPredicateBinding.java b/api/src/test/java/org/apache/iceberg/expressions/TestPredicateBinding.java index a07c8fd1569d..960c109c3412 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestPredicateBinding.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestPredicateBinding.java @@ -41,10 +41,13 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.math.BigDecimal; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.geospatial.BoundingBox; +import org.apache.iceberg.geospatial.GeospatialBound; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.StructType; import org.junit.jupiter.api.Test; @@ -648,4 +651,53 @@ public void testNotInPredicateBindingConversionToExpression() { .as("Should change NOT_IN to alwaysTrue expression") .isEqualTo(Expressions.alwaysTrue()); } + + @Test + public void testGeospatialPredicateBinding() { + StructType struct = + StructType.of( + required(20, "geom", Types.GeometryType.crs84()), + required(21, "geog", Types.GeographyType.crs84())); + + // Create a bounding box for testing + GeospatialBound min = GeospatialBound.createXY(1.0, 2.0); + GeospatialBound max = GeospatialBound.createXY(3.0, 4.0); + BoundingBox bbox = new BoundingBox(min, max); + + // Test ST_INTERSECTS with geometry + UnboundPredicate stIntersectsGeom = + Expressions.geospatialPredicate(Expression.Operation.ST_INTERSECTS, "geom", bbox); + Expression expr = stIntersectsGeom.bind(struct); + BoundPredicate bound = assertAndUnwrap(expr); + assertThat(bound.op()).isEqualTo(Expression.Operation.ST_INTERSECTS); + assertThat(bound.term().ref().fieldId()).isEqualTo(20); + assertThat(bound.asLiteralPredicate().literal().value()).isEqualTo(bbox.toByteBuffer()); + + // Test ST_DISJOINT with geometry + UnboundPredicate stDisjointGeom = + Expressions.geospatialPredicate(Expression.Operation.ST_DISJOINT, "geom", bbox); + expr = stDisjointGeom.bind(struct); + bound = assertAndUnwrap(expr); + assertThat(bound.op()).isEqualTo(Expression.Operation.ST_DISJOINT); + assertThat(bound.term().ref().fieldId()).isEqualTo(20); + assertThat(bound.asLiteralPredicate().literal().value()).isEqualTo(bbox.toByteBuffer()); + + // Test ST_INTERSECTS with geography + UnboundPredicate stIntersectsGeog = + Expressions.geospatialPredicate(Expression.Operation.ST_INTERSECTS, "geog", bbox); + expr = stIntersectsGeog.bind(struct); + bound = assertAndUnwrap(expr); + assertThat(bound.op()).isEqualTo(Expression.Operation.ST_INTERSECTS); + assertThat(bound.term().ref().fieldId()).isEqualTo(21); + assertThat(bound.asLiteralPredicate().literal().value()).isEqualTo(bbox.toByteBuffer()); + + // Test ST_DISJOINT with geography + UnboundPredicate stDisjointGeog = + Expressions.geospatialPredicate(Expression.Operation.ST_DISJOINT, "geog", bbox); + expr = stDisjointGeog.bind(struct); + bound = assertAndUnwrap(expr); + assertThat(bound.op()).isEqualTo(Expression.Operation.ST_DISJOINT); + assertThat(bound.term().ref().fieldId()).isEqualTo(21); + assertThat(bound.asLiteralPredicate().literal().value()).isEqualTo(bbox.toByteBuffer()); + } } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java index f34cd730df77..cfbd657ad5c9 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.expressions.Expressions.and; import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.geospatialPredicate; import static org.apache.iceberg.expressions.Expressions.greaterThan; import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; import static org.apache.iceberg.expressions.Expressions.in; @@ -39,16 +40,22 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.util.stream.Stream; import org.apache.iceberg.DataFile; import org.apache.iceberg.Schema; import org.apache.iceberg.TestHelpers.Row; import org.apache.iceberg.TestHelpers.TestDataFile; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.geospatial.BoundingBox; +import org.apache.iceberg.geospatial.GeospatialBound; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.IntegerType; import org.apache.iceberg.types.Types.StringType; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; public class TestStrictMetricsEvaluator { private static final Schema SCHEMA = @@ -73,7 +80,9 @@ public class TestStrictMetricsEvaluator { Types.StructType.of( Types.NestedField.optional(16, "nested_col_no_stats", Types.IntegerType.get()), Types.NestedField.optional( - 17, "nested_col_with_stats", Types.IntegerType.get())))); + 17, "nested_col_with_stats", Types.IntegerType.get()))), + optional(18, "geom", Types.GeometryType.crs84()), + optional(19, "geog", Types.GeographyType.crs84())); private static final int INT_MIN_VALUE = 30; private static final int INT_MAX_VALUE = 79; @@ -172,6 +181,32 @@ public class TestStrictMetricsEvaluator { // upper bounds ImmutableMap.of(5, toByteBuffer(StringType.get(), "bbb"))); + private static final DataFile FILE_4 = + new TestDataFile( + "file_4.avro", + Row.of(), + 50, + // any value counts, including nulls + ImmutableMap.of( + 1, 50L, + 18, 50L, + 19, 50L), + // null value counts + ImmutableMap.of( + 1, 0L, + 18, 0L, + 19, 0L), + // nan value counts + null, + // lower bounds + ImmutableMap.of( + 18, GeospatialBound.createXY(1, 2).toByteBuffer(), + 19, GeospatialBound.createXY(1, 2).toByteBuffer()), + // upper bounds + ImmutableMap.of( + 18, GeospatialBound.createXY(10, 20).toByteBuffer(), + 19, GeospatialBound.createXY(10, 20).toByteBuffer())); + @Test public void testAllNulls() { boolean shouldRead = new StrictMetricsEvaluator(SCHEMA, notNull("all_nulls")).eval(FILE); @@ -684,4 +719,27 @@ SCHEMA, lessThanOrEqual("struct.nested_col_with_stats", INT_MAX_VALUE)) new StrictMetricsEvaluator(SCHEMA, notNull("struct.nested_col_with_stats")).eval(FILE); assertThat(shouldRead).as("notNull nested column should not match").isFalse(); } + + private static Stream geospatialPredicateParameters() { + return Stream.of( + Arguments.of(Expression.Operation.ST_INTERSECTS, "geom"), + Arguments.of(Expression.Operation.ST_INTERSECTS, "geog"), + Arguments.of(Expression.Operation.ST_DISJOINT, "geom"), + Arguments.of(Expression.Operation.ST_DISJOINT, "geog")); + } + + @ParameterizedTest + @MethodSource("geospatialPredicateParameters") + public void testGeospatialPredicates(Expression.Operation operation, String columnName) { + boolean shouldRead = + new StrictMetricsEvaluator( + SCHEMA, + geospatialPredicate( + operation, + columnName, + new BoundingBox( + GeospatialBound.createXY(1, 2), GeospatialBound.createXY(2, 3)))) + .eval(FILE_4); + assertThat(shouldRead).as("Geospatial predicate should never match").isFalse(); + } } diff --git a/api/src/test/java/org/apache/iceberg/geospatial/TestBoundingBox.java b/api/src/test/java/org/apache/iceberg/geospatial/TestBoundingBox.java new file mode 100644 index 000000000000..e54849844d45 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/geospatial/TestBoundingBox.java @@ -0,0 +1,140 @@ +/* + * 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.geospatial; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import org.junit.jupiter.api.Test; + +public class TestBoundingBox { + + @Test + public void testConstructorAndAccessors() { + GeospatialBound min = GeospatialBound.createXY(1.0, 2.0); + GeospatialBound max = GeospatialBound.createXY(3.0, 4.0); + + BoundingBox box = new BoundingBox(min, max); + + assertThat(box.min()).isEqualTo(min); + assertThat(box.max()).isEqualTo(max); + assertThat(box.min().x()).isEqualTo(1.0); + assertThat(box.min().y()).isEqualTo(2.0); + assertThat(box.max().x()).isEqualTo(3.0); + assertThat(box.max().y()).isEqualTo(4.0); + } + + @Test + public void testCreateFromByteBuffers() { + // Create byte buffers for XY bounds + ByteBuffer minBuffer = ByteBuffer.allocate(16); + minBuffer.order(ByteOrder.LITTLE_ENDIAN); + minBuffer.putDouble(0, 1.0); // x + minBuffer.putDouble(8, 2.0); // y + + ByteBuffer maxBuffer = ByteBuffer.allocate(16); + maxBuffer.order(ByteOrder.LITTLE_ENDIAN); + maxBuffer.putDouble(0, 3.0); // x + maxBuffer.putDouble(8, 4.0); // y + + BoundingBox box = BoundingBox.fromByteBuffers(minBuffer, maxBuffer); + + assertThat(box.min().x()).isEqualTo(1.0); + assertThat(box.min().y()).isEqualTo(2.0); + assertThat(box.max().x()).isEqualTo(3.0); + assertThat(box.max().y()).isEqualTo(4.0); + assertThat(minBuffer.order()).isEqualTo(ByteOrder.LITTLE_ENDIAN); + assertThat(maxBuffer.order()).isEqualTo(ByteOrder.LITTLE_ENDIAN); + } + + @Test + public void testCreateFromBigEndianByteBuffers() { + // Create byte buffers for XY bounds + ByteBuffer minBuffer = ByteBuffer.allocate(16); + minBuffer.order(ByteOrder.LITTLE_ENDIAN); + minBuffer.putDouble(0, 10.0); // x + minBuffer.putDouble(8, 20.0); // y + minBuffer.order(ByteOrder.BIG_ENDIAN); + + ByteBuffer maxBuffer = ByteBuffer.allocate(16); + maxBuffer.order(ByteOrder.LITTLE_ENDIAN); + maxBuffer.putDouble(0, 30.0); // x + maxBuffer.putDouble(8, 40.0); // y + maxBuffer.order(ByteOrder.BIG_ENDIAN); + + BoundingBox box = BoundingBox.fromByteBuffers(minBuffer, maxBuffer); + + assertThat(box.min().x()).isEqualTo(10.0); + assertThat(box.min().y()).isEqualTo(20.0); + assertThat(box.max().x()).isEqualTo(30.0); + assertThat(box.max().y()).isEqualTo(40.0); + assertThat(minBuffer.order()).isEqualTo(ByteOrder.BIG_ENDIAN); + assertThat(maxBuffer.order()).isEqualTo(ByteOrder.BIG_ENDIAN); + } + + @Test + public void testEqualsAndHashCode() { + GeospatialBound min1 = GeospatialBound.createXY(1.0, 2.0); + GeospatialBound max1 = GeospatialBound.createXY(3.0, 4.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + // Same values + GeospatialBound min2 = GeospatialBound.createXY(1.0, 2.0); + GeospatialBound max2 = GeospatialBound.createXY(3.0, 4.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + // Different values + GeospatialBound min3 = GeospatialBound.createXY(0.0, 0.0); + GeospatialBound max3 = GeospatialBound.createXY(10.0, 10.0); + BoundingBox box3 = new BoundingBox(min3, max3); + + // Test equals + assertThat(box1).isEqualTo(box2); + assertThat(box1).isNotEqualTo(box3); + assertThat(box1).isNotEqualTo(null); + assertThat(box1).isNotEqualTo("not a box"); + + // Test hashCode + assertThat(box1.hashCode()).isEqualTo(box2.hashCode()); + assertThat(box1.hashCode()).isNotEqualTo(box3.hashCode()); + } + + @Test + public void testToString() { + GeospatialBound min = GeospatialBound.createXY(1.0, 2.0); + GeospatialBound max = GeospatialBound.createXY(3.0, 4.0); + BoundingBox box = new BoundingBox(min, max); + assertThat(box.toString()).isEqualTo("BoundingBox{min={x=1.0, y=2.0}, max={x=3.0, y=4.0}}"); + } + + @Test + public void testRoundTripSerDe() { + GeospatialBound min = GeospatialBound.createXY(1.0, 2.0); + GeospatialBound max = GeospatialBound.createXY(3.0, 4.0); + BoundingBox original = new BoundingBox(min, max); + BoundingBox deserialized = roundTripSerDe(original); + assertThat(deserialized).isEqualTo(original); + } + + private BoundingBox roundTripSerDe(BoundingBox original) { + ByteBuffer buffer = original.toByteBuffer(); + return BoundingBox.fromByteBuffer(buffer); + } +} diff --git a/api/src/test/java/org/apache/iceberg/geospatial/TestGeospatialBound.java b/api/src/test/java/org/apache/iceberg/geospatial/TestGeospatialBound.java new file mode 100644 index 000000000000..f4b3a7deebfa --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/geospatial/TestGeospatialBound.java @@ -0,0 +1,228 @@ +/* + * 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.geospatial; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.ByteBuffer; +import org.apache.iceberg.util.ByteBuffers; +import org.junit.jupiter.api.Test; + +public class TestGeospatialBound { + + @Test + public void testCreateXY() { + GeospatialBound bound = GeospatialBound.createXY(1.0, 2.0); + assertThat(bound.x()).isEqualTo(1.0); + assertThat(bound.y()).isEqualTo(2.0); + assertThat(bound.hasZ()).isFalse(); + assertThat(bound.hasM()).isFalse(); + assertThat(Double.isNaN(bound.z())).isTrue(); + assertThat(Double.isNaN(bound.m())).isTrue(); + } + + @Test + public void testCreateXYZ() { + GeospatialBound bound = GeospatialBound.createXYZ(1.0, 2.0, 3.0); + assertThat(bound.x()).isEqualTo(1.0); + assertThat(bound.y()).isEqualTo(2.0); + assertThat(bound.z()).isEqualTo(3.0); + assertThat(bound.hasZ()).isTrue(); + assertThat(bound.hasM()).isFalse(); + assertThat(Double.isNaN(bound.m())).isTrue(); + } + + @Test + public void testCreateXYM() { + GeospatialBound bound = GeospatialBound.createXYM(1.0, 2.0, 4.0); + assertThat(bound.x()).isEqualTo(1.0); + assertThat(bound.y()).isEqualTo(2.0); + assertThat(bound.m()).isEqualTo(4.0); + assertThat(bound.hasZ()).isFalse(); + assertThat(bound.hasM()).isTrue(); + assertThat(Double.isNaN(bound.z())).isTrue(); + } + + @Test + public void testCreateXYZM() { + GeospatialBound bound = GeospatialBound.createXYZM(1.0, 2.0, 3.0, 4.0); + assertThat(bound.x()).isEqualTo(1.0); + assertThat(bound.y()).isEqualTo(2.0); + assertThat(bound.z()).isEqualTo(3.0); + assertThat(bound.m()).isEqualTo(4.0); + assertThat(bound.hasZ()).isTrue(); + assertThat(bound.hasM()).isTrue(); + } + + @Test + public void testEqualsAndHashCode() { + GeospatialBound xy1 = GeospatialBound.createXY(1.0, 2.0); + GeospatialBound xy2 = GeospatialBound.createXY(1.0, 2.0); + GeospatialBound xy3 = GeospatialBound.createXY(2.0, 1.0); + assertThat(xy1).isEqualTo(xy2); + assertThat(xy1.hashCode()).isEqualTo(xy2.hashCode()); + assertThat(xy1).isNotEqualTo(xy3); + + GeospatialBound xyz1 = GeospatialBound.createXYZ(1.0, 2.0, 3.0); + GeospatialBound xyz2 = GeospatialBound.createXYZ(1.0, 2.0, 3.0); + GeospatialBound xyz3 = GeospatialBound.createXYZ(1.0, 2.0, 4.0); + assertThat(xyz1).isEqualTo(xyz2); + assertThat(xyz1.hashCode()).isEqualTo(xyz2.hashCode()); + assertThat(xyz1).isNotEqualTo(xyz3); + assertThat(xyz1).isNotEqualTo(xy1); + + GeospatialBound xym1 = GeospatialBound.createXYM(1.0, 2.0, 4.0); + GeospatialBound xym2 = GeospatialBound.createXYM(1.0, 2.0, 4.0); + GeospatialBound xym3 = GeospatialBound.createXYM(1.0, 2.0, 5.0); + assertThat(xym1).isEqualTo(xym2); + assertThat(xym1.hashCode()).isEqualTo(xym2.hashCode()); + assertThat(xym1).isNotEqualTo(xym3); + assertThat(xym1).isNotEqualTo(xy1); + + GeospatialBound xyzm1 = GeospatialBound.createXYZM(1.0, 2.0, 3.0, 4.0); + GeospatialBound xyzm2 = GeospatialBound.createXYZM(1.0, 2.0, 3.0, 4.0); + GeospatialBound xyzm3 = GeospatialBound.createXYZM(1.0, 2.0, 3.0, 5.0); + assertThat(xyzm1).isEqualTo(xyzm2); + assertThat(xyzm1.hashCode()).isEqualTo(xyzm2.hashCode()); + assertThat(xyzm1).isNotEqualTo(xyzm3); + assertThat(xyzm1).isNotEqualTo(xyz1); + } + + @Test + public void testToString() { + GeospatialBound xy = GeospatialBound.createXY(1.0, 2.0); + assertThat(xy.toString()).isEqualTo("GeospatialBound(x=1.0, y=2.0)"); + + GeospatialBound xyz = GeospatialBound.createXYZ(1.0, 2.0, 3.0); + assertThat(xyz.toString()).isEqualTo("GeospatialBound(x=1.0, y=2.0, z=3.0)"); + + GeospatialBound xym = GeospatialBound.createXYM(1.0, 2.0, 4.0); + assertThat(xym.toString()).isEqualTo("GeospatialBound(x=1.0, y=2.0, m=4.0)"); + + GeospatialBound xyzm = GeospatialBound.createXYZM(1.0, 2.0, 3.0, 4.0); + assertThat(xyzm.toString()).isEqualTo("GeospatialBound(x=1.0, y=2.0, z=3.0, m=4.0)"); + } + + @Test + public void testSimpleString() { + GeospatialBound xy = GeospatialBound.createXY(1.0, 2.0); + assertThat(xy.simpleString()).isEqualTo("x=1.0, y=2.0"); + + GeospatialBound xyz = GeospatialBound.createXYZ(1.0, 2.0, 3.0); + assertThat(xyz.simpleString()).isEqualTo("x=1.0, y=2.0, z=3.0"); + + GeospatialBound xym = GeospatialBound.createXYM(1.0, 2.0, 4.0); + assertThat(xym.simpleString()).isEqualTo("x=1.0, y=2.0, m=4.0"); + + GeospatialBound xyzm = GeospatialBound.createXYZM(1.0, 2.0, 3.0, 4.0); + assertThat(xyzm.simpleString()).isEqualTo("x=1.0, y=2.0, z=3.0, m=4.0"); + } + + @Test + public void testSerde() { + // Test XY format (16 bytes: x:y) + // These bytes represent x=10.0, y=13.0 + byte[] xyBytes = + new byte[] { + 0, 0, 0, 0, 0, 0, 36, 64, // 10.0 in little-endian IEEE 754 + 0, 0, 0, 0, 0, 0, 42, 64 // 13.0 in little-endian IEEE 754 + }; + GeospatialBound xy = GeospatialBound.fromByteArray(xyBytes); + assertThat(xy.x()).isEqualTo(10.0); + assertThat(xy.y()).isEqualTo(13.0); + assertThat(xy.hasZ()).isFalse(); + assertThat(xy.hasM()).isFalse(); + assertThat(ByteBuffers.toByteArray(xy.toByteBuffer())).isEqualTo(xyBytes); + + // Test XYZ format (24 bytes: x:y:z) + // These bytes represent x=10.0, y=13.0, z=15.0 + byte[] xyzBytes = + new byte[] { + 0, 0, 0, 0, 0, 0, 36, 64, // 10.0 in little-endian IEEE 754 + 0, 0, 0, 0, 0, 0, 42, 64, // 13.0 in little-endian IEEE 754 + 0, 0, 0, 0, 0, 0, 46, 64 // 15.0 in little-endian IEEE 754 + }; + GeospatialBound xyz = GeospatialBound.fromByteArray(xyzBytes); + assertThat(xyz.x()).isEqualTo(10.0); + assertThat(xyz.y()).isEqualTo(13.0); + assertThat(xyz.z()).isEqualTo(15.0); + assertThat(xyz.hasZ()).isTrue(); + assertThat(xyz.hasM()).isFalse(); + assertThat(ByteBuffers.toByteArray(xyz.toByteBuffer())).isEqualTo(xyzBytes); + // Test XYM format (32 bytes: x:y:NaN:m) + // These bytes represent x=10.0, y=13.0, z=NaN, m=20.0 + byte[] xymBytes = + new byte[] { + 0, 0, 0, 0, 0, 0, 36, 64, // 10.0 in little-endian IEEE 754 + 0, 0, 0, 0, 0, 0, 42, 64, // 13.0 in little-endian IEEE 754 + 0, 0, 0, 0, 0, 0, (byte) 248, 127, // NaN in little-endian IEEE 754 + 0, 0, 0, 0, 0, 0, 52, 64 // 20.0 in little-endian IEEE 754 + }; + GeospatialBound xym = GeospatialBound.fromByteArray(xymBytes); + assertThat(xym.x()).isEqualTo(10.0); + assertThat(xym.y()).isEqualTo(13.0); + assertThat(Double.isNaN(xym.z())).isTrue(); + assertThat(xym.m()).isEqualTo(20.0); + assertThat(xym.hasZ()).isFalse(); + assertThat(xym.hasM()).isTrue(); + assertThat(ByteBuffers.toByteArray(xym.toByteBuffer())).isEqualTo(xymBytes); + + // Test XYZM format (32 bytes: x:y:z:m) + // These bytes represent x=10.0, y=13.0, z=15.0, m=20.0 + byte[] xyzmBytes = + new byte[] { + 0, 0, 0, 0, 0, 0, 36, 64, // 10.0 in little-endian IEEE 754 + 0, 0, 0, 0, 0, 0, 42, 64, // 13.0 in little-endian IEEE 754 + 0, 0, 0, 0, 0, 0, 46, 64, // 15.0 in little-endian IEEE 754 + 0, 0, 0, 0, 0, 0, 52, 64 // 20.0 in little-endian IEEE 754 + }; + GeospatialBound xyzm = GeospatialBound.fromByteArray(xyzmBytes); + assertThat(xyzm.x()).isEqualTo(10.0); + assertThat(xyzm.y()).isEqualTo(13.0); + assertThat(xyzm.z()).isEqualTo(15.0); + assertThat(xyzm.m()).isEqualTo(20.0); + assertThat(xyzm.hasZ()).isTrue(); + assertThat(xyzm.hasM()).isTrue(); + assertThat(ByteBuffers.toByteArray(xyzm.toByteBuffer())).isEqualTo(xyzmBytes); + } + + private GeospatialBound roundTripSerDe(GeospatialBound original) { + ByteBuffer buffer = original.toByteBuffer(); + return GeospatialBound.fromByteBuffer(buffer); + } + + @Test + public void testRoundTripSerDe() { + // Test XY serialization + GeospatialBound xy = GeospatialBound.createXY(1.1, 2.2); + assertThat(roundTripSerDe(xy)).isEqualTo(xy); + + // Test XYZ serialization + GeospatialBound xyz = GeospatialBound.createXYZ(1.1, 2.2, 3.3); + assertThat(roundTripSerDe(xyz)).isEqualTo(xyz); + + // Test XYM serialization + GeospatialBound xym = GeospatialBound.createXYM(1.1, 2.2, 4.4); + assertThat(roundTripSerDe(xym)).isEqualTo(xym); + + // Test XYZM serialization + GeospatialBound xyzm = GeospatialBound.createXYZM(1.1, 2.2, 3.3, 4.4); + assertThat(roundTripSerDe(xyzm)).isEqualTo(xyzm); + } +} diff --git a/api/src/test/java/org/apache/iceberg/geospatial/TestGeospatialPredicateEvaluators.java b/api/src/test/java/org/apache/iceberg/geospatial/TestGeospatialPredicateEvaluators.java new file mode 100644 index 000000000000..e3c2abbd1f86 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/geospatial/TestGeospatialPredicateEvaluators.java @@ -0,0 +1,430 @@ +/* + * 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.geospatial; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.types.EdgeAlgorithm; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestGeospatialPredicateEvaluators { + + @Test + public void testGeometryType() { + Type geometryType = Types.GeometryType.crs84(); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geometryType); + + assertThat(evaluator).isInstanceOf(GeospatialPredicateEvaluators.GeometryEvaluator.class); + } + + @Test + public void testOverlappingBoxesIntersect() { + Type geometryType = Types.GeometryType.crs84(); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geometryType); + + GeospatialBound min1 = GeospatialBound.createXY(0.0, 0.0); + GeospatialBound max1 = GeospatialBound.createXY(5.0, 5.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + GeospatialBound min2 = GeospatialBound.createXY(3.0, 3.0); + GeospatialBound max2 = GeospatialBound.createXY(8.0, 8.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + assertThat(evaluator.intersects(box1, box2)).isTrue(); + assertThat(evaluator.intersects(box2, box1)).isTrue(); + } + + @Test + public void testNonOverlappingBoxesDontIntersect() { + Type geometryType = Types.GeometryType.crs84(); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geometryType); + + GeospatialBound min1 = GeospatialBound.createXY(0.0, 0.0); + GeospatialBound max1 = GeospatialBound.createXY(2.0, 2.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + GeospatialBound min2 = GeospatialBound.createXY(3.0, 3.0); + GeospatialBound max2 = GeospatialBound.createXY(5.0, 5.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + assertThat(evaluator.intersects(box1, box2)).isFalse(); + assertThat(evaluator.intersects(box2, box1)).isFalse(); + } + + @Test + public void testBoxesTouchingAtCornerIntersect() { + Type geometryType = Types.GeometryType.crs84(); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geometryType); + + GeospatialBound min1 = GeospatialBound.createXY(0.0, 0.0); + GeospatialBound max1 = GeospatialBound.createXY(2.0, 2.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + GeospatialBound min2 = GeospatialBound.createXY(2.0, 2.0); + GeospatialBound max2 = GeospatialBound.createXY(4.0, 4.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + assertThat(evaluator.intersects(box1, box2)).isTrue(); + assertThat(evaluator.intersects(box2, box1)).isTrue(); + } + + @Test + public void testBoxesTouchingAtEdgeIntersect() { + Type geometryType = Types.GeometryType.crs84(); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geometryType); + + GeospatialBound min1 = GeospatialBound.createXY(0.0, 0.0); + GeospatialBound max1 = GeospatialBound.createXY(2.0, 2.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + GeospatialBound min2 = GeospatialBound.createXY(2.0, 0.0); + GeospatialBound max2 = GeospatialBound.createXY(4.0, 2.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + assertThat(evaluator.intersects(box1, box2)).isTrue(); + assertThat(evaluator.intersects(box2, box1)).isTrue(); + } + + @Test + public void testBoxContainedWithinAnotherIntersects() { + Type geometryType = Types.GeometryType.crs84(); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geometryType); + + GeospatialBound min1 = GeospatialBound.createXY(0.0, 0.0); + GeospatialBound max1 = GeospatialBound.createXY(10.0, 10.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + GeospatialBound min2 = GeospatialBound.createXY(2.0, 2.0); + GeospatialBound max2 = GeospatialBound.createXY(5.0, 5.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + assertThat(evaluator.intersects(box1, box2)).isTrue(); + assertThat(evaluator.intersects(box2, box1)).isTrue(); + } + + @Test + public void testBoxesWithZCoordinate() { + Type geometryType = Types.GeometryType.crs84(); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geometryType); + + // Two boxes with Z coordinates that overlap in X and Y but not in Z + // Note: The current implementation only checks X and Y coordinates + GeospatialBound min1 = GeospatialBound.createXYZ(0.0, 0.0, 0.0); + GeospatialBound max1 = GeospatialBound.createXYZ(2.0, 2.0, 1.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + GeospatialBound min2 = GeospatialBound.createXYZ(1.0, 1.0, 2.0); + GeospatialBound max2 = GeospatialBound.createXYZ(3.0, 3.0, 3.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + // They should intersect because the current implementation only checks X and Y + assertThat(evaluator.intersects(box1, box2)).isTrue(); + assertThat(evaluator.intersects(box2, box1)).isTrue(); + } + + @Test + public void testBoxesWithMCoordinate() { + Type geometryType = Types.GeometryType.crs84(); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geometryType); + + // Two boxes with M coordinates that overlap in X and Y but not in M + // Note: The current implementation only checks X and Y coordinates + GeospatialBound min1 = GeospatialBound.createXYM(0.0, 0.0, 0.0); + GeospatialBound max1 = GeospatialBound.createXYM(2.0, 2.0, 1.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + GeospatialBound min2 = GeospatialBound.createXYM(1.0, 1.0, 2.0); + GeospatialBound max2 = GeospatialBound.createXYM(3.0, 3.0, 3.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + // They should intersect because the current implementation only checks X and Y + assertThat(evaluator.intersects(box1, box2)).isTrue(); + assertThat(evaluator.intersects(box2, box1)).isTrue(); + } + + @Test + public void testGeometryWrapAroundOnA() { + Type geometryType = Types.GeometryType.crs84(); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geometryType); + + // First box wraps around antimeridian (min.x > max.x), second doesn't + GeospatialBound min1 = GeospatialBound.createXY(170.0, 0.0); + GeospatialBound max1 = GeospatialBound.createXY(-170.0, 10.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + // Box that overlaps with the part after the wrap around + GeospatialBound min2 = GeospatialBound.createXY(-175.0, 5.0); + GeospatialBound max2 = GeospatialBound.createXY(-160.0, 15.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + assertThat(evaluator.intersects(box1, box2)).isTrue(); + assertThat(evaluator.intersects(box2, box1)).isTrue(); + + // Box that overlaps with the part before the wrap around + GeospatialBound min3 = GeospatialBound.createXY(160.0, 5.0); + GeospatialBound max3 = GeospatialBound.createXY(175.0, 15.0); + BoundingBox box3 = new BoundingBox(min3, max3); + + assertThat(evaluator.intersects(box1, box3)).isTrue(); + assertThat(evaluator.intersects(box3, box1)).isTrue(); + + // Box that doesn't overlap with either part + GeospatialBound min4 = GeospatialBound.createXY(-150.0, 20.0); + GeospatialBound max4 = GeospatialBound.createXY(-140.0, 30.0); + BoundingBox box4 = new BoundingBox(min4, max4); + + assertThat(evaluator.intersects(box1, box4)).isFalse(); + assertThat(evaluator.intersects(box4, box1)).isFalse(); + } + + @Test + public void testGeometryWrapAroundOnB() { + Type geometryType = Types.GeometryType.crs84(); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geometryType); + + // First box doesn't wrap around, second does (min.x > max.x) + GeospatialBound min1 = GeospatialBound.createXY(-175.0, 5.0); + GeospatialBound max1 = GeospatialBound.createXY(-160.0, 15.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + GeospatialBound min2 = GeospatialBound.createXY(170.0, 0.0); + GeospatialBound max2 = GeospatialBound.createXY(-170.0, 10.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + assertThat(evaluator.intersects(box1, box2)).isTrue(); + assertThat(evaluator.intersects(box2, box1)).isTrue(); + } + + @Test + public void testBothGeometryWrappingAround() { + Type geometryType = Types.GeometryType.crs84(); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geometryType); + + // Both boxes wrap around (min.x > max.x) + GeospatialBound min1 = GeospatialBound.createXY(170.0, 0.0); + GeospatialBound max1 = GeospatialBound.createXY(-170.0, 10.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + GeospatialBound min2 = GeospatialBound.createXY(160.0, 5.0); + GeospatialBound max2 = GeospatialBound.createXY(-160.0, 15.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + // When both wrap around, they must intersect + assertThat(evaluator.intersects(box1, box2)).isTrue(); + assertThat(evaluator.intersects(box2, box1)).isTrue(); + } + + @Test + public void testBasicGeographyCases() { + Type geographyType = Types.GeographyType.of("srid:4326", EdgeAlgorithm.SPHERICAL); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geographyType); + + // Two overlapping boxes + GeospatialBound min1 = GeospatialBound.createXY(0.0, 0.0); + GeospatialBound max1 = GeospatialBound.createXY(10.0, 10.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + GeospatialBound min2 = GeospatialBound.createXY(5.0, 5.0); + GeospatialBound max2 = GeospatialBound.createXY(15.0, 15.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + assertThat(evaluator.intersects(box1, box2)).isTrue(); + assertThat(evaluator.intersects(box2, box1)).isTrue(); + + // Non-overlapping boxes + GeospatialBound min3 = GeospatialBound.createXY(20.0, 20.0); + GeospatialBound max3 = GeospatialBound.createXY(30.0, 30.0); + BoundingBox box3 = new BoundingBox(min3, max3); + + assertThat(evaluator.intersects(box1, box3)).isFalse(); + assertThat(evaluator.intersects(box3, box1)).isFalse(); + + // Boxes at extreme valid latitudes + GeospatialBound min4 = GeospatialBound.createXY(-10.0, -90.0); + GeospatialBound max4 = GeospatialBound.createXY(10.0, -80.0); + BoundingBox box4 = new BoundingBox(min4, max4); + + GeospatialBound min5 = GeospatialBound.createXY(-5.0, 80.0); + GeospatialBound max5 = GeospatialBound.createXY(15.0, 90.0); + BoundingBox box5 = new BoundingBox(min5, max5); + + assertThat(evaluator.intersects(box4, box5)).isFalse(); + assertThat(evaluator.intersects(box5, box4)).isFalse(); + } + + @Test + public void testGeographyWrapAround() { + Type geographyType = Types.GeographyType.of("srid:4326", EdgeAlgorithm.SPHERICAL); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geographyType); + + // Box that wraps around the antimeridian + GeospatialBound min1 = GeospatialBound.createXY(170.0, 0.0); + GeospatialBound max1 = GeospatialBound.createXY(-170.0, 10.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + // Box that overlaps with the part after the wrap around + GeospatialBound min2 = GeospatialBound.createXY(-175.0, 5.0); + GeospatialBound max2 = GeospatialBound.createXY(-160.0, 15.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + assertThat(evaluator.intersects(box1, box2)).isTrue(); + assertThat(evaluator.intersects(box2, box1)).isTrue(); + } + + @Test + public void testInvalidGeographyLatitude() { + Type geographyType = Types.GeographyType.of("srid:4326", EdgeAlgorithm.SPHERICAL); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geographyType); + + // Box with latitude below -90 + GeospatialBound min1 = GeospatialBound.createXY(0.0, -91.0); + GeospatialBound max1 = GeospatialBound.createXY(10.0, 0.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + // Box with latitude above 90 + GeospatialBound min2 = GeospatialBound.createXY(0.0, 0.0); + GeospatialBound max2 = GeospatialBound.createXY(10.0, 91.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + GeospatialBound validMin = GeospatialBound.createXY(0.0, 0.0); + GeospatialBound validMax = GeospatialBound.createXY(10.0, 10.0); + BoundingBox validBox = new BoundingBox(validMin, validMax); + + assertThatThrownBy(() -> evaluator.intersects(box1, validBox)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Latitude out of range"); + + assertThatThrownBy(() -> evaluator.intersects(validBox, box1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Latitude out of range"); + + assertThatThrownBy(() -> evaluator.intersects(box2, validBox)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Latitude out of range"); + + assertThatThrownBy(() -> evaluator.intersects(validBox, box2)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Latitude out of range"); + } + + @Test + public void testInvalidGeographyLongitude() { + Type geographyType = Types.GeographyType.of("srid:4326", EdgeAlgorithm.SPHERICAL); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geographyType); + + // Box with longitude below -180 + GeospatialBound min1 = GeospatialBound.createXY(-181.0, 0.0); + GeospatialBound max1 = GeospatialBound.createXY(0.0, 10.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + // Box with longitude above 180 + GeospatialBound min2 = GeospatialBound.createXY(0.0, 0.0); + GeospatialBound max2 = GeospatialBound.createXY(181.0, 10.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + GeospatialBound validMin = GeospatialBound.createXY(0.0, 0.0); + GeospatialBound validMax = GeospatialBound.createXY(10.0, 10.0); + BoundingBox validBox = new BoundingBox(validMin, validMax); + + assertThatThrownBy(() -> evaluator.intersects(box1, validBox)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Longitude out of range"); + + assertThatThrownBy(() -> evaluator.intersects(validBox, box1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Longitude out of range"); + + assertThatThrownBy(() -> evaluator.intersects(box2, validBox)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Longitude out of range"); + + assertThatThrownBy(() -> evaluator.intersects(validBox, box2)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Longitude out of range"); + } + + @Test + public void testExtremeLongitudeBoundaries() { + // Tests valid boxes at the extreme boundaries of longitude + Type geographyType = Types.GeographyType.of("srid:4326", EdgeAlgorithm.SPHERICAL); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geographyType); + + // Box at -180 longitude + GeospatialBound min1 = GeospatialBound.createXY(-180.0, 0.0); + GeospatialBound max1 = GeospatialBound.createXY(-170.0, 10.0); + BoundingBox box1 = new BoundingBox(min1, max1); + + // Box at 180 longitude + GeospatialBound min2 = GeospatialBound.createXY(170.0, 0.0); + GeospatialBound max2 = GeospatialBound.createXY(180.0, 10.0); + BoundingBox box2 = new BoundingBox(min2, max2); + + // These boxes should not intersect + assertThat(evaluator.intersects(box1, box2)).isFalse(); + assertThat(evaluator.intersects(box2, box1)).isFalse(); + + // Box that wraps around the antimeridian, touching -180 and 180 + GeospatialBound min3 = GeospatialBound.createXY(180.0, 0.0); + GeospatialBound max3 = GeospatialBound.createXY(-180.0, 10.0); + BoundingBox box3 = new BoundingBox(min3, max3); + + // This should intersect with both boxes at the extreme edges + assertThat(evaluator.intersects(box1, box3)).isTrue(); + assertThat(evaluator.intersects(box3, box1)).isTrue(); + assertThat(evaluator.intersects(box2, box3)).isTrue(); + assertThat(evaluator.intersects(box3, box2)).isTrue(); + } + + @Test + public void testSphericalGeographyType() { + Type geographyType = Types.GeographyType.of("srid:4326", EdgeAlgorithm.SPHERICAL); + GeospatialPredicateEvaluators.GeospatialPredicateEvaluator evaluator = + GeospatialPredicateEvaluators.create(geographyType); + + assertThat(evaluator).isInstanceOf(GeospatialPredicateEvaluators.GeographyEvaluator.class); + } + + @Test + public void testUnsupportedType() { + Type stringType = Types.StringType.get(); + + assertThatThrownBy(() -> GeospatialPredicateEvaluators.create(stringType)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Unsupported type for BoundingBox"); + } +} diff --git a/api/src/test/java/org/apache/iceberg/types/TestConversions.java b/api/src/test/java/org/apache/iceberg/types/TestConversions.java index e207cfd8d59a..3e1edd031005 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestConversions.java +++ b/api/src/test/java/org/apache/iceberg/types/TestConversions.java @@ -45,6 +45,7 @@ public class TestConversions { @Test + @SuppressWarnings("MethodLength") public void testByteBufferConversions() { // booleans are stored as 0x00 for 'false' and a non-zero byte for 'true' assertConversion(false, BooleanType.get(), new byte[] {0x00}); @@ -189,6 +190,13 @@ public void testByteBufferConversions() { assertConversion(new BigDecimal("0.011"), DecimalType.of(10, 3), new byte[] {11}); assertThat(Literal.of(new BigDecimal("0.011")).toByteBuffer().array()) .isEqualTo(new byte[] {11}); + + // geospatial bounds were kept as-is + // this is a geospatial bound with x = 10.0, y = 20.0 + byte[] geospatialBound = new byte[] {0, 0, 0, 0, 0, 0, 36, 64, 0, 0, 0, 0, 0, 0, 52, 64}; + assertConversion(ByteBuffer.wrap(geospatialBound), Types.GeometryType.crs84(), geospatialBound); + assertConversion( + ByteBuffer.wrap(geospatialBound), Types.GeographyType.crs84(), geospatialBound); } private void assertConversion(T value, Type type, byte[] expectedBinary) { diff --git a/api/src/test/java/org/apache/iceberg/types/TestTypes.java b/api/src/test/java/org/apache/iceberg/types/TestTypes.java index cc8d3586b862..fa5ed4304d3c 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestTypes.java @@ -98,6 +98,8 @@ public void fromPrimitiveString() { assertThat(Types.fromPrimitiveString("geometry")).isEqualTo(Types.GeometryType.crs84()); assertThat(Types.fromPrimitiveString("Geometry")).isEqualTo(Types.GeometryType.crs84()); + assertThat(((Types.GeometryType) Types.fromPrimitiveString("geometry")).crs()) + .isEqualTo(Types.GeometryType.DEFAULT_CRS); assertThat(Types.fromPrimitiveString("geometry(srid:3857)")) .isEqualTo(Types.GeometryType.of("srid:3857")); assertThat(Types.fromPrimitiveString("geometry( srid:3857 )")) @@ -113,12 +115,13 @@ public void fromPrimitiveString() { assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> Types.fromPrimitiveString("geometry( )")) .withMessageContaining("Invalid CRS: (empty string)"); - assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> Types.fromPrimitiveString("geometry(srid:123,456)")) - .withMessageContaining("Invalid CRS: srid:123,456"); assertThat(Types.fromPrimitiveString("geography")).isEqualTo(Types.GeographyType.crs84()); assertThat(Types.fromPrimitiveString("Geography")).isEqualTo(Types.GeographyType.crs84()); + assertThat(((Types.GeographyType) Types.fromPrimitiveString("geography")).crs()) + .isEqualTo(Types.GeographyType.DEFAULT_CRS); + assertThat(((Types.GeographyType) Types.fromPrimitiveString("geography")).algorithm()) + .isEqualTo(Types.GeographyType.DEFAULT_ALGORITHM); assertThat(Types.fromPrimitiveString("geography(srid:4269)")) .isEqualTo(Types.GeographyType.of("srid:4269")); assertThat(Types.fromPrimitiveString("geography(srid: 4269)")) diff --git a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java index 9bb5b7d05f0b..52ddb95f7559 100644 --- a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java +++ b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java @@ -31,6 +31,9 @@ import java.util.function.Supplier; import org.apache.iceberg.Schema; import org.apache.iceberg.SingleValueParser; +import org.apache.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.geospatial.BoundingBox; +import org.apache.iceberg.geospatial.GeospatialBound; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -159,8 +162,13 @@ public Void predicate(BoundPredicate pred) { if (pred.isLiteralPredicate()) { gen.writeFieldName(VALUE); - SingleValueParser.toJson( - pred.term().type(), pred.asLiteralPredicate().literal().value(), gen); + if (pred.op() == Operation.ST_INTERSECTS || pred.op() == Operation.ST_DISJOINT) { + ByteBuffer value = (ByteBuffer) pred.asLiteralPredicate().literal().value(); + geospatialBoundingBox(BoundingBox.fromByteBuffer(value)); + } else { + SingleValueParser.toJson( + pred.term().type(), pred.asLiteralPredicate().literal().value(), gen); + } } else if (pred.isSetPredicate()) { gen.writeArrayFieldStart(VALUES); for (T value : pred.asSetPredicate().literalSet()) { @@ -192,6 +200,11 @@ public Void predicate(UnboundPredicate pred) { } gen.writeEndArray(); + } else if (pred.op() == Expression.Operation.ST_INTERSECTS + || pred.op() == Expression.Operation.ST_DISJOINT) { + gen.writeFieldName(VALUE); + ByteBuffer value = (ByteBuffer) pred.literal().value(); + geospatialBoundingBox(BoundingBox.fromByteBuffer(value)); } else { gen.writeFieldName(VALUE); unboundLiteral(pred.literal().value()); @@ -229,6 +242,44 @@ private void unboundLiteral(Object object) throws IOException { } } + private void geospatialBoundingBox(BoundingBox value) throws IOException { + gen.writeStartObject(); + + // Write x coordinate + gen.writeFieldName("x"); + gen.writeStartObject(); + gen.writeNumberField("min", value.min().x()); + gen.writeNumberField("max", value.max().x()); + gen.writeEndObject(); + + // Write y coordinate + gen.writeFieldName("y"); + gen.writeStartObject(); + gen.writeNumberField("min", value.min().y()); + gen.writeNumberField("max", value.max().y()); + gen.writeEndObject(); + + // Write z coordinate if present + if (value.min().hasZ() || value.max().hasZ()) { + gen.writeFieldName("z"); + gen.writeStartObject(); + gen.writeNumberField("min", value.min().z()); + gen.writeNumberField("max", value.max().z()); + gen.writeEndObject(); + } + + // Write m coordinate if present + if (value.min().hasM() || value.max().hasM()) { + gen.writeFieldName("m"); + gen.writeStartObject(); + gen.writeNumberField("min", value.min().m()); + gen.writeNumberField("max", value.max().m()); + gen.writeEndObject(); + } + + gen.writeEndObject(); + } + private String operationType(Expression.Operation op) { return op.toString().replaceAll("_", "-").toLowerCase(Locale.ENGLISH); } @@ -306,6 +357,9 @@ static Expression fromJson(JsonNode json, Schema schema) { return Expressions.or( fromJson(JsonUtil.get(LEFT, json), schema), fromJson(JsonUtil.get(RIGHT, json), schema)); + case ST_INTERSECTS: + case ST_DISJOINT: + return geospatialPredicateFromJson(op, json); } return predicateFromJson(op, json, schema); @@ -374,6 +428,15 @@ private static UnboundPredicate predicateFromJson( } } + private static Expression geospatialPredicateFromJson(Expression.Operation op, JsonNode node) { + UnboundTerm term = term(JsonUtil.get(TERM, node)); + Preconditions.checkArgument(node.has(VALUE), "Cannot parse %s predicate: missing value", op); + Preconditions.checkArgument( + !node.has(VALUES), "Cannot parse %s predicate: has invalid values field", op); + BoundingBox boundingBox = geospatialBoundingBox(JsonUtil.get(VALUE, node)); + return Expressions.geospatialPredicate(op, term, boundingBox); + } + private static T literal(JsonNode valueNode, Function toValue) { if (valueNode.isObject() && valueNode.has(TYPE)) { String type = JsonUtil.getString(TYPE, valueNode); @@ -386,6 +449,51 @@ private static T literal(JsonNode valueNode, Function toValue) return toValue.apply(valueNode); } + private static BoundingBox geospatialBoundingBox(JsonNode valueNode) { + // X and Y coordinates are required + double xMin = valueNode.get("x").get("min").asDouble(); + double xMax = valueNode.get("x").get("max").asDouble(); + double yMin = valueNode.get("y").get("min").asDouble(); + double yMax = valueNode.get("y").get("max").asDouble(); + + // Create GeospatialBound objects for min and max + GeospatialBound minBound; + GeospatialBound maxBound; + + // Check if Z coordinate exists + boolean hasZ = valueNode.has("z"); + // Check if M coordinate exists + boolean hasM = valueNode.has("m"); + + if (hasZ && hasM) { + // Both Z and M present + double zMin = valueNode.get("z").get("min").asDouble(); + double zMax = valueNode.get("z").get("max").asDouble(); + double mMin = valueNode.get("m").get("min").asDouble(); + double mMax = valueNode.get("m").get("max").asDouble(); + minBound = GeospatialBound.createXYZM(xMin, yMin, zMin, mMin); + maxBound = GeospatialBound.createXYZM(xMax, yMax, zMax, mMax); + } else if (hasZ) { + // Only Z present, no M + double zMin = valueNode.get("z").get("min").asDouble(); + double zMax = valueNode.get("z").get("max").asDouble(); + minBound = GeospatialBound.createXYZ(xMin, yMin, zMin); + maxBound = GeospatialBound.createXYZ(xMax, yMax, zMax); + } else if (hasM) { + // Only M present, no Z + double mMin = valueNode.get("m").get("min").asDouble(); + double mMax = valueNode.get("m").get("max").asDouble(); + minBound = GeospatialBound.createXYM(xMin, yMin, mMin); + maxBound = GeospatialBound.createXYM(xMax, yMax, mMax); + } else { + // Only X and Y present + minBound = GeospatialBound.createXY(xMin, yMin); + maxBound = GeospatialBound.createXY(xMax, yMax); + } + + return new BoundingBox(minBound, maxBound); + } + private static Object asObject(JsonNode node) { if (node.isIntegralNumber() && node.canConvertToLong()) { return node.asLong(); diff --git a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java index 43e2f13b55c9..fca3ed192501 100644 --- a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java +++ b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java @@ -28,6 +28,8 @@ import java.nio.ByteBuffer; import java.util.UUID; import org.apache.iceberg.Schema; +import org.apache.iceberg.geospatial.BoundingBox; +import org.apache.iceberg.geospatial.GeospatialBound; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; @@ -51,7 +53,9 @@ public class TestExpressionParser { required(114, "dec_9_0", Types.DecimalType.of(9, 0)), required(115, "dec_11_2", Types.DecimalType.of(11, 2)), required(116, "dec_38_10", Types.DecimalType.of(38, 10)), // maximum precision - required(117, "time", Types.TimeType.get())); + required(117, "time", Types.TimeType.get()), + required(118, "geom", Types.GeometryType.crs84()), + required(119, "geog", Types.GeographyType.crs84())); private static final Schema SCHEMA = new Schema(SUPPORTED_PRIMITIVES.fields()); @Test @@ -94,7 +98,22 @@ public void testSimpleExpressions() { Expressions.or( Expressions.greaterThan(Expressions.day("ts"), "2022-08-14"), Expressions.equal("date", "2022-08-14")), - Expressions.not(Expressions.in("l", 1, 2, 3, 4)) + Expressions.not(Expressions.in("l", 1, 2, 3, 4)), + Expressions.stIntersects( + "geom", + new BoundingBox(GeospatialBound.createXY(1, 2), GeospatialBound.createXY(3, 4))), + Expressions.stDisjoint( + "geom", + new BoundingBox( + GeospatialBound.createXYM(1, 2, 3), GeospatialBound.createXYM(3, 4, 5))), + Expressions.stIntersects( + "geog", + new BoundingBox( + GeospatialBound.createXYZ(1, 2, 3), GeospatialBound.createXYZ(3, 4, 5))), + Expressions.stDisjoint( + "geog", + new BoundingBox( + GeospatialBound.createXYZM(1, 2, 3, 4), GeospatialBound.createXYZM(3, 4, 5, 6))) }; for (Expression expr : expressions) { @@ -544,4 +563,122 @@ public void testNegativeScaleDecimalLiteral() { assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) .isEqualTo(expected); } + + @Test + public void testSpatialPredicate() { + String expected = + "{\n" + + " \"type\" : \"st-intersects\",\n" + + " \"term\" : \"column-name\",\n" + + " \"value\" : {\n" + + " \"x\" : {\n" + + " \"min\" : 1.0,\n" + + " \"max\" : 3.0\n" + + " },\n" + + " \"y\" : {\n" + + " \"min\" : 2.0,\n" + + " \"max\" : 4.0\n" + + " }\n" + + " }\n" + + "}"; + + Expression expression = + Expressions.stIntersects( + "column-name", + new BoundingBox(GeospatialBound.createXY(1, 2), GeospatialBound.createXY(3, 4))); + assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) + .isEqualTo(expected); + + expected = + "{\n" + + " \"type\" : \"st-intersects\",\n" + + " \"term\" : \"column-name\",\n" + + " \"value\" : {\n" + + " \"x\" : {\n" + + " \"min\" : 1.0,\n" + + " \"max\" : 3.0\n" + + " },\n" + + " \"y\" : {\n" + + " \"min\" : 2.0,\n" + + " \"max\" : 4.0\n" + + " },\n" + + " \"m\" : {\n" + + " \"min\" : 3.0,\n" + + " \"max\" : 5.0\n" + + " }\n" + + " }\n" + + "}"; + + expression = + Expressions.stIntersects( + "column-name", + new BoundingBox( + GeospatialBound.createXYM(1, 2, 3), GeospatialBound.createXYM(3, 4, 5))); + assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) + .isEqualTo(expected); + + expected = + "{\n" + + " \"type\" : \"st-intersects\",\n" + + " \"term\" : \"column-name\",\n" + + " \"value\" : {\n" + + " \"x\" : {\n" + + " \"min\" : 1.0,\n" + + " \"max\" : 3.0\n" + + " },\n" + + " \"y\" : {\n" + + " \"min\" : 2.0,\n" + + " \"max\" : 4.0\n" + + " },\n" + + " \"z\" : {\n" + + " \"min\" : 3.0,\n" + + " \"max\" : 5.0\n" + + " }\n" + + " }\n" + + "}"; + + expression = + Expressions.stIntersects( + "column-name", + new BoundingBox( + GeospatialBound.createXYZ(1, 2, 3), GeospatialBound.createXYZ(3, 4, 5))); + assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) + .isEqualTo(expected); + + expected = + "{\n" + + " \"type\" : \"st-intersects\",\n" + + " \"term\" : \"column-name\",\n" + + " \"value\" : {\n" + + " \"x\" : {\n" + + " \"min\" : 1.0,\n" + + " \"max\" : 3.0\n" + + " },\n" + + " \"y\" : {\n" + + " \"min\" : 2.0,\n" + + " \"max\" : 4.0\n" + + " },\n" + + " \"z\" : {\n" + + " \"min\" : 3.0,\n" + + " \"max\" : 5.0\n" + + " },\n" + + " \"m\" : {\n" + + " \"min\" : 4.0,\n" + + " \"max\" : 6.0\n" + + " }\n" + + " }\n" + + "}"; + + expression = + Expressions.stIntersects( + "column-name", + new BoundingBox( + GeospatialBound.createXYZM(1, 2, 3, 4), GeospatialBound.createXYZM(3, 4, 5, 6))); + assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) + .isEqualTo(expected); + } }