JingsongLi commented on code in PR #396:
URL: https://github.com/apache/flink-table-store/pull/396#discussion_r1033295182


##########
flink-table-store-core/src/test/java/org/apache/flink/table/store/table/SchemaEvolutionTableTestBase.java:
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.flink.table.store.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.file.schema.AtomicDataType;
+import org.apache.flink.table.store.file.schema.DataField;
+import org.apache.flink.table.store.file.schema.SchemaChange;
+import org.apache.flink.table.store.file.schema.SchemaManager;
+import org.apache.flink.table.store.file.schema.TableSchema;
+import org.apache.flink.table.store.file.schema.UpdateSchema;
+import org.apache.flink.table.store.file.utils.TestAtomicRenameFileSystem;
+import org.apache.flink.table.store.file.utils.TraceableFileSystem;
+import org.apache.flink.table.store.table.sink.TableCommit;
+import org.apache.flink.table.store.table.sink.TableWrite;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Base test class for schema evolution in {@link FileStoreTable}. */
+public abstract class SchemaEvolutionTableTestBase {

Review Comment:
   Can we extract a `TableTestBase` for `SchemaEvolutionTableTestBase` and 
`FileStoreTableTestBase`?



##########
flink-table-store-common/src/main/java/org/apache/flink/table/store/file/predicate/AlwaysFalse.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.flink.table.store.file.predicate;
+
+import org.apache.flink.table.store.format.FieldStats;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import java.util.Optional;
+
+/** Return false for all values. */
+public class AlwaysFalse extends LeafUnaryFunction {
+    public static final AlwaysFalse INSTANCE = new AlwaysFalse();
+
+    private AlwaysFalse() {}
+
+    @Override
+    public Optional<LeafFunction> negate() {
+        return Optional.of(AlwaysTrue.INSTANCE);
+    }
+
+    @Override
+    public boolean test(LogicalType type, Object value) {

Review Comment:
   We can have better way to `AlwaysFalse` and `AlwaysTrue`, because they don't 
need to have field reference. But this requires more refactor.
   
   Can you add TODO in the code and create a JIRA for this?



##########
flink-table-store-core/src/test/java/org/apache/flink/table/store/table/SchemaEvolutionTableTestBase.java:
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.flink.table.store.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.file.schema.AtomicDataType;
+import org.apache.flink.table.store.file.schema.DataField;
+import org.apache.flink.table.store.file.schema.SchemaChange;
+import org.apache.flink.table.store.file.schema.SchemaManager;
+import org.apache.flink.table.store.file.schema.TableSchema;
+import org.apache.flink.table.store.file.schema.UpdateSchema;
+import org.apache.flink.table.store.file.utils.TestAtomicRenameFileSystem;
+import org.apache.flink.table.store.file.utils.TraceableFileSystem;
+import org.apache.flink.table.store.table.sink.TableCommit;
+import org.apache.flink.table.store.table.sink.TableWrite;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Base test class for schema evolution in {@link FileStoreTable}. */
+public abstract class SchemaEvolutionTableTestBase {
+    protected static final List<DataField> SCHEMA_0_FIELDS =
+            Arrays.asList(
+                    new DataField(0, "a", new 
AtomicDataType(DataTypes.STRING().getLogicalType())),
+                    new DataField(1, "pt", new 
AtomicDataType(DataTypes.INT().getLogicalType())),
+                    new DataField(2, "b", new 
AtomicDataType(DataTypes.INT().getLogicalType())),
+                    new DataField(3, "c", new 
AtomicDataType(DataTypes.STRING().getLogicalType())),
+                    new DataField(4, "kt", new 
AtomicDataType(DataTypes.BIGINT().getLogicalType())),
+                    new DataField(5, "d", new 
AtomicDataType(DataTypes.STRING().getLogicalType())));
+    protected static final List<DataField> SCHEMA_1_FIELDS =
+            Arrays.asList(
+                    new DataField(1, "pt", new 
AtomicDataType(DataTypes.INT().getLogicalType())),
+                    new DataField(2, "d", new 
AtomicDataType(DataTypes.INT().getLogicalType())),
+                    new DataField(4, "kt", new 
AtomicDataType(DataTypes.BIGINT().getLogicalType())),
+                    new DataField(6, "a", new 
AtomicDataType(DataTypes.INT().getLogicalType())),
+                    new DataField(7, "f", new 
AtomicDataType(DataTypes.STRING().getLogicalType())),
+                    new DataField(8, "b", new 
AtomicDataType(DataTypes.STRING().getLogicalType())));
+    protected static final List<String> PARTITION_NAMES = 
Collections.singletonList("pt");
+    protected static final List<String> PRIMARY_KEY_NAMES = 
Arrays.asList("pt", "kt");
+
+    protected Path tablePath;
+    protected String commitUser;
+    protected final Configuration tableConfig = new Configuration();
+
+    @TempDir java.nio.file.Path tempDir;
+
+    @BeforeEach
+    public void before() throws Exception {
+        tablePath = new Path(TestAtomicRenameFileSystem.SCHEME + "://" + 
tempDir.toString());
+        commitUser = UUID.randomUUID().toString();
+        tableConfig.set(CoreOptions.PATH, tablePath.toString());
+        tableConfig.set(CoreOptions.BUCKET, 2);
+    }
+
+    @AfterEach
+    public void after() throws IOException {
+        // assert all connections are closed
+        FileSystem fileSystem = tablePath.getFileSystem();
+        assertThat(fileSystem).isInstanceOf(TraceableFileSystem.class);
+        TraceableFileSystem traceableFileSystem = (TraceableFileSystem) 
fileSystem;
+
+        java.util.function.Predicate<Path> pathPredicate =
+                path -> path.toString().contains(tempDir.toString());
+        
assertThat(traceableFileSystem.openInputStreams(pathPredicate)).isEmpty();
+        
assertThat(traceableFileSystem.openOutputStreams(pathPredicate)).isEmpty();
+    }
+
+    protected List<String> getPrimaryKeyNames() {
+        return PRIMARY_KEY_NAMES;
+    }
+
+    protected abstract FileStoreTable createFileStoreTable(Map<Long, 
TableSchema> tableSchemas);
+
+    public static <R> void writeAndCheckFileResult(
+            Function<Map<Long, TableSchema>, R> firstChecker,
+            BiConsumer<R, Map<Long, TableSchema>> secondChecker,
+            List<String> primaryKeyNames,
+            Configuration tableConfig,
+            Function<Map<Long, TableSchema>, FileStoreTable> 
createFileStoreTable)
+            throws Exception {
+        Map<Long, TableSchema> tableSchemas = new HashMap<>();
+        tableSchemas.put(
+                0L,
+                new TableSchema(
+                        0,
+                        SCHEMA_0_FIELDS,
+                        5,
+                        PARTITION_NAMES,
+                        primaryKeyNames,
+                        tableConfig.toMap(),
+                        ""));
+        FileStoreTable table = createFileStoreTable.apply(tableSchemas);
+        TableWrite write = table.newWrite("user");
+        TableCommit commit = table.newCommit("user");
+
+        write.write(
+                GenericRowData.of(
+                        StringData.fromString("S001"),

Review Comment:
   Maybe we can create a util method, `GenericRowData rowData(Object... 
values)`, we iterate over each element. If there is a String, it will be 
converted to StringData.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/schema/SchemaEvolutionUtil.java:
##########
@@ -62,4 +88,240 @@ public static int[] createIndexMapping(
         }
         return null;
     }
+
+    /**
+     * Create index mapping from table projection to underlying data 
projection. For example, the
+     * table and data fields are as follows
+     *
+     * <ul>
+     *   <li>table fields: 1->c, 3->a, 4->e, 5->d, 6->b
+     *   <li>data fields: 1->a, 2->b, 3->c, 4->d
+     * </ul>
+     *
+     * <p>The table and data top projections are as follows
+     *
+     * <ul>
+     *   <li>table projection: [0, 4, 1]
+     *   <li>data projection: [0, 2]
+     * </ul>
+     *
+     * <p>We can first get fields list for table and data projections from 
their fields as follows
+     *
+     * <ul>
+     *   <li>table projection field list: [1->c, 6->b, 3->a]
+     *   <li>data projection field list: [1->a, 3->c]
+     * </ul>
+     *
+     * <p>Then create index mapping based on the fields list.
+     *
+     * <p>/// TODO should support nest index mapping when nest schema 
evolution is supported.
+     *
+     * @param tableProjection the table projection
+     * @param tableFields the fields in table
+     * @param dataProjection the underlying data projection
+     * @param dataFields the fields in underlying data
+     * @return the index mapping
+     */
+    @Nullable
+    public static int[] createIndexMapping(
+            int[] tableProjection,
+            List<DataField> tableFields,
+            int[] dataProjection,
+            List<DataField> dataFields) {
+        List<DataField> tableProjectFields = new 
ArrayList<>(tableProjection.length);
+        for (int index : tableProjection) {
+            tableProjectFields.add(tableFields.get(index));
+        }
+
+        List<DataField> dataProjectFields = new 
ArrayList<>(dataProjection.length);
+        for (int index : dataProjection) {
+            dataProjectFields.add(dataFields.get(index));
+        }
+
+        return createIndexMapping(tableProjectFields, dataProjectFields);
+    }
+
+    /**
+     * Create index mapping from table projection to data with key and value 
fields. We should first
+     * create table and data fields with their key/value fields, then create 
index mapping with
+     * their projections and fields. For example, the table and data 
projections and fields are as
+     * follows
+     *
+     * <ul>
+     *   <li>Table key fields: 1->ka, 3->kb, 5->kc, 6->kd; value fields: 0->a, 
2->d, 4->b;
+     *       projection: [0, 2, 3, 4, 5, 7] where 0 is 1->ka, 2 is 5->kc, 3 is 
5->kc, 4/5 are seq
+     *       and kind, 7 is 2->d
+     *   <li>Data key fields: 1->kb, 5->ka; value fields: 2->aa, 4->f; 
projection: [0, 1, 2, 3, 4]
+     *       where 0 is 1->kb, 1 is 5->ka, 2/3 are seq and kind, 4 is 2->aa
+     * </ul>
+     *
+     * <p>First we will get max key id from table and data fields which is 6, 
then create table and
+     * data fields on it
+     *
+     * <ul>
+     *   <li>Table fields: 1->ka, 3->kb, 5->kc, 6->kd, 7->seq, 8->kind, 9->a, 
11->d, 13->b
+     *   <li>Data fields: 1->kb, 5->ka, 7->seq, 8->kind, 11->aa, 13->f
+     * </ul>
+     *
+     * <p>Finally we can create index mapping with table/data projections and 
fields.
+     *
+     * <p>/// TODO should support nest index mapping when nest schema 
evolution is supported.
+     *
+     * @param tableProjection the table projection
+     * @param tableKeyFields the table key fields
+     * @param tableValueFields the table value fields
+     * @param dataProjection the data projection
+     * @param dataKeyFields the data key fields
+     * @param dataValueFields the data value fields
+     * @return the result index mapping
+     */
+    @Nullable
+    public static int[] createIndexMapping(
+            int[] tableProjection,
+            List<DataField> tableKeyFields,
+            List<DataField> tableValueFields,
+            int[] dataProjection,
+            List<DataField> dataKeyFields,
+            List<DataField> dataValueFields) {
+        int maxKeyId =
+                Math.max(
+                        
tableKeyFields.stream().mapToInt(DataField::id).max().orElse(0),
+                        
dataKeyFields.stream().mapToInt(DataField::id).max().orElse(0));
+        List<DataField> tableFields =
+                KeyValue.createKeyValueFields(tableKeyFields, 
tableValueFields, maxKeyId);
+        List<DataField> dataFields =
+                KeyValue.createKeyValueFields(dataKeyFields, dataValueFields, 
maxKeyId);
+        return createIndexMapping(tableProjection, tableFields, 
dataProjection, dataFields);
+    }
+
+    /**
+     * Create data projection from table projection. For example, the table 
and data fields are as
+     * follows
+     *
+     * <ul>
+     *   <li>table fields: 1->c, 3->a, 4->e, 5->d, 6->b
+     *   <li>data fields: 1->a, 2->b, 3->c, 4->d
+     * </ul>
+     *
+     * <p>When we project 1->c, 6->b, 3->a from table fields, the table 
projection is [[0], [4],
+     * [1]], in which 0 is the index of field 1->c, 4 is the index of field 
6->b, 1 is the index of
+     * field 3->a in table fields. We need to create data projection from 
[[0], [4], [1]] as
+     * follows:
+     *
+     * <ul>
+     *   <li>Get field id of each index in table projection from table fields
+     *   <li>Get index of each field above from data fields
+     * </ul>
+     *
+     * <p>The we can create table projection as follows: [[0], [-1], [2]], in 
which 0, -1 and 2 are
+     * the index of fields [1->c, 6->b, 3->a] in data fields. When we project 
column from underlying
+     * data, we need to specify the field index and name. It is difficult to 
assign a proper field
+     * id and name for 6->b in data projection and add it to data fields, and 
we can't use 6->b
+     * directly because the field index of b in underlying is 2. We can remove 
the -1 field index in
+     * data projection, then the result data projection is: [[0], [2]].
+     *
+     * <p>We create {@link RowData} for 1->a, 3->c after projecting them from 
underlying data, then
+     * create {@link ProjectedRowData} with a index mapping and return null 
for 6->b in table
+     * fields.
+     *
+     * @param tableFields the fields of table
+     * @param dataFields the fields of underlying data
+     * @param tableProjection the projection of table
+     * @return the projection of data
+     */
+    public static int[][] createDataProjection(
+            List<DataField> tableFields, List<DataField> dataFields, int[][] 
tableProjection) {
+        List<Integer> dataFieldIdList =
+                
dataFields.stream().map(DataField::id).collect(Collectors.toList());
+        return Arrays.stream(tableProjection)
+                .map(p -> Arrays.copyOf(p, p.length))
+                .peek(
+                        p -> {
+                            int fieldId = tableFields.get(p[0]).id();
+                            p[0] = dataFieldIdList.indexOf(fieldId);
+                        })
+                .filter(p -> p[0] >= 0)
+                .toArray(int[][]::new);
+    }
+
+    /**
+     * Create predicate list from data fields. We will visit all predicate in 
filters, reset it's
+     * field index, name and type, and use {@link AlwaysFalse} or {@link 
AlwaysTrue} if the field is
+     * not exist.
+     *
+     * @param tableFields the table fields
+     * @param dataFields the underlying data fields
+     * @param filters the filters
+     * @return the data filters
+     */
+    @Nullable
+    public static List<Predicate> createDataFilters(
+            List<DataField> tableFields, List<DataField> dataFields, 
List<Predicate> filters) {

Review Comment:
   Maybe we can construct them first like:
   - `Map<String, DataField> tableFields`?
   - `LinkedHashMap<Integer, DataField> dataFields` and we can create a 
`indexOf` for `LinkedHashMap`?



##########
flink-table-store-core/src/test/java/org/apache/flink/table/store/table/ChangelogWithKeyFileMetaFilterTest.java:
##########
@@ -46,7 +46,7 @@ public void before() throws Exception {
     @Test
     @Override
     public void testTableScan() throws Exception {
-        writeAndCheckFileMeta(
+        writeAndCheckFileResult(

Review Comment:
   Why need to override three methods? I think we can add comments to explain 
in the code.



##########
flink-table-store-core/src/test/java/org/apache/flink/table/store/table/FileDataFilterTestBase.java:
##########
@@ -0,0 +1,460 @@
+/*
+ * 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.flink.table.store.table;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.mergetree.compact.ConcatRecordReader;
+import org.apache.flink.table.store.file.predicate.Equal;
+import org.apache.flink.table.store.file.predicate.IsNull;
+import org.apache.flink.table.store.file.predicate.LeafPredicate;
+import org.apache.flink.table.store.file.predicate.Predicate;
+import org.apache.flink.table.store.file.predicate.PredicateBuilder;
+import org.apache.flink.table.store.file.schema.RowDataType;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.store.file.utils.RecordReaderIterator;
+import org.apache.flink.table.store.table.source.Split;
+import org.apache.flink.table.store.table.source.TableRead;
+import org.apache.flink.types.RowKind;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Base test class of file data for schema evolution in {@link 
FileStoreTable}. */
+public abstract class FileDataFilterTestBase extends 
SchemaEvolutionTableTestBase {
+
+    protected static final int[] PROJECTION = new int[] {3, 2, 1};
+
+    protected static final Function<RowData, String> SCHEMA_0_ROW_TO_STRING =
+            rowData ->
+                    getNullOrString(rowData, 0)
+                            + "|"
+                            + getNullOrInt(rowData, 1)
+                            + "|"
+                            + getNullOrInt(rowData, 2)
+                            + "|"
+                            + getNullOrString(rowData, 3)
+                            + "|"
+                            + getNullOrLong(rowData, 4)
+                            + "|"
+                            + getNullOrString(rowData, 5);
+
+    protected static final Function<RowData, String> 
STREAMING_SCHEMA_0_ROW_TO_STRING =
+            rowData ->
+                    (rowData.getRowKind() == RowKind.INSERT ? "+" : "-")
+                            + getNullOrString(rowData, 0)
+                            + "|"
+                            + getNullOrInt(rowData, 1)
+                            + "|"
+                            + getNullOrInt(rowData, 2)
+                            + "|"
+                            + getNullOrString(rowData, 3)
+                            + "|"
+                            + getNullOrLong(rowData, 4)
+                            + "|"
+                            + getNullOrString(rowData, 5);
+
+    protected static final Function<RowData, String> 
SCHEMA_0_PROJECT_ROW_TO_STRING =
+            rowData ->
+                    getNullOrString(rowData, 0)
+                            + "|"
+                            + getNullOrInt(rowData, 1)
+                            + "|"
+                            + getNullOrInt(rowData, 2);
+
+    protected static final Function<RowData, String> 
STREAMING_SCHEMA_0_PROJECT_ROW_TO_STRING =
+            rowData ->
+                    (rowData.getRowKind() == RowKind.INSERT ? "+" : "-")
+                            + getNullOrString(rowData, 0)
+                            + "|"
+                            + getNullOrInt(rowData, 1)
+                            + "|"
+                            + getNullOrInt(rowData, 2);
+
+    protected static final Function<RowData, String> SCHEMA_1_ROW_TO_STRING =
+            rowData ->
+                    getNullOrInt(rowData, 0)
+                            + "|"
+                            + getNullOrInt(rowData, 1)
+                            + "|"
+                            + getNullOrLong(rowData, 2)
+                            + "|"
+                            + getNullOrInt(rowData, 3)
+                            + "|"
+                            + getNullOrString(rowData, 4)
+                            + "|"
+                            + getNullOrString(rowData, 5);
+
+    protected static final Function<RowData, String> 
STREAMING_SCHEMA_1_ROW_TO_STRING =
+            rowData ->
+                    (rowData.getRowKind() == RowKind.INSERT ? "+" : "-")
+                            + getNullOrInt(rowData, 0)
+                            + "|"
+                            + getNullOrInt(rowData, 1)
+                            + "|"
+                            + getNullOrLong(rowData, 2)
+                            + "|"
+                            + getNullOrInt(rowData, 3)
+                            + "|"
+                            + getNullOrString(rowData, 4)
+                            + "|"
+                            + getNullOrString(rowData, 5);
+
+    protected static final Function<RowData, String> 
SCHEMA_1_PROJECT_ROW_TO_STRING =
+            rowData ->
+                    getNullOrInt(rowData, 0)
+                            + "|"
+                            + getNullOrLong(rowData, 1)
+                            + "|"
+                            + getNullOrInt(rowData, 2);
+
+    protected static final Function<RowData, String> 
STREAMING_SCHEMA_1_PROJECT_ROW_TO_STRING =
+            rowData ->
+                    (rowData.getRowKind() == RowKind.INSERT ? "+" : "-")
+                            + getNullOrInt(rowData, 0)
+                            + "|"
+                            + getNullOrLong(rowData, 1)
+                            + "|"
+                            + getNullOrInt(rowData, 2);
+
+    private static String getNullOrInt(RowData rowData, int index) {
+        return rowData.isNullAt(index) ? "null" : 
String.valueOf(rowData.getInt(index));
+    }
+
+    private static String getNullOrLong(RowData rowData, int index) {
+        return rowData.isNullAt(index) ? "null" : 
String.valueOf(rowData.getLong(index));
+    }
+
+    private static String getNullOrString(RowData rowData, int index) {
+        return rowData.isNullAt(index) ? "null" : 
rowData.getString(index).toString();
+    }
+
+    @Test
+    public void testReadFilterExistField() throws Exception {
+        writeAndCheckFileResult(
+                schemas -> {
+                    PredicateBuilder builder =
+                            new PredicateBuilder(RowDataType.toRowType(false, 
SCHEMA_0_FIELDS));
+                    FileStoreTable table = createFileStoreTable(schemas);
+                    List<Split> splits = table.newScan().plan().splits();
+                    // filter with "b" = 15 in schema0
+                    TableRead read = 
table.newRead().withFilter(builder.equal(2, 15));
+
+                    assertThat(getResult(read, splits, SCHEMA_0_ROW_TO_STRING))
+                            .hasSameElementsAs(
+                                    Arrays.asList(
+                                            "S005|2|15|S15|115|S115", 
"S006|2|16|S16|116|S116"));
+                    return null;
+                },
+                (files, schemas) -> {
+                    PredicateBuilder builder =
+                            new PredicateBuilder(RowDataType.toRowType(false, 
SCHEMA_1_FIELDS));
+                    FileStoreTable table = createFileStoreTable(schemas);
+                    List<Split> splits = table.newScan().plan().splits();
+
+                    // filter with "d" = 15 in schema1 which should be mapped 
to "b" = 15 in schema0
+                    TableRead read1 = 
table.newRead().withFilter(builder.equal(1, 15));
+                    assertThat(getResult(read1, splits, 
SCHEMA_1_ROW_TO_STRING))
+                            .hasSameElementsAs(
+                                    Arrays.asList(
+                                            "2|15|115|null|null|null", 
"2|16|116|null|null|null"));
+
+                    // filter with "d" = 21 in schema1
+                    TableRead read2 = 
table.newRead().withFilter(builder.equal(1, 21));
+                    assertThat(getResult(read2, splits, 
SCHEMA_1_ROW_TO_STRING))
+                            .hasSameElementsAs(
+                                    Arrays.asList(
+                                            "1|21|121|1121|S011|S21", 
"1|22|122|1122|S012|S22"));
+                },
+                getPrimaryKeyNames(),
+                tableConfig,
+                this::createFileStoreTable);
+    }
+
+    @Test
+    public void testReadFilterNonExistField() throws Exception {
+        writeAndCheckFileResult(
+                schemas -> null,
+                (files, schemas) -> {
+                    PredicateBuilder builder =
+                            new PredicateBuilder(RowDataType.toRowType(false, 
SCHEMA_1_FIELDS));
+                    FileStoreTable table = createFileStoreTable(schemas);
+                    List<Split> splits = table.newScan().plan().splits();
+
+                    // filter with "a" = 1122 in schema1 which is not exist in 
schema0
+                    TableRead read1 = 
table.newRead().withFilter(builder.equal(3, 1122));
+                    assertThat(getResult(read1, splits, 
SCHEMA_1_ROW_TO_STRING))
+                            .hasSameElementsAs(
+                                    Arrays.asList(
+                                            "2|12|112|null|null|null",
+                                            "2|15|115|null|null|null",
+                                            "2|16|116|null|null|null",
+                                            "1|11|111|null|null|null",
+                                            "1|13|113|null|null|null",
+                                            "1|14|114|null|null|null",
+                                            "1|21|121|1121|S011|S21",
+                                            "1|22|122|1122|S012|S22"));
+
+                    // filter with "a" = 1122 in scan and read
+                    splits = table.newScan().withFilter(builder.equal(3, 
1122)).plan().splits();
+                    TableRead read2 = 
table.newRead().withFilter(builder.equal(3, 1122));
+                    assertThat(getResult(read2, splits, 
SCHEMA_1_ROW_TO_STRING))
+                            .hasSameElementsAs(
+                                    Arrays.asList(
+                                            "1|21|121|1121|S011|S21", 
"1|22|122|1122|S012|S22"));
+                },
+                getPrimaryKeyNames(),
+                tableConfig,
+                this::createFileStoreTable);
+    }
+
+    @Test
+    public void testReadFilterMultipleFields() throws Exception {
+        writeAndCheckFileResult(
+                schemas -> null,
+                (files, schemas) -> {
+                    List<Predicate> predicateList =
+                            Arrays.asList(
+                                    new LeafPredicate(
+                                            Equal.INSTANCE,
+                                            DataTypes.INT().getLogicalType(),
+                                            1,
+                                            "d",
+                                            Arrays.asList(21)),

Review Comment:
   `Collections.singletonList`



##########
flink-table-store-core/src/test/java/org/apache/flink/table/store/table/SchemaEvolutionTableTestBase.java:
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.flink.table.store.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.file.schema.AtomicDataType;
+import org.apache.flink.table.store.file.schema.DataField;
+import org.apache.flink.table.store.file.schema.SchemaChange;
+import org.apache.flink.table.store.file.schema.SchemaManager;
+import org.apache.flink.table.store.file.schema.TableSchema;
+import org.apache.flink.table.store.file.schema.UpdateSchema;
+import org.apache.flink.table.store.file.utils.TestAtomicRenameFileSystem;
+import org.apache.flink.table.store.file.utils.TraceableFileSystem;
+import org.apache.flink.table.store.table.sink.TableCommit;
+import org.apache.flink.table.store.table.sink.TableWrite;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Base test class for schema evolution in {@link FileStoreTable}. */
+public abstract class SchemaEvolutionTableTestBase {
+    protected static final List<DataField> SCHEMA_0_FIELDS =
+            Arrays.asList(
+                    new DataField(0, "a", new 
AtomicDataType(DataTypes.STRING().getLogicalType())),
+                    new DataField(1, "pt", new 
AtomicDataType(DataTypes.INT().getLogicalType())),
+                    new DataField(2, "b", new 
AtomicDataType(DataTypes.INT().getLogicalType())),
+                    new DataField(3, "c", new 
AtomicDataType(DataTypes.STRING().getLogicalType())),
+                    new DataField(4, "kt", new 
AtomicDataType(DataTypes.BIGINT().getLogicalType())),
+                    new DataField(5, "d", new 
AtomicDataType(DataTypes.STRING().getLogicalType())));
+    protected static final List<DataField> SCHEMA_1_FIELDS =
+            Arrays.asList(
+                    new DataField(1, "pt", new 
AtomicDataType(DataTypes.INT().getLogicalType())),
+                    new DataField(2, "d", new 
AtomicDataType(DataTypes.INT().getLogicalType())),
+                    new DataField(4, "kt", new 
AtomicDataType(DataTypes.BIGINT().getLogicalType())),
+                    new DataField(6, "a", new 
AtomicDataType(DataTypes.INT().getLogicalType())),
+                    new DataField(7, "f", new 
AtomicDataType(DataTypes.STRING().getLogicalType())),
+                    new DataField(8, "b", new 
AtomicDataType(DataTypes.STRING().getLogicalType())));
+    protected static final List<String> PARTITION_NAMES = 
Collections.singletonList("pt");
+    protected static final List<String> PRIMARY_KEY_NAMES = 
Arrays.asList("pt", "kt");
+
+    protected Path tablePath;
+    protected String commitUser;
+    protected final Configuration tableConfig = new Configuration();
+
+    @TempDir java.nio.file.Path tempDir;
+
+    @BeforeEach
+    public void before() throws Exception {
+        tablePath = new Path(TestAtomicRenameFileSystem.SCHEME + "://" + 
tempDir.toString());
+        commitUser = UUID.randomUUID().toString();
+        tableConfig.set(CoreOptions.PATH, tablePath.toString());
+        tableConfig.set(CoreOptions.BUCKET, 2);
+    }
+
+    @AfterEach
+    public void after() throws IOException {
+        // assert all connections are closed
+        FileSystem fileSystem = tablePath.getFileSystem();
+        assertThat(fileSystem).isInstanceOf(TraceableFileSystem.class);
+        TraceableFileSystem traceableFileSystem = (TraceableFileSystem) 
fileSystem;
+
+        java.util.function.Predicate<Path> pathPredicate =
+                path -> path.toString().contains(tempDir.toString());
+        
assertThat(traceableFileSystem.openInputStreams(pathPredicate)).isEmpty();
+        
assertThat(traceableFileSystem.openOutputStreams(pathPredicate)).isEmpty();
+    }
+
+    protected List<String> getPrimaryKeyNames() {
+        return PRIMARY_KEY_NAMES;
+    }
+
+    protected abstract FileStoreTable createFileStoreTable(Map<Long, 
TableSchema> tableSchemas);
+
+    public static <R> void writeAndCheckFileResult(
+            Function<Map<Long, TableSchema>, R> firstChecker,
+            BiConsumer<R, Map<Long, TableSchema>> secondChecker,
+            List<String> primaryKeyNames,
+            Configuration tableConfig,
+            Function<Map<Long, TableSchema>, FileStoreTable> 
createFileStoreTable)
+            throws Exception {
+        Map<Long, TableSchema> tableSchemas = new HashMap<>();
+        tableSchemas.put(
+                0L,
+                new TableSchema(
+                        0,
+                        SCHEMA_0_FIELDS,
+                        5,
+                        PARTITION_NAMES,
+                        primaryKeyNames,
+                        tableConfig.toMap(),
+                        ""));
+        FileStoreTable table = createFileStoreTable.apply(tableSchemas);
+        TableWrite write = table.newWrite("user");
+        TableCommit commit = table.newCommit("user");
+
+        write.write(
+                GenericRowData.of(
+                        StringData.fromString("S001"),
+                        1,
+                        11,
+                        StringData.fromString("S11"),
+                        111L,
+                        StringData.fromString("S111")));
+        write.write(
+                GenericRowData.of(
+                        StringData.fromString("S002"),
+                        2,
+                        12,
+                        StringData.fromString("S12"),
+                        112L,
+                        StringData.fromString("S112")));
+        write.write(
+                GenericRowData.of(
+                        StringData.fromString("S003"),
+                        1,
+                        13,
+                        StringData.fromString("S13"),
+                        113L,
+                        StringData.fromString("S113")));
+        commit.commit(0, write.prepareCommit(true, 0));
+
+        write.write(
+                GenericRowData.of(
+                        StringData.fromString("S004"),
+                        1,
+                        14,
+                        StringData.fromString("S14"),
+                        114L,
+                        StringData.fromString("S114")));
+        write.write(
+                GenericRowData.of(
+                        StringData.fromString("S005"),
+                        2,
+                        15,
+                        StringData.fromString("S15"),
+                        115L,
+                        StringData.fromString("S115")));
+        write.write(
+                GenericRowData.of(
+                        StringData.fromString("S006"),
+                        2,
+                        16,
+                        StringData.fromString("S16"),
+                        116L,
+                        StringData.fromString("S116")));
+        commit.commit(0, write.prepareCommit(true, 0));
+        write.close();
+        R result = firstChecker.apply(tableSchemas);
+
+        tableSchemas.put(

Review Comment:
   Add comments to explain fields removing and fields adding, this can help to 
understand the test code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to