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:28:43 UTC

[GitHub] [flink-table-store] tsreaper commented on a diff in pull request #99: [FLINK-27307] Flink table store support append-only ingestion without primary keys.

tsreaper commented on code in PR #99:
URL: https://github.com/apache/flink-table-store/pull/99#discussion_r867696446


##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/writer/AppendOnlyWriter.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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 append-only records. */
+public class AppendOnlyWriter implements RecordWriter {
+    private final BulkWriter.Factory<RowData> writerFactory;
+    private final RowType writeSchema;
+    private final long targetFileSize;
+    private final List<DataFileMeta> existingFiles;
+    private final DataFilePathFactory pathFactory;
+    private final FileStatsExtractor fileStatsExtractor;
+
+    private final AtomicLong nextSeqNum;
+
+    private RowRollingWriter writer;
+
+    public AppendOnlyWriter(
+            FileFormat fileFormat,
+            long targetFileSize,
+            RowType writeSchema,
+            List<DataFileMeta> existingFiles,
+            DataFilePathFactory pathFactory,
+            FileStatsExtractor fileStatsExtractor) {
+
+        this.writerFactory = fileFormat.createWriterFactory(writeSchema);
+        this.writeSchema = writeSchema;
+        this.targetFileSize = targetFileSize;
+
+        this.existingFiles = existingFiles;
+        this.pathFactory = pathFactory;
+        this.fileStatsExtractor = fileStatsExtractor;
+
+        this.nextSeqNum = new AtomicLong(maxSequenceNumber() + 1);
+
+        this.writer = createRollingRowWriter();
+    }
+
+    private long maxSequenceNumber() {
+        return existingFiles.stream()
+                .map(DataFileMeta::maxSequenceNumber)
+                .max(Long::compare)
+                .orElse(-1L);
+    }
+
+    @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 {
+        // TODO add the flush method for FileWriter.

Review Comment:
   Is this a must? Before each checkpoint and before the end of input `prepareCommit` will be called and in that method `writer.close()` is called. `writer.close()` will flush the writer.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/writer/AppendOnlyWriter.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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 append-only records. */
+public class AppendOnlyWriter implements RecordWriter {
+    private final BulkWriter.Factory<RowData> writerFactory;
+    private final RowType writeSchema;
+    private final long targetFileSize;
+    private final List<DataFileMeta> existingFiles;
+    private final DataFilePathFactory pathFactory;
+    private final FileStatsExtractor fileStatsExtractor;
+
+    private final AtomicLong nextSeqNum;
+
+    private RowRollingWriter writer;
+
+    public AppendOnlyWriter(
+            FileFormat fileFormat,
+            long targetFileSize,
+            RowType writeSchema,
+            List<DataFileMeta> existingFiles,
+            DataFilePathFactory pathFactory,
+            FileStatsExtractor fileStatsExtractor) {
+
+        this.writerFactory = fileFormat.createWriterFactory(writeSchema);
+        this.writeSchema = writeSchema;
+        this.targetFileSize = targetFileSize;
+
+        this.existingFiles = existingFiles;
+        this.pathFactory = pathFactory;
+        this.fileStatsExtractor = fileStatsExtractor;
+
+        this.nextSeqNum = new AtomicLong(maxSequenceNumber() + 1);
+
+        this.writer = createRollingRowWriter();
+    }
+
+    private long maxSequenceNumber() {
+        return existingFiles.stream()
+                .map(DataFileMeta::maxSequenceNumber)
+                .max(Long::compare)
+                .orElse(-1L);
+    }
+
+    @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 {
+        // TODO add the flush method for FileWriter.
+    }
+
+    @Override
+    public List<DataFileMeta> close() throws Exception {
+        List<DataFileMeta> result = Lists.newArrayList();
+        if (writer != null) {
+            writer.close();
+
+            result.addAll(writer.result());
+            writer = null;
+        }
+
+        return result;

Review Comment:
   Everything in the `result` are not committed and thus should be deleted. The return value of this method is the list of deleted files. See `MergeTreeWriter#close`.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/data/DataFileMeta.java:
##########
@@ -36,6 +37,14 @@
 /** Metadata of a data file. */
 public class DataFileMeta {
 
+    // Append only data files don't have any value columns and meaningful level value. it will use

Review Comment:
   `don't have any value columns` -> `don't have any key columns`?



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/FileStoreWriteImpl.java:
##########
@@ -34,30 +36,38 @@
 import org.apache.flink.table.store.file.mergetree.compact.CompactStrategy;
 import org.apache.flink.table.store.file.mergetree.compact.MergeFunction;
 import org.apache.flink.table.store.file.mergetree.compact.UniversalCompaction;
+import org.apache.flink.table.store.file.stats.FileStatsExtractor;
 import org.apache.flink.table.store.file.utils.FileStorePathFactory;
 import org.apache.flink.table.store.file.utils.RecordReaderIterator;
+import org.apache.flink.table.store.file.writer.AppendOnlyWriter;
 import org.apache.flink.table.store.file.writer.RecordWriter;
 import org.apache.flink.table.types.logical.RowType;
 
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
+
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
 import java.util.function.Supplier;
-import java.util.stream.Collectors;
 
 /** Default implementation of {@link FileStoreWrite}. */
 public class FileStoreWriteImpl implements FileStoreWrite {
 
     private final DataFileReader.Factory dataFileReaderFactory;
     private final DataFileWriter.Factory dataFileWriterFactory;
+    private final WriteMode writeMode;
+    private final FileFormat fileFormat;
+    private final RowType writeSchema;
     private final Supplier<Comparator<RowData>> keyComparatorSupplier;
     private final MergeFunction mergeFunction;
     private final FileStorePathFactory pathFactory;
     private final FileStoreScan scan;
     private final MergeTreeOptions options;
 
     public FileStoreWriteImpl(
+            WriteMode writeMode,
+            RowType writeSchema,
             RowType keyType,
             RowType valueType,

Review Comment:
   What is this `writeSchema` for? We already have a `valueType`.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/writer/AppendOnlyWriter.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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 append-only records. */
+public class AppendOnlyWriter implements RecordWriter {
+    private final BulkWriter.Factory<RowData> writerFactory;
+    private final RowType writeSchema;
+    private final long targetFileSize;
+    private final List<DataFileMeta> existingFiles;
+    private final DataFilePathFactory pathFactory;
+    private final FileStatsExtractor fileStatsExtractor;
+
+    private final AtomicLong nextSeqNum;
+
+    private RowRollingWriter writer;
+
+    public AppendOnlyWriter(
+            FileFormat fileFormat,
+            long targetFileSize,
+            RowType writeSchema,
+            List<DataFileMeta> existingFiles,
+            DataFilePathFactory pathFactory,
+            FileStatsExtractor fileStatsExtractor) {
+
+        this.writerFactory = fileFormat.createWriterFactory(writeSchema);
+        this.writeSchema = writeSchema;
+        this.targetFileSize = targetFileSize;
+
+        this.existingFiles = existingFiles;
+        this.pathFactory = pathFactory;
+        this.fileStatsExtractor = fileStatsExtractor;
+
+        this.nextSeqNum = new AtomicLong(maxSequenceNumber() + 1);

Review Comment:
   Why `AtomicLong`? Each writer instance is used by only one thread.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/FileStoreWriteImpl.java:
##########
@@ -82,18 +95,41 @@ public FileStoreWriteImpl(
     public RecordWriter createWriter(
             BinaryRowData partition, int bucket, ExecutorService compactExecutor) {
         Long latestSnapshotId = pathFactory.latestSnapshotId();
-        if (latestSnapshotId == null) {
-            return createEmptyWriter(partition, bucket, compactExecutor);
-        } else {
-            return createMergeTreeWriter(
-                    partition,
-                    bucket,
-                    scan.withSnapshot(latestSnapshotId)
-                            .withPartitionFilter(Collections.singletonList(partition))
-                            .withBucket(bucket).plan().files().stream()
-                            .map(ManifestEntry::file)
-                            .collect(Collectors.toList()),
-                    compactExecutor);
+        List<DataFileMeta> existingFileMetas = Lists.newArrayList();
+        if (latestSnapshotId != null) {
+            // Concat all the DataFileMeta of existing files into existingFileMetas.
+            scan.withSnapshot(latestSnapshotId)
+                    .withPartitionFilter(Collections.singletonList(partition)).withBucket(bucket)
+                    .plan().files().stream()
+                    .map(ManifestEntry::file)
+                    .forEach(existingFileMetas::add);
+        }
+
+        switch (writeMode) {
+            case APPEND_ONLY:
+                DataFilePathFactory factory =
+                        pathFactory.createDataFilePathFactory(partition, bucket);
+                FileStatsExtractor fileStatsExtractor =
+                        fileFormat.createStatsExtractor(writeSchema).orElse(null);
+
+                return new AppendOnlyWriter(
+                        fileFormat,
+                        options.targetFileSize,
+                        writeSchema,
+                        existingFileMetas,
+                        factory,
+                        fileStatsExtractor);
+
+            case CHANGE_LOG:
+                if (latestSnapshotId == null) {
+                    return createEmptyWriter(partition, bucket, compactExecutor);
+                } else {
+                    return createMergeTreeWriter(
+                            partition, bucket, existingFileMetas, compactExecutor);
+                }

Review Comment:
   As we've calculated `existingFileMetas` we don't need a special `createEmptyWriter` method. Remove this method and this if branching.



##########
flink-table-store-core/src/main/java/org/apache/flink/table/store/file/operation/FileStoreReadImpl.java:
##########
@@ -83,6 +89,34 @@ public FileStoreRead withValueProjection(int[][] projectedFields) {
     @Override
     public RecordReader createReader(BinaryRowData partition, int bucket, List<DataFileMeta> files)
             throws IOException {
+        switch (writeMode) {
+            case APPEND_ONLY:
+                return createAppendOnlyReader(partition, bucket, files);
+
+            case CHANGE_LOG:
+                return createLSMReader(partition, bucket, files);
+
+            default:
+                throw new UnsupportedOperationException("Unknown write mode: " + writeMode);
+        }
+    }
+
+    private RecordReader createAppendOnlyReader(
+            BinaryRowData partition, int bucket, List<DataFileMeta> files) throws IOException {
+        Preconditions.checkArgument(
+                !dropDelete, "Cannot drop delete message for append-only table.");

Review Comment:
   Does this check make sense? There is no delete messages in an append only table so I guess the value of `dropDelete` does not matter.



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