Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion .github/trigger_files/IO_Iceberg_Integration_Tests.json
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run.",
"modification": 3
"modification": 2
}
5 changes: 5 additions & 0 deletions sdks/java/io/iceberg/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,10 @@ dependencies {
implementation "org.apache.iceberg:iceberg-orc:$iceberg_version"
implementation "org.apache.iceberg:iceberg-data:$iceberg_version"
implementation library.java.hadoop_common
// TODO(https://github.com/apache/beam/issues/21156): Determine how to build without this dependency
provided "org.immutables:value:2.8.8"
permitUnusedDeclared "org.immutables:value:2.8.8"
implementation library.java.vendored_calcite_1_28_0
runtimeOnly "org.apache.iceberg:iceberg-gcp:$iceberg_version"

testImplementation project(":sdks:java:managed")
Expand Down Expand Up @@ -173,6 +177,7 @@ task dataflowIntegrationTest(type: Test) {

filter {
includeTestsMatching 'org.apache.beam.sdk.io.iceberg.catalog.BigQueryMetastoreCatalogIT.testRead'
includeTestsMatching 'org.apache.beam.sdk.io.iceberg.catalog.BigQueryMetastoreCatalogIT.testReadWithFilter'
includeTestsMatching 'org.apache.beam.sdk.io.iceberg.catalog.BigQueryMetastoreCatalogIT.testStreamingRead'
includeTestsMatching 'org.apache.beam.sdk.io.iceberg.catalog.BigQueryMetastoreCatalogIT.testWrite'
includeTestsMatching 'org.apache.beam.sdk.io.iceberg.catalog.BigQueryMetastoreCatalogIT.testWriteRead'
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.iceberg.DataOperations;
import org.apache.iceberg.IncrementalAppendScan;
import org.apache.iceberg.Table;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.io.CloseableIterable;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Instant;
Expand Down Expand Up @@ -86,6 +87,10 @@ public void process(
if (fromSnapshot != null) {
scan = scan.fromSnapshotExclusive(fromSnapshot);
}
@Nullable Expression filter = scanConfig.getFilter();
if (filter != null) {
scan = scan.filter(filter);
}

createAndOutputReadTasks(scan, snapshot, out);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,283 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.io.iceberg;

import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;

import java.math.BigDecimal;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.function.BiFunction;
import java.util.stream.Collectors;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlBasicCall;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlLiteral;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNodeList;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParseException;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParser;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.Schema;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.Expression.Operation;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.types.Type.TypeID;
import org.apache.iceberg.util.DateTimeUtil;
import org.apache.iceberg.util.NaNUtil;
import org.checkerframework.checker.nullness.qual.Nullable;

/**
* Utilities that convert between a SQL filter expression and an Iceberg {@link Expression}. Uses
* Apache Calcite semantics.
*
* <p>Note: Only supports top-level fields (i.e. cannot reference nested fields).
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's make sure we clearly fail for unsupported queries.

*/
class FilterUtils {
private static final Map<SqlKind, Operation> FILTERS =
ImmutableMap.<SqlKind, Operation>builder()
.put(SqlKind.IS_NULL, Operation.IS_NULL)
.put(SqlKind.IS_NOT_NULL, Operation.NOT_NULL)
.put(SqlKind.LESS_THAN, Operation.LT)
.put(SqlKind.LESS_THAN_OR_EQUAL, Operation.LT_EQ)
.put(SqlKind.GREATER_THAN, Operation.GT)
.put(SqlKind.GREATER_THAN_OR_EQUAL, Operation.GT_EQ)
.put(SqlKind.EQUALS, Operation.EQ)
.put(SqlKind.NOT_EQUALS, Operation.NOT_EQ)
.put(SqlKind.IN, Operation.IN)
.put(SqlKind.NOT_IN, Operation.NOT_IN)
.put(SqlKind.AND, Operation.AND)
.put(SqlKind.OR, Operation.OR)
.build();

static Expression convert(@Nullable String filter, Schema schema) {
if (filter == null) {
return Expressions.alwaysTrue();
}

SqlParser parser = SqlParser.create(filter);
try {
SqlNode expression = parser.parseExpression();
return convert(expression, schema);
} catch (Exception exception) {
throw new RuntimeException(
String.format("Encountered an error when parsing filter: '%s'", filter), exception);
}
}

private static Expression convert(SqlNode expression, Schema schema) throws SqlParseException {
checkArgument(expression instanceof SqlBasicCall);
SqlBasicCall call = (SqlBasicCall) expression;

SqlOperator op = call.getOperator();
SqlKind kind = op.getKind();

Operation operation =
checkArgumentNotNull(
FILTERS.get(kind),
"Unable to convert SQL operation '%s' in Iceberg expression: %s",
kind,
expression.toString());

switch (operation) {
case IS_NULL:
return Expressions.isNull(getOnlyChildName(call));
case NOT_NULL:
return Expressions.notNull(getOnlyChildName(call));
case LT:
return convertFieldAndLiteral(
Expressions::lessThan, Expressions::greaterThan, call, schema);
case LT_EQ:
return convertFieldAndLiteral(
Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call, schema);
case GT:
return convertFieldAndLiteral(
Expressions::greaterThan, Expressions::lessThan, call, schema);
case GT_EQ:
return convertFieldAndLiteral(
Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call, schema);
case EQ:
return convertFieldAndLiteral(
(ref, lit) -> {
if (lit == null) {
return Expressions.isNull(ref);
} else if (NaNUtil.isNaN(lit)) {
return Expressions.isNaN(ref);
} else {
return Expressions.equal(ref, lit);
}
},
call,
schema);
case NOT_EQ:
return convertFieldAndLiteral(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's try to do 100% test coverage for this file.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Tried to do that in FilterUtilsTest. Let me know if anything is missing

(ref, lit) -> {
if (lit == null) {
return Expressions.notNull(ref);
} else if (NaNUtil.isNaN(lit)) {
return Expressions.notNaN(ref);
} else {
return Expressions.notEqual(ref, lit);
}
},
call,
schema);
case IN:
return convertFieldInLiteral(Operation.IN, call, schema);
case NOT_IN:
return convertFieldInLiteral(Operation.NOT_IN, call, schema);
case AND:
return convertLogicalExpr(Expressions::and, call, schema);
case OR:
return convertLogicalExpr(Expressions::or, call, schema);
default:
throw new IllegalArgumentException(
String.format("Unsupported operation '%s' in filter expression: %s", operation, call));
}
}

private static String getOnlyChildName(SqlBasicCall call) {
checkArgument(
call.operandCount() == 1,
"Expected only 1 operand but got %s in filter: %s",
call.getOperandList(),
call.toString());
SqlNode ref = call.operand(0);
Preconditions.checkState(
ref instanceof SqlIdentifier, "Expected operand '%s' to be a reference.", ref);
return ((SqlIdentifier) ref).getSimple();
}

private static SqlNode getLeftChild(SqlBasicCall call) {
checkArgument(
call.operandCount() == 2,
"Expected 2 operands but got %s in filter: %s",
call.getOperandList(),
call.toString());
return call.operand(0);
}

private static SqlNode getRightChild(SqlBasicCall call) {
checkArgument(
call.operandCount() == 2,
"Expected 2 operands but got %s in filter: %s",
call.getOperandList(),
call.toString());
return call.operand(1);
}

private static Expression convertLogicalExpr(
BiFunction<Expression, Expression, Expression> expr, SqlBasicCall call, Schema schema)
throws SqlParseException {
SqlNode left = getLeftChild(call);
SqlNode right = getRightChild(call);
return expr.apply(convert(left, schema), convert(right, schema));
}

private static Expression convertFieldInLiteral(Operation op, SqlBasicCall call, Schema schema) {
checkArgument(
call.operandCount() == 2,
"Expected only 2 operands but got %s: %s",
call.operandCount(),
call);
SqlNode term = call.operand(0);
SqlNode value = call.operand(1);
checkArgument(
term instanceof SqlIdentifier,
"Expected left hand side to be a field identifier but got " + term.getClass());
checkArgument(
value instanceof SqlNodeList,
"Expected right hand side to be a list but got " + value.getClass());
String name = ((SqlIdentifier) term).getSimple();
TypeID type = schema.findType(name).typeId();
List<SqlNode> list =
((SqlNodeList) value)
.getList().stream().filter(Objects::nonNull).collect(Collectors.toList());
checkArgument(list.stream().allMatch(o -> o instanceof SqlLiteral));
List<Object> values =
list.stream()
.map(o -> convertLiteral((SqlLiteral) o, name, type))
.collect(Collectors.toList());
return op == Operation.IN ? Expressions.in(name, values) : Expressions.notIn(name, values);
}

private static Expression convertFieldAndLiteral(
BiFunction<String, Object, Expression> expr, SqlBasicCall call, Schema schema) {
return convertFieldAndLiteral(expr, expr, call, schema);
}

private static Expression convertFieldAndLiteral(
BiFunction<String, Object, Expression> convertLR,
BiFunction<String, Object, Expression> convertRL,
SqlBasicCall call,
Schema schema) {
SqlNode left = getLeftChild(call);
SqlNode right = getRightChild(call);
if (left instanceof SqlIdentifier && right instanceof SqlLiteral) {
String name = ((SqlIdentifier) left).getSimple();
TypeID type = schema.findType(name).typeId();
Object value = convertLiteral((SqlLiteral) right, name, type);
return convertLR.apply(name, value);
} else if (left instanceof SqlLiteral && right instanceof SqlIdentifier) {
String name = ((SqlIdentifier) right).getSimple();
TypeID type = schema.findType(name).typeId();
Object value = convertLiteral((SqlLiteral) left, name, type);
return convertRL.apply(name, value);
} else {
throw new IllegalArgumentException("Unsupported operands for expression: " + call);
}
}

private static Object convertLiteral(SqlLiteral literal, String field, TypeID type) {
switch (type) {
case BOOLEAN:
return literal.getValueAs(Boolean.class);
case INTEGER:
return literal.getValueAs(Integer.class);
case LONG:
return literal.getValueAs(Long.class);
case FLOAT:
return literal.getValueAs(Float.class);
case DOUBLE:
return literal.getValueAs(Double.class);
case DECIMAL:
return literal.getValueAs(BigDecimal.class);
case STRING:
return literal.getValueAs(String.class);
case DATE:
LocalDate date = LocalDate.parse(literal.getValueAs(String.class));
return DateTimeUtil.daysFromDate(date);
case TIME:
LocalTime time = LocalTime.parse(literal.getValueAs(String.class));
return DateTimeUtil.microsFromTime(time);
case TIMESTAMP:
LocalDateTime dateTime = LocalDateTime.parse(literal.getValueAs(String.class));
return DateTimeUtil.microsFromTimestamp(dateTime);
default:
throw new IllegalArgumentException(
String.format("Unsupported filter type in field '%s': %s", field, type));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,8 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) {
.withStartingStrategy(strategy)
.streaming(configuration.getStreaming())
.keeping(configuration.getKeep())
.dropping(configuration.getDrop());
.dropping(configuration.getDrop())
.withFilter(configuration.getFilter());

@Nullable Integer pollIntervalSeconds = configuration.getPollIntervalSeconds();
if (pollIntervalSeconds != null) {
Expand Down Expand Up @@ -179,6 +180,12 @@ static Builder builder() {
"The interval at which to poll for new snapshots. Defaults to 60 seconds.")
abstract @Nullable Integer getPollIntervalSeconds();

@SchemaFieldDescription(
"SQL-like predicate to filter data at scan time. Example: \"id > 5 AND status = 'ACTIVE'\". "
+ "Uses Apache Calcite syntax: https://calcite.apache.org/docs/reference.html")
@Nullable
abstract String getFilter();

@SchemaFieldDescription(
"A subset of column names to read exclusively. If null or empty, all columns will be read.")
abstract @Nullable List<String> getKeep();
Expand Down Expand Up @@ -215,6 +222,8 @@ abstract static class Builder {

abstract Builder setDrop(List<String> drop);

abstract Builder setFilter(String filter);

abstract Configuration build();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -497,6 +497,8 @@ public enum StartingStrategy {

abstract @Nullable List<String> getDrop();

abstract @Nullable String getFilter();

abstract Builder toBuilder();

@AutoValue.Builder
Expand Down Expand Up @@ -525,6 +527,8 @@ abstract static class Builder {

abstract Builder setDrop(@Nullable List<String> fields);

abstract Builder setFilter(@Nullable String filter);

abstract ReadRows build();
}

Expand Down Expand Up @@ -572,6 +576,10 @@ public ReadRows dropping(@Nullable List<String> drop) {
return toBuilder().setDrop(drop).build();
}

public ReadRows withFilter(@Nullable String filter) {
return toBuilder().setFilter(filter).build();
}

@Override
public PCollection<Row> expand(PBegin input) {
TableIdentifier tableId =
Expand All @@ -595,6 +603,7 @@ public PCollection<Row> expand(PBegin input) {
.setUseCdc(getUseCdc())
.setKeepFields(getKeep())
.setDropFields(getDrop())
.setFilter(FilterUtils.convert(getFilter(), table.schema()))
.build();
scanConfig.validate(table);

Expand Down
Loading
Loading