rdblue commented on a change in pull request #1346: URL: https://github.com/apache/iceberg/pull/1346#discussion_r494662994
########## File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java ########## @@ -0,0 +1,340 @@ +/* + * 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.flink.source; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Locale; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.iceberg.types.Types.NestedField.required; + +@RunWith(Parameterized.class) +public abstract class TestFlinkScan extends AbstractTestBase { + + private static final Schema SCHEMA = new Schema( + required(1, "data", Types.StringType.get()), + required(2, "id", Types.LongType.get()), + required(3, "dt", Types.StringType.get())); + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA) + .identity("dt") + .bucket("id", 1) + .build(); + + // before variables + private Configuration conf; + String warehouse; + private HadoopCatalog catalog; + + // parametrized variables + private final FileFormat fileFormat; + + @Parameterized.Parameters + public static Object[] parameters() { + // TODO add orc and parquet + return new Object[] {"avro"}; + } + + TestFlinkScan(String fileFormat) { + this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH)); + } + + @Before + public void before() throws IOException { + File warehouseFile = TEMPORARY_FOLDER.newFolder(); + Assert.assertTrue(warehouseFile.delete()); + conf = new Configuration(); + warehouse = "file:" + warehouseFile; + catalog = new HadoopCatalog(conf, warehouse); + } + + private List<Row> execute(Table table) throws IOException { + return executeWithOptions(table, null, null, null, null, null, null, null, null); + } + + private List<Row> execute(Table table, List<String> projectFields) throws IOException { + return executeWithOptions(table, projectFields, null, null, null, null, null, null, null); + } + + protected abstract List<Row> executeWithOptions( + Table table, List<String> projectFields, CatalogLoader loader, Long snapshotId, + Long startSnapshotId, Long endSnapshotId, Long asOfTimestamp, List<Expression> filters, String sqlFilter) + throws IOException; + + protected abstract void assertResiduals(List<Row> results, List<Record> writeRecords, List<Record> filteredRecords) + throws IOException; + + @Test + public void testUnpartitionedTable() throws Exception { + Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA); + List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 2, 0L); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); + assertRecords(execute(table), expectedRecords); + } + + @Test + public void testPartitionedTable() throws Exception { + Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC); + List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L); + expectedRecords.get(0).set(2, "2020-03-20"); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable( + org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); + assertRecords(execute(table), expectedRecords); + } + + @Test + public void testProjection() throws Exception { Review comment: Field reordering tests are at the file format level. Each file format has to be able to project columns in the requested order. So any reordered schema should work as long as it is passed down correctly, which is what the new `convert` method does. ########## File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java ########## @@ -0,0 +1,385 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.flink.source; + +import java.io.File; +import java.io.IOException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Locale; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.conversion.DataStructureConverters; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.iceberg.types.Types.NestedField.required; + +@RunWith(Parameterized.class) +public abstract class TestFlinkScan extends AbstractTestBase { + + private static final Schema SCHEMA = new Schema( + required(1, "data", Types.StringType.get()), + required(2, "id", Types.LongType.get()), + required(3, "dt", Types.StringType.get())); + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA) + .identity("dt") + .bucket("id", 1) + .build(); + + private HadoopCatalog catalog; + protected String warehouse; + + // parametrized variables + private final FileFormat fileFormat; + + @Parameterized.Parameters(name = "format={0}") + public static Object[] parameters() { + return new Object[] {"avro", "parquet", "orc"}; + } + + TestFlinkScan(String fileFormat) { + this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH)); + } + + @Before + public void before() throws IOException { + File warehouseFile = TEMPORARY_FOLDER.newFolder(); + Assert.assertTrue(warehouseFile.delete()); + // before variables + Configuration conf = new Configuration(); + warehouse = "file:" + warehouseFile; + catalog = new HadoopCatalog(conf, warehouse); + } + + private List<Row> execute(Table table) throws IOException { + return execute(table, ScanOptions.builder().build()); + } + + protected abstract List<Row> execute(Table table, List<String> projectFields) throws IOException; + + protected abstract List<Row> execute(Table table, ScanOptions options) throws IOException; + + protected abstract List<Row> execute(Table table, List<Expression> filters, String sqlFilter) throws IOException; + + /** + * The Flink SQL has no residuals, because there will be operator to filter all the data that should be filtered. + * But the FlinkInputFormat can't. + */ + protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords, + List<Record> filteredRecords) throws IOException; + + /** + * Schema: [data, nested[f1, f2, f3], id] + * Projection: [nested.f2, data] + * The Flink SQL output: [f2, data] + * The FlinkInputFormat output: [nested[f2], data]. + */ + protected abstract void assertNestedProjection(Table table, List<Record> records) throws IOException; + + @Test + public void testUnpartitionedTable() throws Exception { + Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA); + List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 2, 0L); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); + assertRecords(execute(table), expectedRecords, SCHEMA); + } + + @Test + public void testPartitionedTable() throws Exception { + Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC); + List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L); + expectedRecords.get(0).set(2, "2020-03-20"); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable( + org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); + assertRecords(execute(table), expectedRecords, SCHEMA); + } + + @Test + public void testProjection() throws Exception { + Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC); + List<Record> inputRecords = RandomGenericData.generate(SCHEMA, 1, 0L); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable( + org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), inputRecords); + assertRows(execute(table, Collections.singletonList("data")), Row.of(inputRecords.get(0).get(0))); + } + + @Test + public void testIdentityPartitionProjections() throws Exception { + Schema logSchema = new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "level", Types.StringType.get()), + Types.NestedField.optional(4, "message", Types.StringType.get()) + ); + PartitionSpec spec = + PartitionSpec.builderFor(logSchema).identity("dt").identity("level").build(); + + Table table = catalog.createTable(TableIdentifier.of("default", "t"), logSchema, spec); + List<Record> inputRecords = RandomGenericData.generate(logSchema, 10, 0L); + + int idx = 0; + AppendFiles append = table.newAppend(); + for (Record record : inputRecords) { + record.set(1, "2020-03-2" + idx); + record.set(2, Integer.toString(idx)); + append.appendFile(new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).writeFile( + org.apache.iceberg.TestHelpers.Row.of("2020-03-2" + idx, Integer.toString(idx)), ImmutableList.of(record))); + idx += 1; + } + append.commit(); + + // individual fields + validateIdentityPartitionProjections(table, Collections.singletonList("dt"), inputRecords); + validateIdentityPartitionProjections(table, Collections.singletonList("level"), inputRecords); + validateIdentityPartitionProjections(table, Collections.singletonList("message"), inputRecords); + validateIdentityPartitionProjections(table, Collections.singletonList("id"), inputRecords); + // field pairs + validateIdentityPartitionProjections(table, Arrays.asList("dt", "message"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("level", "message"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("dt", "level"), inputRecords); + // out-of-order pairs + validateIdentityPartitionProjections(table, Arrays.asList("message", "dt"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("message", "level"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("level", "dt"), inputRecords); + // out-of-order triplets + validateIdentityPartitionProjections(table, Arrays.asList("dt", "level", "message"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("level", "dt", "message"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("dt", "message", "level"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("level", "message", "dt"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("message", "dt", "level"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("message", "level", "dt"), inputRecords); + } + + private void validateIdentityPartitionProjections(Table table, List<String> projectedFields, + List<Record> inputRecords) throws IOException { + List<Row> rows = execute(table, projectedFields); + + for (int pos = 0; pos < inputRecords.size(); pos++) { + Record inputRecord = inputRecords.get(pos); + Row actualRecord = rows.get(pos); + + for (int i = 0; i < projectedFields.size(); i++) { + String name = projectedFields.get(i); + Assert.assertEquals( + "Projected field " + name + " should match", inputRecord.getField(name), actualRecord.getField(i)); + } + } + } + + @Test + public void testSnapshotReads() throws Exception { + Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER); + + List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L); + helper.appendToTable(expectedRecords); + long snapshotId = table.currentSnapshot().snapshotId(); + + long timestampMillis = table.currentSnapshot().timestampMillis(); + + // produce another timestamp + Thread.sleep(10); Review comment: We usually introduce a spin to avoid sleeping for long durations in lots of tests, like this: https://github.com/apache/iceberg/blob/master/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java#L78 ########## File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java ########## @@ -0,0 +1,385 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.flink.source; + +import java.io.File; +import java.io.IOException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Locale; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.conversion.DataStructureConverters; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.iceberg.types.Types.NestedField.required; + +@RunWith(Parameterized.class) +public abstract class TestFlinkScan extends AbstractTestBase { + + private static final Schema SCHEMA = new Schema( + required(1, "data", Types.StringType.get()), + required(2, "id", Types.LongType.get()), + required(3, "dt", Types.StringType.get())); + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA) + .identity("dt") + .bucket("id", 1) + .build(); + + private HadoopCatalog catalog; + protected String warehouse; + + // parametrized variables + private final FileFormat fileFormat; + + @Parameterized.Parameters(name = "format={0}") + public static Object[] parameters() { + return new Object[] {"avro", "parquet", "orc"}; + } + + TestFlinkScan(String fileFormat) { + this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH)); + } + + @Before + public void before() throws IOException { + File warehouseFile = TEMPORARY_FOLDER.newFolder(); + Assert.assertTrue(warehouseFile.delete()); + // before variables + Configuration conf = new Configuration(); + warehouse = "file:" + warehouseFile; + catalog = new HadoopCatalog(conf, warehouse); + } + + private List<Row> execute(Table table) throws IOException { + return execute(table, ScanOptions.builder().build()); + } + + protected abstract List<Row> execute(Table table, List<String> projectFields) throws IOException; + + protected abstract List<Row> execute(Table table, ScanOptions options) throws IOException; + + protected abstract List<Row> execute(Table table, List<Expression> filters, String sqlFilter) throws IOException; + + /** + * The Flink SQL has no residuals, because there will be operator to filter all the data that should be filtered. + * But the FlinkInputFormat can't. + */ + protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords, + List<Record> filteredRecords) throws IOException; + + /** + * Schema: [data, nested[f1, f2, f3], id] + * Projection: [nested.f2, data] + * The Flink SQL output: [f2, data] + * The FlinkInputFormat output: [nested[f2], data]. + */ + protected abstract void assertNestedProjection(Table table, List<Record> records) throws IOException; Review comment: I find it really strange that this is delegated to a subclass, given that it builds a very specific nested projection. Why not make this use a method like `execute(Table, List<String>)`, but pass in the projection instead of a list of fields? Then you could keep all of the schema details in the test method here, rather than delegating this assertion. I think it would be cleaner. ########## File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java ########## @@ -0,0 +1,385 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.flink.source; + +import java.io.File; +import java.io.IOException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Locale; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.conversion.DataStructureConverters; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.iceberg.types.Types.NestedField.required; + +@RunWith(Parameterized.class) +public abstract class TestFlinkScan extends AbstractTestBase { + + private static final Schema SCHEMA = new Schema( + required(1, "data", Types.StringType.get()), + required(2, "id", Types.LongType.get()), + required(3, "dt", Types.StringType.get())); + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA) + .identity("dt") + .bucket("id", 1) + .build(); + + private HadoopCatalog catalog; + protected String warehouse; + + // parametrized variables + private final FileFormat fileFormat; + + @Parameterized.Parameters(name = "format={0}") + public static Object[] parameters() { + return new Object[] {"avro", "parquet", "orc"}; + } + + TestFlinkScan(String fileFormat) { + this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH)); + } + + @Before + public void before() throws IOException { + File warehouseFile = TEMPORARY_FOLDER.newFolder(); + Assert.assertTrue(warehouseFile.delete()); + // before variables + Configuration conf = new Configuration(); + warehouse = "file:" + warehouseFile; + catalog = new HadoopCatalog(conf, warehouse); + } + + private List<Row> execute(Table table) throws IOException { + return execute(table, ScanOptions.builder().build()); + } + + protected abstract List<Row> execute(Table table, List<String> projectFields) throws IOException; + + protected abstract List<Row> execute(Table table, ScanOptions options) throws IOException; + + protected abstract List<Row> execute(Table table, List<Expression> filters, String sqlFilter) throws IOException; + + /** + * The Flink SQL has no residuals, because there will be operator to filter all the data that should be filtered. + * But the FlinkInputFormat can't. + */ + protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords, + List<Record> filteredRecords) throws IOException; Review comment: Why is this implemented by the subclass? Couldn't this just call `assertRecords` directly? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org