You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/06/08 11:43:59 UTC

[GitHub] [flink-table-store] tsreaper opened a new pull request, #148: [FLINK-27957] Extract AppendOnlyFileStore out of KeyValueFileStore

tsreaper opened a new pull request, #148:
URL: https://github.com/apache/flink-table-store/pull/148

   Currently `FileStore` for append only records and key-values are mixed in one `FileStoreImpl` class. This makes the code base messy and also introduce bugs (for example, `AppendOnlyFileStore` should rely on a special reader implementation but it is not, causing failures when using avro format).
   
   We need to extract `AppendOnlyFileStore` out of `KeyValueFileStore`.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-table-store] JingsongLi commented on a diff in pull request #148: [FLINK-27957] Extract AppendOnlyFileStore out of KeyValueFileStore

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on code in PR #148:
URL: https://github.com/apache/flink-table-store/pull/148#discussion_r897602947


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/data/AppendOnlyReader.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.data;
+
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.RecordAndPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.store.file.format.FileFormat;
+import org.apache.flink.table.store.file.schema.SchemaManager;
+import org.apache.flink.table.store.file.utils.FileStorePathFactory;
+import org.apache.flink.table.store.file.utils.FileUtils;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.types.logical.RowType;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+/** Reads {@link RowData} from data files. */
+public class AppendOnlyReader {
+
+    private final SchemaManager schemaManager;
+    private final long schemaId;
+
+    // TODO introduce Map<SchemaId, readerFactory>
+    private final BulkFormat<RowData, FileSourceSplit> readerFactory;
+    private final DataFilePathFactory pathFactory;
+
+    private AppendOnlyReader(
+            SchemaManager schemaManager,
+            long schemaId,
+            BulkFormat<RowData, FileSourceSplit> readerFactory,
+            DataFilePathFactory pathFactory) {
+        this.schemaManager = schemaManager;
+        this.schemaId = schemaId;
+        this.readerFactory = readerFactory;
+        this.pathFactory = pathFactory;
+    }
+
+    public RecordReader<RowData> read(String fileName) throws IOException {
+        System.out.println("read " + fileName);

Review Comment:
   Remove this



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/AbstractFileStoreScan.java:
##########
@@ -249,22 +229,14 @@ private boolean filterManifestFileMeta(ManifestFileMeta manifest) {
                         manifest.partitionStats().fields(partitionStatsConverter));
     }
 
-    private boolean filterManifestEntry(ManifestEntry entry) {
+    protected boolean filterManifestEntry(ManifestEntry entry) {

Review Comment:
   I think a `filterByPartitionBucket` here. And add a abstract class `filterByStats` is better.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-table-store] tsreaper commented on a diff in pull request #148: [FLINK-27957] Extract AppendOnlyFileStore out of KeyValueFileStore

Posted by GitBox <gi...@apache.org>.
tsreaper commented on code in PR #148:
URL: https://github.com/apache/flink-table-store/pull/148#discussion_r897709976


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/table/AppendOnlyFileStoreTable.java:
##########
@@ -62,7 +60,7 @@ public TableScan newScan(boolean incremental) {
         return new TableScan(scan, schema, store.pathFactory()) {
             @Override
             protected void withNonPartitionFilter(Predicate predicate) {
-                scan.withValueFilter(predicate);
+                ((AppendOnlyFileStoreScan) scan).withFilter(predicate);

Review Comment:
   We can't remove this casting even if `AppendOnlyFileStore` returns `AppendOnlyFileStoreScan`. Because in `TableScan` class, `scan` is marked as a `FileTableScan` type. We have to use casting to change it into a subclass.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-table-store] JingsongLi commented on a diff in pull request #148: [FLINK-27957] Extract AppendOnlyFileStore out of KeyValueFileStore

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on code in PR #148:
URL: https://github.com/apache/flink-table-store/pull/148#discussion_r897602817


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/data/AppendOnlyReader.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.data;
+
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.RecordAndPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.store.file.format.FileFormat;
+import org.apache.flink.table.store.file.schema.SchemaManager;
+import org.apache.flink.table.store.file.utils.FileStorePathFactory;
+import org.apache.flink.table.store.file.utils.FileUtils;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.types.logical.RowType;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+/** Reads {@link RowData} from data files. */
+public class AppendOnlyReader {

Review Comment:
   This should be a `RecordReader`?
   We already have a `AppendOnlyFileStoreRead`, it is the factory of `RecordReader`.
   So this class should just be a `RecordReader`.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-table-store] JingsongLi merged pull request #148: [FLINK-27957] Extract AppendOnlyFileStore out of KeyValueFileStore

Posted by GitBox <gi...@apache.org>.
JingsongLi merged PR #148:
URL: https://github.com/apache/flink-table-store/pull/148


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-table-store] JingsongLi commented on a diff in pull request #148: [FLINK-27957] Extract AppendOnlyFileStore out of KeyValueFileStore

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on code in PR #148:
URL: https://github.com/apache/flink-table-store/pull/148#discussion_r897752011


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/table/AppendOnlyFileStoreTable.java:
##########
@@ -62,7 +60,7 @@ public TableScan newScan(boolean incremental) {
         return new TableScan(scan, schema, store.pathFactory()) {
             @Override
             protected void withNonPartitionFilter(Predicate predicate) {
-                scan.withValueFilter(predicate);
+                ((AppendOnlyFileStoreScan) scan).withFilter(predicate);

Review Comment:
   ```
   @Override
       public TableScan newScan() {
           AppendOnlyFileStoreScan scan = store.newScan();
           return new TableScan(scan, schema, store.pathFactory()) {
               @Override
               protected void withNonPartitionFilter(Predicate predicate) {
                   scan.withFilter(predicate);
               }
           };
       }
   ```



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-table-store] JingsongLi commented on a diff in pull request #148: [FLINK-27957] Extract AppendOnlyFileStore out of KeyValueFileStore

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on code in PR #148:
URL: https://github.com/apache/flink-table-store/pull/148#discussion_r892269567


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/table/AppendOnlyFileStoreTable.java:
##########
@@ -62,7 +60,7 @@ public TableScan newScan(boolean incremental) {
         return new TableScan(scan, schema, store.pathFactory()) {
             @Override
             protected void withNonPartitionFilter(Predicate predicate) {
-                scan.withValueFilter(predicate);
+                ((AppendOnlyFileStoreScan) scan).withFilter(predicate);

Review Comment:
   You can remove these cast if `AppendOnlyFileStore.newScan()` returns `AppendOnlyFileStoreScan`.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-table-store] JingsongLi commented on a diff in pull request #148: [FLINK-27957] Extract AppendOnlyFileStore out of KeyValueFileStore

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on code in PR #148:
URL: https://github.com/apache/flink-table-store/pull/148#discussion_r897599930


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/KeyValueFileStore.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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;
+
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.mergetree.compact.MergeFunction;
+import org.apache.flink.table.store.file.operation.KeyValueFileStoreRead;
+import org.apache.flink.table.store.file.operation.KeyValueFileStoreScan;
+import org.apache.flink.table.store.file.operation.KeyValueFileStoreWrite;
+import org.apache.flink.table.store.file.schema.SchemaManager;
+import org.apache.flink.table.store.file.utils.KeyComparatorSupplier;
+import org.apache.flink.table.types.logical.RowType;
+
+import javax.annotation.Nullable;
+
+import java.util.Comparator;
+import java.util.function.Supplier;
+
+/** {@link FileStore} for querying and updating {@link KeyValue}s. */
+public class KeyValueFileStore extends AbstractFileStore<KeyValue> {
+
+    private final RowType keyType;
+    private final RowType valueType;
+    private final Supplier<Comparator<RowData>> keyComparatorSupplier;
+    @Nullable private final MergeFunction mergeFunction;

Review Comment:
   Is this `@Nullable`?



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/data/AppendOnlyReader.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.data;
+
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.RecordAndPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.store.file.format.FileFormat;
+import org.apache.flink.table.store.file.schema.SchemaManager;
+import org.apache.flink.table.store.file.utils.FileStorePathFactory;
+import org.apache.flink.table.store.file.utils.FileUtils;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.types.logical.RowType;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+/** Reads {@link RowData} from data files. */
+public class AppendOnlyReader {
+
+    private final SchemaManager schemaManager;
+    private final long schemaId;
+
+    // TODO introduce Map<SchemaId, readerFactory>
+    private final BulkFormat<RowData, FileSourceSplit> readerFactory;
+    private final DataFilePathFactory pathFactory;
+
+    private AppendOnlyReader(
+            SchemaManager schemaManager,
+            long schemaId,
+            BulkFormat<RowData, FileSourceSplit> readerFactory,
+            DataFilePathFactory pathFactory) {
+        this.schemaManager = schemaManager;
+        this.schemaId = schemaId;
+        this.readerFactory = readerFactory;
+        this.pathFactory = pathFactory;
+    }
+
+    public RecordReader<RowData> read(String fileName) throws IOException {
+        System.out.println("read " + fileName);

Review Comment:
   Remove this



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/table/AppendOnlyFileStoreTable.java:
##########
@@ -62,7 +60,7 @@ public TableScan newScan(boolean incremental) {
         return new TableScan(scan, schema, store.pathFactory()) {
             @Override
             protected void withNonPartitionFilter(Predicate predicate) {
-                scan.withValueFilter(predicate);
+                ((AppendOnlyFileStoreScan) scan).withFilter(predicate);

Review Comment:
   Any update?



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/data/AppendOnlyReader.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.data;
+
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.RecordAndPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.store.file.format.FileFormat;
+import org.apache.flink.table.store.file.schema.SchemaManager;
+import org.apache.flink.table.store.file.utils.FileStorePathFactory;
+import org.apache.flink.table.store.file.utils.FileUtils;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.types.logical.RowType;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+/** Reads {@link RowData} from data files. */
+public class AppendOnlyReader {

Review Comment:
   This should be a `RecordReader`?
   We already have a `AppendOnlyFileStoreRead`, it is the factory of `RecordReader`.
   So this class should just be a `RecordReader`.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/data/AppendOnlyReader.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.data;
+
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.RecordAndPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.store.file.format.FileFormat;
+import org.apache.flink.table.store.file.schema.SchemaManager;
+import org.apache.flink.table.store.file.utils.FileStorePathFactory;
+import org.apache.flink.table.store.file.utils.FileUtils;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.types.logical.RowType;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+/** Reads {@link RowData} from data files. */
+public class AppendOnlyReader {
+
+    private final SchemaManager schemaManager;
+    private final long schemaId;
+
+    // TODO introduce Map<SchemaId, readerFactory>
+    private final BulkFormat<RowData, FileSourceSplit> readerFactory;
+    private final DataFilePathFactory pathFactory;
+
+    private AppendOnlyReader(
+            SchemaManager schemaManager,
+            long schemaId,
+            BulkFormat<RowData, FileSourceSplit> readerFactory,
+            DataFilePathFactory pathFactory) {
+        this.schemaManager = schemaManager;
+        this.schemaId = schemaId;
+        this.readerFactory = readerFactory;
+        this.pathFactory = pathFactory;
+    }
+
+    public RecordReader<RowData> read(String fileName) throws IOException {
+        System.out.println("read " + fileName);

Review Comment:
   Remove this



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/AbstractFileStoreScan.java:
##########
@@ -249,22 +229,14 @@ private boolean filterManifestFileMeta(ManifestFileMeta manifest) {
                         manifest.partitionStats().fields(partitionStatsConverter));
     }
 
-    private boolean filterManifestEntry(ManifestEntry entry) {
+    protected boolean filterManifestEntry(ManifestEntry entry) {

Review Comment:
   I think a `filterByPartitionBucket` here. And add an abstract class `filterByStats` is better.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/AbstractFileStoreScan.java:
##########
@@ -249,22 +229,14 @@ private boolean filterManifestFileMeta(ManifestFileMeta manifest) {
                         manifest.partitionStats().fields(partitionStatsConverter));
     }
 
-    private boolean filterManifestEntry(ManifestEntry entry) {
+    protected boolean filterManifestEntry(ManifestEntry entry) {

Review Comment:
   I think a `filterByPartitionBucket` here. And add a abstract class `filterByStats` is better.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/data/AppendOnlyReader.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.data;
+
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.RecordAndPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.store.file.format.FileFormat;
+import org.apache.flink.table.store.file.schema.SchemaManager;
+import org.apache.flink.table.store.file.utils.FileStorePathFactory;
+import org.apache.flink.table.store.file.utils.FileUtils;
+import org.apache.flink.table.store.file.utils.RecordReader;
+import org.apache.flink.table.types.logical.RowType;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+
+/** Reads {@link RowData} from data files. */
+public class AppendOnlyReader {

Review Comment:
   This should be a `RecordReader`?
   We already have a `AppendOnlyFileStoreRead`, it is the factory of `RecordReader`.
   So this class should just be a `RecordReader`.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org