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/05/09 07:41:44 UTC

[GitHub] [flink-table-store] openinx commented on a diff in pull request #115: [FLINK-27546] Add append only writer which implements the RecordWriter interface

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


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/writer/AppendOnlyWriter.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * 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();
+
+            // Reopen the writer to accept further records.
+            newFiles.addAll(writer.result());
+            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();
+        if (writer != null) {
+            // Abort this writer to clear uncommitted files.
+            writer.abort();

Review Comment:
   I'm just curious that the `RecordWriter#close` will close the writer and **delete the uncommitted files** automatically,  which is a bit mismatching the normal `close` semantic of writer.  [1]
   
   Actually, the behavior is more matching the `abort` semantic. So I filed a separate issue to address this issue. I mean we can define separate `abort()` and `close()` methods in this interface,  and the `close()` will just close the opening writer, and the `abort()` will clean the uncommitted files. I think its semantic is more clear.
   
   [1] https://github.com/apache/flink-table-store/blob/1fedb6586ba292bedf00bb1b6922d5110a2f7fa8/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/writer/RecordWriter.java#L51



-- 
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