openinx commented on code in PR #115:
URL: https://github.com/apache/flink-table-store/pull/115#discussion_r869077944


##########
flink-table-store-core/src/test/java/org/apache/flink/table/store/file/writer/TestAppendOnlyWriter.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.writer;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.binary.BinaryRowDataUtil;
+import org.apache.flink.table.store.file.ValueKind;
+import org.apache.flink.table.store.file.data.DataFileMeta;
+import org.apache.flink.table.store.file.data.DataFilePathFactory;
+import org.apache.flink.table.store.file.format.FileFormat;
+import org.apache.flink.table.store.file.mergetree.Increment;
+import org.apache.flink.table.store.file.stats.FieldStats;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.util.Collections;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test the correctness for {@link AppendOnlyWriter}. */
+public class TestAppendOnlyWriter {

Review Comment:
   Thanks for the remaining,  let me rename this class name.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/writer/AppendOnlyWriter.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.writer;
+
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.ValueKind;
+import org.apache.flink.table.store.file.data.DataFileMeta;
+import org.apache.flink.table.store.file.data.DataFilePathFactory;
+import org.apache.flink.table.store.file.format.FileFormat;
+import org.apache.flink.table.store.file.mergetree.Increment;
+import org.apache.flink.table.store.file.stats.FieldStats;
+import org.apache.flink.table.store.file.stats.FieldStatsCollector;
+import org.apache.flink.table.store.file.stats.FileStatsExtractor;
+import org.apache.flink.table.store.file.utils.FileUtils;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Supplier;
+
+/**
+ * A {@link RecordWriter} implementation that only accepts records which are 
always insert
+ * operations and don't have any unique keys or sort keys.
+ */
+public class AppendOnlyWriter implements RecordWriter {
+    private final BulkWriter.Factory<RowData> writerFactory;
+    private final RowType writeSchema;
+    private final long targetFileSize;
+    private final DataFilePathFactory pathFactory;
+    private final FileStatsExtractor fileStatsExtractor;
+    private final AtomicLong nextSeqNum;
+
+    private RowRollingWriter writer;
+
+    public AppendOnlyWriter(
+            FileFormat fileFormat,
+            long targetFileSize,
+            RowType writeSchema,
+            long maxWroteSeqNumber,
+            DataFilePathFactory pathFactory) {
+
+        this.writerFactory = fileFormat.createWriterFactory(writeSchema);
+        this.writeSchema = writeSchema;
+        this.targetFileSize = targetFileSize;
+        this.pathFactory = pathFactory;
+        this.fileStatsExtractor = 
fileFormat.createStatsExtractor(writeSchema).orElse(null);
+        this.nextSeqNum = new AtomicLong(maxWroteSeqNumber + 1);
+
+        this.writer = createRollingRowWriter();
+    }
+
+    @Override
+    public void write(ValueKind valueKind, RowData key, RowData value) throws 
Exception {
+        Preconditions.checkArgument(
+                valueKind == ValueKind.ADD,
+                "Append-only writer cannot accept ValueKind: " + valueKind);
+
+        writer.write(value);
+    }
+
+    @Override
+    public Increment prepareCommit() throws Exception {
+        List<DataFileMeta> newFiles = Lists.newArrayList();
+
+        if (writer != null) {
+            writer.close();
+            newFiles.addAll(writer.result());
+
+            // Reopen the writer to accept further records.
+            writer = createRollingRowWriter();
+        }
+
+        return new Increment(Lists.newArrayList(newFiles));
+    }
+
+    @Override
+    public void sync() throws Exception {
+        // Do nothing here, as this writer don't introduce any async 
compaction thread currently.
+    }
+
+    @Override
+    public List<DataFileMeta> close() throws Exception {
+        sync();
+
+        List<DataFileMeta> result = Lists.newArrayList();

Review Comment:
   Okay, let's follow the coding rule.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/writer/AppendOnlyWriter.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.writer;
+
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.file.ValueKind;
+import org.apache.flink.table.store.file.data.DataFileMeta;
+import org.apache.flink.table.store.file.data.DataFilePathFactory;
+import org.apache.flink.table.store.file.format.FileFormat;
+import org.apache.flink.table.store.file.mergetree.Increment;
+import org.apache.flink.table.store.file.stats.FieldStats;
+import org.apache.flink.table.store.file.stats.FieldStatsCollector;
+import org.apache.flink.table.store.file.stats.FileStatsExtractor;
+import org.apache.flink.table.store.file.utils.FileUtils;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Supplier;
+
+/**
+ * A {@link RecordWriter} implementation that only accepts records which are 
always insert
+ * operations and don't have any unique keys or sort keys.
+ */
+public class AppendOnlyWriter implements RecordWriter {
+    private final BulkWriter.Factory<RowData> writerFactory;
+    private final RowType writeSchema;
+    private final long targetFileSize;
+    private final DataFilePathFactory pathFactory;
+    private final FileStatsExtractor fileStatsExtractor;
+    private final AtomicLong nextSeqNum;
+
+    private RowRollingWriter writer;
+
+    public AppendOnlyWriter(
+            FileFormat fileFormat,
+            long targetFileSize,
+            RowType writeSchema,
+            long maxWroteSeqNumber,
+            DataFilePathFactory pathFactory) {
+
+        this.writerFactory = fileFormat.createWriterFactory(writeSchema);
+        this.writeSchema = writeSchema;
+        this.targetFileSize = targetFileSize;
+        this.pathFactory = pathFactory;
+        this.fileStatsExtractor = 
fileFormat.createStatsExtractor(writeSchema).orElse(null);
+        this.nextSeqNum = new AtomicLong(maxWroteSeqNumber + 1);
+
+        this.writer = createRollingRowWriter();
+    }
+
+    @Override
+    public void write(ValueKind valueKind, RowData key, RowData value) throws 
Exception {
+        Preconditions.checkArgument(
+                valueKind == ValueKind.ADD,
+                "Append-only writer cannot accept ValueKind: " + valueKind);
+
+        writer.write(value);
+    }
+
+    @Override
+    public Increment prepareCommit() throws Exception {
+        List<DataFileMeta> newFiles = Lists.newArrayList();
+
+        if (writer != null) {
+            writer.close();
+            newFiles.addAll(writer.result());
+
+            // Reopen the writer to accept further records.
+            writer = createRollingRowWriter();

Review Comment:
   Thanks for the careful reviewing.  In fact, here we are creating the 
`RollingFileWriter` which will initializing the `DataFileWriter` lazily ( The 
`DataFileWriter` is the real writer that will try to open file descriptor to 
write records ). 
   
   Please check this lines: 
https://github.com/apache/flink-table-store/blob/master/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/writer/RollingFileWriter.java#L58-L69
   
   So in theory,  we won't add any extra empty `DataFileMeta` into the table 
format, even if we open an `AppendOnlyWriter` and call `prepareCommit` without 
writing any record.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/data/DataFileMeta.java:
##########
@@ -49,6 +57,26 @@ public class DataFileMeta {
     private final long maxSequenceNumber;
     private final int level;
 
+    public DataFileMeta(

Review Comment:
   Okay, that seems to be more clear.



-- 
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-unsubscr...@flink.apache.org

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

Reply via email to