JingsongLi commented on code in PR #115: URL: https://github.com/apache/flink-table-store/pull/115#discussion_r869956416
########## flink-table-store-core/src/main/java/org/apache/flink/table/store/file/data/DataFileMeta.java: ########## @@ -49,6 +57,46 @@ public class DataFileMeta { private final long maxSequenceNumber; private final int level; + public static DataFileMeta forAppend( + String fileName, + long fileSize, + long rowCount, + FieldStats[] rowStats, + long minSequenceNumber, + long maxSequenceNumber) { + return new DataFileMeta( + fileName, + fileSize, + rowCount, + EMPTY_MIN_KEY, + EMPTY_MAX_KEY, + EMPTY_KEY_STATS, + rowStats, + minSequenceNumber, + maxSequenceNumber, + DUMMY_LEVEL); + } + + public DataFileMeta( Review Comment: Remove this method? ########## flink-table-store-core/src/main/java/org/apache/flink/table/store/file/mergetree/Increment.java: ########## @@ -35,12 +35,19 @@ */ public class Increment { + private static final List<DataFileMeta> EMPTY_COMPACT_BEFORE = Collections.emptyList(); + private static final List<DataFileMeta> EMPTY_COMPACT_AFTER = Collections.emptyList(); + private final List<DataFileMeta> newFiles; private final List<DataFileMeta> compactBefore; private final List<DataFileMeta> compactAfter; + public Increment(List<DataFileMeta> newFiles) { Review Comment: Ditto: introduce a `forAppend` ########## flink-table-store-core/src/main/java/org/apache/flink/table/store/file/writer/AppendOnlyWriter.java: ########## @@ -0,0 +1,168 @@ +/* + * 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 java.io.IOException; +import java.util.ArrayList; +import java.util.List; +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 long 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 = 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); Review Comment: We should avoid string concat here. Use ``` checkArgument( boolean condition, @Nullable String errorMessageTemplate, @Nullable Object... errorMessageArgs ``` -- 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